|
@@ -38,15 +38,13 @@ import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Trash;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
-import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
-import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
-import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
@@ -55,9 +53,6 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
|
|
|
-import org.apache.hadoop.ipc.RPC;
|
|
|
-import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
|
|
@@ -172,19 +167,18 @@ public class NameNode {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
-
|
|
|
public static final int DEFAULT_PORT = 8020;
|
|
|
-
|
|
|
public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
|
|
|
public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange");
|
|
|
public static final HAState ACTIVE_STATE = new ActiveState();
|
|
|
public static final HAState STANDBY_STATE = new StandbyState();
|
|
|
|
|
|
protected FSNamesystem namesystem;
|
|
|
+ protected final Configuration conf;
|
|
|
protected NamenodeRole role;
|
|
|
private HAState state;
|
|
|
private final boolean haEnabled;
|
|
|
+ private final HAContext haContext;
|
|
|
|
|
|
|
|
|
/** httpServer */
|
|
@@ -313,12 +307,11 @@ public class NameNode {
|
|
|
* Given a configuration get the address of the service rpc server
|
|
|
* If the service rpc is not configured returns null
|
|
|
*/
|
|
|
- protected InetSocketAddress getServiceRpcServerAddress(Configuration conf)
|
|
|
- throws IOException {
|
|
|
+ protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) {
|
|
|
return NameNode.getServiceAddress(conf, false);
|
|
|
}
|
|
|
|
|
|
- protected InetSocketAddress getRpcServerAddress(Configuration conf) throws IOException {
|
|
|
+ protected InetSocketAddress getRpcServerAddress(Configuration conf) {
|
|
|
return getAddress(conf);
|
|
|
}
|
|
|
|
|
@@ -396,7 +389,7 @@ public class NameNode {
|
|
|
throw e;
|
|
|
}
|
|
|
|
|
|
- activate(conf);
|
|
|
+ startCommonServices(conf);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -430,19 +423,11 @@ public class NameNode {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Activate name-node servers and threads.
|
|
|
- */
|
|
|
- void activate(Configuration conf) throws IOException {
|
|
|
- if ((isRole(NamenodeRole.NAMENODE))
|
|
|
- && (UserGroupInformation.isSecurityEnabled())) {
|
|
|
- namesystem.activateSecretManager();
|
|
|
- }
|
|
|
- namesystem.activate(conf);
|
|
|
+ /** Start the services common to active and standby states */
|
|
|
+ private void startCommonServices(Configuration conf) throws IOException {
|
|
|
+ namesystem.startCommonServices(conf);
|
|
|
startHttpServer(conf);
|
|
|
rpcServer.start();
|
|
|
- startTrashEmptier(conf);
|
|
|
-
|
|
|
plugins = conf.getInstances(DFS_NAMENODE_PLUGINS_KEY,
|
|
|
ServicePlugin.class);
|
|
|
for (ServicePlugin p: plugins) {
|
|
@@ -452,13 +437,28 @@ public class NameNode {
|
|
|
LOG.warn("ServicePlugin " + p + " could not be started", t);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
LOG.info(getRole() + " up at: " + rpcServer.getRpcAddress());
|
|
|
if (rpcServer.getServiceRpcAddress() != null) {
|
|
|
- LOG.info(getRole() + " service server is up at: " + rpcServer.getServiceRpcAddress());
|
|
|
+ LOG.info(getRole() + " service server is up at: "
|
|
|
+ + rpcServer.getServiceRpcAddress());
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ private void stopCommonServices() {
|
|
|
+ if(namesystem != null) namesystem.close();
|
|
|
+ if(rpcServer != null) rpcServer.stop();
|
|
|
+ if (plugins != null) {
|
|
|
+ for (ServicePlugin p : plugins) {
|
|
|
+ try {
|
|
|
+ p.stop();
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.warn("ServicePlugin " + p + " could not be stopped", t);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ stopHttpServer();
|
|
|
+ }
|
|
|
+
|
|
|
private void startTrashEmptier(Configuration conf) throws IOException {
|
|
|
long trashInterval
|
|
|
= conf.getLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY,
|
|
@@ -470,11 +470,26 @@ public class NameNode {
|
|
|
this.emptier.start();
|
|
|
}
|
|
|
|
|
|
+ private void stopTrashEmptier() {
|
|
|
+ if (this.emptier != null) {
|
|
|
+ emptier.interrupt();
|
|
|
+ emptier = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private void startHttpServer(final Configuration conf) throws IOException {
|
|
|
httpServer = new NameNodeHttpServer(conf, this, getHttpServerAddress(conf));
|
|
|
httpServer.start();
|
|
|
setHttpServerAddress(conf);
|
|
|
}
|
|
|
+
|
|
|
+ private void stopHttpServer() {
|
|
|
+ try {
|
|
|
+ if (httpServer != null) httpServer.stop();
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("Exception while stopping httpserver", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Start NameNode.
|
|
@@ -509,18 +524,28 @@ public class NameNode {
|
|
|
|
|
|
protected NameNode(Configuration conf, NamenodeRole role)
|
|
|
throws IOException {
|
|
|
+ this.conf = conf;
|
|
|
this.role = role;
|
|
|
this.haEnabled = DFSUtil.isHAEnabled(conf);
|
|
|
- this.state = !haEnabled ? ACTIVE_STATE : STANDBY_STATE;
|
|
|
+ this.haContext = new NameNodeHAContext();
|
|
|
try {
|
|
|
initializeGenericKeys(conf, getNameServiceId(conf));
|
|
|
initialize(conf);
|
|
|
+ if (!haEnabled) {
|
|
|
+ state = ACTIVE_STATE;
|
|
|
+ } else {
|
|
|
+ state = STANDBY_STATE;;
|
|
|
+ }
|
|
|
+ state.enterState(haContext);
|
|
|
} catch (IOException e) {
|
|
|
this.stop();
|
|
|
throw e;
|
|
|
} catch (HadoopIllegalArgumentException e) {
|
|
|
this.stop();
|
|
|
throw e;
|
|
|
+ } catch (ServiceFailedException e) {
|
|
|
+ this.stop();
|
|
|
+ throw new IOException("Service failed to start", e);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -532,6 +557,7 @@ public class NameNode {
|
|
|
try {
|
|
|
this.rpcServer.join();
|
|
|
} catch (InterruptedException ie) {
|
|
|
+ LOG.info("Caught interrupted exception ", ie);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -544,23 +570,12 @@ public class NameNode {
|
|
|
return;
|
|
|
stopRequested = true;
|
|
|
}
|
|
|
- if (plugins != null) {
|
|
|
- for (ServicePlugin p : plugins) {
|
|
|
- try {
|
|
|
- p.stop();
|
|
|
- } catch (Throwable t) {
|
|
|
- LOG.warn("ServicePlugin " + p + " could not be stopped", t);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
try {
|
|
|
- if (httpServer != null) httpServer.stop();
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Exception while stopping httpserver", e);
|
|
|
+ state.exitState(haContext);
|
|
|
+ } catch (ServiceFailedException e) {
|
|
|
+ LOG.warn("Encountered exception while exiting state ", e);
|
|
|
}
|
|
|
- if(namesystem != null) namesystem.close();
|
|
|
- if(emptier != null) emptier.interrupt();
|
|
|
- if(rpcServer != null) rpcServer.stop();
|
|
|
+ stopCommonServices();
|
|
|
if (metrics != null) {
|
|
|
metrics.shutdown();
|
|
|
}
|
|
@@ -876,27 +891,61 @@ public class NameNode {
|
|
|
if (!haEnabled) {
|
|
|
throw new ServiceFailedException("HA for namenode is not enabled");
|
|
|
}
|
|
|
- state.setState(this, ACTIVE_STATE);
|
|
|
+ state.setState(haContext, ACTIVE_STATE);
|
|
|
}
|
|
|
|
|
|
synchronized void transitionToStandby() throws ServiceFailedException {
|
|
|
if (!haEnabled) {
|
|
|
throw new ServiceFailedException("HA for namenode is not enabled");
|
|
|
}
|
|
|
- state.setState(this, STANDBY_STATE);
|
|
|
+ state.setState(haContext, STANDBY_STATE);
|
|
|
}
|
|
|
|
|
|
/** Check if an operation of given category is allowed */
|
|
|
protected synchronized void checkOperation(final OperationCategory op)
|
|
|
throws UnsupportedActionException {
|
|
|
- state.checkOperation(this, op);
|
|
|
+ state.checkOperation(haContext, op);
|
|
|
}
|
|
|
|
|
|
- public synchronized HAState getState() {
|
|
|
- return state;
|
|
|
- }
|
|
|
-
|
|
|
- public synchronized void setState(final HAState s) {
|
|
|
- state = s;
|
|
|
+ /**
|
|
|
+ * Class used as expose {@link NameNode} as context to {@link HAState}
|
|
|
+ *
|
|
|
+ * TODO:HA
|
|
|
+ * When entering and exiting state, on failing to start services,
|
|
|
+ * appropriate action is needed todo either shutdown the node or recover
|
|
|
+ * from failure.
|
|
|
+ */
|
|
|
+ private class NameNodeHAContext implements HAContext {
|
|
|
+ @Override
|
|
|
+ public void setState(HAState s) {
|
|
|
+ state = s;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public HAState getState() {
|
|
|
+ return state;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void startActiveServices() throws IOException {
|
|
|
+ namesystem.startActiveServices();
|
|
|
+ startTrashEmptier(conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void stopActiveServices() throws IOException {
|
|
|
+ namesystem.stopActiveServices();
|
|
|
+ stopTrashEmptier();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void startStandbyServices() throws IOException {
|
|
|
+ // TODO:HA Start reading editlog from active
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void stopStandbyServices() throws IOException {
|
|
|
+ // TODO:HA Stop reading editlog from active
|
|
|
+ }
|
|
|
}
|
|
|
}
|