|
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.HAUtil;
|
|
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
|
+import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.proto.HAZKInfoProtos.ActiveNodeInfo;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
@@ -42,10 +43,9 @@ import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.security.authorize.PolicyProvider;
|
|
|
+import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
-import org.apache.hadoop.util.ToolRunner;
|
|
|
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
import com.google.protobuf.InvalidProtocolBufferException;
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
@@ -53,9 +53,9 @@ public class DFSZKFailoverController extends ZKFailoverController {
|
|
|
|
|
|
private static final Log LOG =
|
|
|
LogFactory.getLog(DFSZKFailoverController.class);
|
|
|
- private NNHAServiceTarget localTarget;
|
|
|
- private Configuration localNNConf;
|
|
|
private AccessControlList adminAcl;
|
|
|
+ /* the same as superclass's localTarget, but with the more specfic NN type */
|
|
|
+ private final NNHAServiceTarget localNNTarget;
|
|
|
|
|
|
@Override
|
|
|
protected HAServiceTarget dataToTarget(byte[] data) {
|
|
@@ -67,7 +67,7 @@ public class DFSZKFailoverController extends ZKFailoverController {
|
|
|
StringUtils.byteToHexString(data));
|
|
|
}
|
|
|
NNHAServiceTarget ret = new NNHAServiceTarget(
|
|
|
- getConf(), proto.getNameserviceId(), proto.getNamenodeId());
|
|
|
+ conf, proto.getNameserviceId(), proto.getNamenodeId());
|
|
|
InetSocketAddress addressFromProtobuf = new InetSocketAddress(
|
|
|
proto.getHostname(), proto.getPort());
|
|
|
|
|
@@ -89,15 +89,15 @@ public class DFSZKFailoverController extends ZKFailoverController {
|
|
|
.setHostname(addr.getHostName())
|
|
|
.setPort(addr.getPort())
|
|
|
.setZkfcPort(target.getZKFCAddress().getPort())
|
|
|
- .setNameserviceId(localTarget.getNameServiceId())
|
|
|
- .setNamenodeId(localTarget.getNameNodeId())
|
|
|
+ .setNameserviceId(localNNTarget.getNameServiceId())
|
|
|
+ .setNamenodeId(localNNTarget.getNameNodeId())
|
|
|
.build()
|
|
|
.toByteArray();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected InetSocketAddress getRpcAddressToBindTo() {
|
|
|
- int zkfcPort = getZkfcPort(localNNConf);
|
|
|
+ int zkfcPort = getZkfcPort(conf);
|
|
|
return new InetSocketAddress(localTarget.getAddress().getAddress(),
|
|
|
zkfcPort);
|
|
|
}
|
|
@@ -112,10 +112,9 @@ public class DFSZKFailoverController extends ZKFailoverController {
|
|
|
return conf.getInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY,
|
|
|
DFSConfigKeys.DFS_HA_ZKFC_PORT_DEFAULT);
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public void setConf(Configuration conf) {
|
|
|
- localNNConf = DFSHAAdmin.addSecurityConfiguration(conf);
|
|
|
+
|
|
|
+ public static DFSZKFailoverController create(Configuration conf) {
|
|
|
+ Configuration localNNConf = DFSHAAdmin.addSecurityConfiguration(conf);
|
|
|
String nsId = DFSUtil.getNamenodeNameServiceId(conf);
|
|
|
|
|
|
if (!HAUtil.isHAEnabled(localNNConf, nsId)) {
|
|
@@ -126,47 +125,50 @@ public class DFSZKFailoverController extends ZKFailoverController {
|
|
|
NameNode.initializeGenericKeys(localNNConf, nsId, nnId);
|
|
|
DFSUtil.setGenericConf(localNNConf, nsId, nnId, ZKFC_CONF_KEYS);
|
|
|
|
|
|
- localTarget = new NNHAServiceTarget(localNNConf, nsId, nnId);
|
|
|
-
|
|
|
+ NNHAServiceTarget localTarget = new NNHAServiceTarget(
|
|
|
+ localNNConf, nsId, nnId);
|
|
|
+ return new DFSZKFailoverController(localNNConf, localTarget);
|
|
|
+ }
|
|
|
+
|
|
|
+ private DFSZKFailoverController(Configuration conf,
|
|
|
+ NNHAServiceTarget localTarget) {
|
|
|
+ super(conf, localTarget);
|
|
|
+ this.localNNTarget = localTarget;
|
|
|
// Setup ACLs
|
|
|
adminAcl = new AccessControlList(
|
|
|
conf.get(DFSConfigKeys.DFS_ADMIN, " "));
|
|
|
-
|
|
|
- super.setConf(localNNConf);
|
|
|
LOG.info("Failover controller configured for NameNode " +
|
|
|
- nsId + "." + nnId);
|
|
|
- }
|
|
|
+ localTarget);
|
|
|
+}
|
|
|
|
|
|
|
|
|
@Override
|
|
|
protected void initRPC() throws IOException {
|
|
|
super.initRPC();
|
|
|
- localTarget.setZkfcPort(rpcServer.getAddress().getPort());
|
|
|
+ localNNTarget.setZkfcPort(rpcServer.getAddress().getPort());
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public HAServiceTarget getLocalTarget() {
|
|
|
- Preconditions.checkState(localTarget != null,
|
|
|
- "setConf() should have already been called");
|
|
|
- return localTarget;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public void loginAsFCUser() throws IOException {
|
|
|
- InetSocketAddress socAddr = NameNode.getAddress(localNNConf);
|
|
|
- SecurityUtil.login(getConf(), DFS_NAMENODE_KEYTAB_FILE_KEY,
|
|
|
+ InetSocketAddress socAddr = NameNode.getAddress(conf);
|
|
|
+ SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY,
|
|
|
DFS_NAMENODE_USER_NAME_KEY, socAddr.getHostName());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected String getScopeInsideParentNode() {
|
|
|
- return localTarget.getNameServiceId();
|
|
|
+ return localNNTarget.getNameServiceId();
|
|
|
}
|
|
|
|
|
|
public static void main(String args[])
|
|
|
throws Exception {
|
|
|
- System.exit(ToolRunner.run(
|
|
|
- new DFSZKFailoverController(), args));
|
|
|
+
|
|
|
+ GenericOptionsParser parser = new GenericOptionsParser(
|
|
|
+ new HdfsConfiguration(), args);
|
|
|
+ DFSZKFailoverController zkfc = DFSZKFailoverController.create(
|
|
|
+ parser.getConfiguration());
|
|
|
+
|
|
|
+ System.exit(zkfc.run(parser.getRemainingArgs()));
|
|
|
}
|
|
|
|
|
|
@Override
|