diff --git a/hadoop-common-project/hadoop-common/CHANGES.HDFS-1623.txt b/hadoop-common-project/hadoop-common/CHANGES.HDFS-1623.txt
new file mode 100644
index 0000000000..748ff939ce
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/CHANGES.HDFS-1623.txt
@@ -0,0 +1,53 @@
+Changes for HDFS-1623 branch.
+
+This change list will be merged into the trunk CHANGES.txt when the HDFS-1623
+branch is merged.
+------------------------------
+
+HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
+
+HADOOP-7774. HA: Administrative CLI to control HA daemons. (todd)
+
+HADOOP-7896. HA: if both NNs are in Standby mode, client needs to try failing
+back and forth several times with sleeps. (atm)
+
+HADOOP-7922. Improve some logging for client IPC failovers and
+StandbyExceptions (todd)
+
+HADOOP-7921. StandbyException should extend IOException (todd)
+
+HADOOP-7928. HA: Client failover policy is incorrectly trying to fail over all
+IOExceptions (atm)
+
+HADOOP-7925. Add interface and update CLI to query current state to
+HAServiceProtocol (eli via todd)
+
+HADOOP-7932. Make client connection retries on socket time outs configurable.
+(Uma Maheswara Rao G via todd)
+
+HADOOP-7924. FailoverController for client-based configuration (eli)
+
+HADOOP-7961. Move HA fencing to common. (eli)
+
+HADOOP-7970. HAServiceProtocol methods must throw IOException.
+(Hari Mankude via suresh).
+
+HADOOP-7992. Add ZKClient library to facilitate leader election.
+(Bikas Saha via suresh).
+
+HADOOP-7983. HA: failover should be able to pass args to fencers. (eli)
+
+HADOOP-7938. HA: the FailoverController should optionally fence the
+active during failover. (eli)
+
+HADOOP-7991. HA: the FailoverController should check the standby is
+ready before failing over. (eli)
+
+HADOOP-8038. Add 'ipc.client.connect.max.retries.on.timeouts' entry in
+core-default.xml file. (Uma Maheswara Rao G via atm)
+
+HADOOP-8041. Log a warning when a failover is first attempted (todd)
+
+HADOOP-8068. void methods can swallow exceptions when going through failover path (todd)
+
+HADOOP-8116. RetriableCommand is using RetryPolicy incorrectly after HADOOP-7896. (atm)
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index 3624c99871..855b028453 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -278,8 +278,12 @@
-
+
+
+
+
+
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 12d98c6e90..fd18b607a2 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -263,6 +263,38 @@
json-simplecompile
+
+ com.jcraft
+ jsch
+
+
+
+ org.apache.zookeeper
+ zookeeper
+ 3.4.2
+
+
+
+ junit
+ junit
+
+
+ com.sun.jdmk
+ jmxtools
+
+
+ com.sun.jmx
+ jmxri
+
+
+
+
+ org.apache.zookeeper
+ zookeeper
+ 3.4.2
+ test-jar
+ test
+
diff --git a/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/service_level_auth.xml b/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/service_level_auth.xml
index b8f5f511d3..771ac052b3 100644
--- a/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/service_level_auth.xml
+++ b/hadoop-common-project/hadoop-common/src/main/docs/src/documentation/content/xdocs/service_level_auth.xml
@@ -138,6 +138,12 @@
dfsadmin and mradmin commands to refresh the security policy in-effect.
+
+
security.ha.service.protocol.acl
+
ACL for HAService protocol used by HAAdmin to manage the
+ active and stand-by states of namenode.
+
+
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index f0ca72b00e..c2a6479dd2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -114,11 +114,12 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
public static final String
HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_USER_MAPPINGS =
"security.refresh.user.mappings.protocol.acl";
+ public static final String
+ SECURITY_HA_SERVICE_PROTOCOL_ACL = "security.ha.service.protocol.acl";
public static final String HADOOP_SECURITY_TOKEN_SERVICE_USE_IP =
"hadoop.security.token.service.use_ip";
public static final boolean HADOOP_SECURITY_TOKEN_SERVICE_USE_IP_DEFAULT =
true;
-
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 401d07ab11..7953411b57 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -172,6 +172,11 @@ public class CommonConfigurationKeysPublic {
/** Default value for IPC_CLIENT_CONNECT_MAX_RETRIES_KEY */
public static final int IPC_CLIENT_CONNECT_MAX_RETRIES_DEFAULT = 10;
/** See core-default.xml */
+ public static final String IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY =
+ "ipc.client.connect.max.retries.on.timeouts";
+ /** Default value for IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY */
+ public static final int IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 45;
+ /** See core-default.xml */
public static final String IPC_CLIENT_TCPNODELAY_KEY =
"ipc.client.tcpnodelay";
/** Defalt value for IPC_CLIENT_TCPNODELAY_KEY */
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
new file mode 100644
index 0000000000..7da2d3e1bf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
@@ -0,0 +1,593 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.AsyncCallback.*;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.KeeperException.Code;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ *
+ * This class implements a simple library to perform leader election on top of
+ * Apache Zookeeper. Using Zookeeper as a coordination service, leader election
+ * can be performed by atomically creating an ephemeral lock file (znode) on
+ * Zookeeper. The service instance that successfully creates the znode becomes
+ * active and the rest become standbys.
+ * This election mechanism is only efficient for small number of election
+ * candidates (order of 10's) because contention on single znode by a large
+ * number of candidates can result in Zookeeper overload.
+ * The elector does not guarantee fencing (protection of shared resources) among
+ * service instances. After it has notified an instance about becoming a leader,
+ * then that instance must ensure that it meets the service consistency
+ * requirements. If it cannot do so, then it is recommended to quit the
+ * election. The application implements the {@link ActiveStandbyElectorCallback}
+ * to interact with the elector
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ActiveStandbyElector implements Watcher, StringCallback,
+ StatCallback {
+
+ /**
+ * Callback interface to interact with the ActiveStandbyElector object.
+ * The application will be notified with a callback only on state changes
+ * (i.e. there will never be successive calls to becomeActive without an
+ * intermediate call to enterNeutralMode).
+ * The callbacks will be running on Zookeeper client library threads. The
+ * application should return from these callbacks quickly so as not to impede
+ * Zookeeper client library performance and notifications. The app will
+ * typically remember the state change and return from the callback. It will
+ * then proceed with implementing actions around that state change. It is
+ * possible to be called back again while these actions are in flight and the
+ * app should handle this scenario.
+ */
+ public interface ActiveStandbyElectorCallback {
+ /**
+ * This method is called when the app becomes the active leader
+ */
+ void becomeActive();
+
+ /**
+ * This method is called when the app becomes a standby
+ */
+ void becomeStandby();
+
+ /**
+ * If the elector gets disconnected from Zookeeper and does not know about
+ * the lock state, then it will notify the service via the enterNeutralMode
+ * interface. The service may choose to ignore this or stop doing state
+ * changing operations. Upon reconnection, the elector verifies the leader
+ * status and calls back on the becomeActive and becomeStandby app
+ * interfaces.
+ * Zookeeper disconnects can happen due to network issues or loss of
+ * Zookeeper quorum. Thus enterNeutralMode can be used to guard against
+ * split-brain issues. In such situations it might be prudent to call
+ * becomeStandby too. However, such state change operations might be
+ * expensive and enterNeutralMode can help guard against doing that for
+ * transient issues.
+ */
+ void enterNeutralMode();
+
+ /**
+ * If there is any fatal error (e.g. wrong ACL's, unexpected Zookeeper
+ * errors or Zookeeper persistent unavailability) then notifyFatalError is
+ * called to notify the app about it.
+ */
+ void notifyFatalError(String errorMessage);
+ }
+
+ /**
+ * Name of the lock znode used by the library. Protected for access in test
+ * classes
+ */
+ @VisibleForTesting
+ protected static final String LOCKFILENAME = "ActiveStandbyElectorLock";
+
+ public static final Log LOG = LogFactory.getLog(ActiveStandbyElector.class);
+
+ private static final int NUM_RETRIES = 3;
+
+ private enum ConnectionState {
+ DISCONNECTED, CONNECTED, TERMINATED
+ };
+
+ private enum State {
+ INIT, ACTIVE, STANDBY, NEUTRAL
+ };
+
+ private State state = State.INIT;
+ private int createRetryCount = 0;
+ private int statRetryCount = 0;
+ private ZooKeeper zkClient;
+ private ConnectionState zkConnectionState = ConnectionState.TERMINATED;
+
+ private final ActiveStandbyElectorCallback appClient;
+ private final String zkHostPort;
+ private final int zkSessionTimeout;
+ private final List zkAcl;
+ private byte[] appData;
+ private final String zkLockFilePath;
+ private final String znodeWorkingDir;
+
+ /**
+ * Create a new ActiveStandbyElector object
+ * The elector is created by providing to it the Zookeeper configuration, the
+ * parent znode under which to create the znode and a reference to the
+ * callback interface.
+ * The parent znode name must be the same for all service instances and
+ * different across services.
+ * After the leader has been lost, a new leader will be elected after the
+ * session timeout expires. Hence, the app must set this parameter based on
+ * its needs for failure response time. The session timeout must be greater
+ * than the Zookeeper disconnect timeout and is recommended to be 3X that
+ * value to enable Zookeeper to retry transient disconnections. Setting a very
+ * short session timeout may result in frequent transitions between active and
+ * standby states during issues like network outages/GS pauses.
+ *
+ * @param zookeeperHostPorts
+ * ZooKeeper hostPort for all ZooKeeper servers
+ * @param zookeeperSessionTimeout
+ * ZooKeeper session timeout
+ * @param parentZnodeName
+ * znode under which to create the lock
+ * @param acl
+ * ZooKeeper ACL's
+ * @param app
+ * reference to callback interface object
+ * @throws IOException
+ * @throws HadoopIllegalArgumentException
+ */
+ public ActiveStandbyElector(String zookeeperHostPorts,
+ int zookeeperSessionTimeout, String parentZnodeName, List acl,
+ ActiveStandbyElectorCallback app) throws IOException,
+ HadoopIllegalArgumentException {
+ if (app == null || acl == null || parentZnodeName == null
+ || zookeeperHostPorts == null || zookeeperSessionTimeout <= 0) {
+ throw new HadoopIllegalArgumentException("Invalid argument");
+ }
+ zkHostPort = zookeeperHostPorts;
+ zkSessionTimeout = zookeeperSessionTimeout;
+ zkAcl = acl;
+ appClient = app;
+ znodeWorkingDir = parentZnodeName;
+ zkLockFilePath = znodeWorkingDir + "/" + LOCKFILENAME;
+
+ // createConnection for future API calls
+ createConnection();
+ }
+
+ /**
+ * To participate in election, the app will call joinElection. The result will
+ * be notified by a callback on either the becomeActive or becomeStandby app
+ * interfaces.
+ * After this the elector will automatically monitor the leader status and
+ * perform re-election if necessary
+ * The app could potentially start off in standby mode and ignore the
+ * becomeStandby call.
+ *
+ * @param data
+ * to be set by the app. non-null data must be set.
+ * @throws HadoopIllegalArgumentException
+ * if valid data is not supplied
+ */
+ public synchronized void joinElection(byte[] data)
+ throws HadoopIllegalArgumentException {
+ LOG.debug("Attempting active election");
+
+ if (data == null) {
+ throw new HadoopIllegalArgumentException("data cannot be null");
+ }
+
+ appData = new byte[data.length];
+ System.arraycopy(data, 0, appData, 0, data.length);
+
+ joinElectionInternal();
+ }
+
+ /**
+ * Any service instance can drop out of the election by calling quitElection.
+ *
+ * This will lose any leader status, if held, and stop monitoring of the lock
+ * node.
+ * If the instance wants to participate in election again, then it needs to
+ * call joinElection().
+ * This allows service instances to take themselves out of rotation for known
+ * impending unavailable states (e.g. long GC pause or software upgrade).
+ */
+ public synchronized void quitElection() {
+ LOG.debug("Yielding from election");
+ reset();
+ }
+
+ /**
+ * Exception thrown when there is no active leader
+ */
+ public static class ActiveNotFoundException extends Exception {
+ private static final long serialVersionUID = 3505396722342846462L;
+ }
+
+ /**
+ * get data set by the active leader
+ *
+ * @return data set by the active instance
+ * @throws ActiveNotFoundException
+ * when there is no active leader
+ * @throws KeeperException
+ * other zookeeper operation errors
+ * @throws InterruptedException
+ * @throws IOException
+ * when ZooKeeper connection could not be established
+ */
+ public synchronized byte[] getActiveData() throws ActiveNotFoundException,
+ KeeperException, InterruptedException, IOException {
+ try {
+ if (zkClient == null) {
+ createConnection();
+ }
+ Stat stat = new Stat();
+ return zkClient.getData(zkLockFilePath, false, stat);
+ } catch(KeeperException e) {
+ Code code = e.code();
+ if (operationNodeDoesNotExist(code)) {
+ // handle the commonly expected cases that make sense for us
+ throw new ActiveNotFoundException();
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ /**
+ * interface implementation of Zookeeper callback for create
+ */
+ @Override
+ public synchronized void processResult(int rc, String path, Object ctx,
+ String name) {
+ LOG.debug("CreateNode result: " + rc + " for path: " + path
+ + " connectionState: " + zkConnectionState);
+ if (zkClient == null) {
+ // zkClient is nulled before closing the connection
+ // this is the callback with session expired after we closed the session
+ return;
+ }
+
+ Code code = Code.get(rc);
+ if (operationSuccess(code)) {
+ // we successfully created the znode. we are the leader. start monitoring
+ becomeActive();
+ monitorActiveStatus();
+ return;
+ }
+
+ if (operationNodeExists(code)) {
+ if (createRetryCount == 0) {
+ // znode exists and we did not retry the operation. so a different
+ // instance has created it. become standby and monitor lock.
+ becomeStandby();
+ }
+ // if we had retried then the znode could have been created by our first
+ // attempt to the server (that we lost) and this node exists response is
+ // for the second attempt. verify this case via ephemeral node owner. this
+ // will happen on the callback for monitoring the lock.
+ monitorActiveStatus();
+ return;
+ }
+
+ String errorMessage = "Received create error from Zookeeper. code:"
+ + code.toString();
+ LOG.debug(errorMessage);
+
+ if (operationRetry(code)) {
+ if (createRetryCount < NUM_RETRIES) {
+ LOG.debug("Retrying createNode createRetryCount: " + createRetryCount);
+ ++createRetryCount;
+ createNode();
+ return;
+ }
+ errorMessage = errorMessage
+ + ". Not retrying further znode create connection errors.";
+ }
+
+ fatalError(errorMessage);
+ }
+
+ /**
+ * interface implementation of Zookeeper callback for monitor (exists)
+ */
+ @Override
+ public synchronized void processResult(int rc, String path, Object ctx,
+ Stat stat) {
+ LOG.debug("StatNode result: " + rc + " for path: " + path
+ + " connectionState: " + zkConnectionState);
+ if (zkClient == null) {
+ // zkClient is nulled before closing the connection
+ // this is the callback with session expired after we closed the session
+ return;
+ }
+
+ Code code = Code.get(rc);
+ if (operationSuccess(code)) {
+ // the following owner check completes verification in case the lock znode
+ // creation was retried
+ if (stat.getEphemeralOwner() == zkClient.getSessionId()) {
+ // we own the lock znode. so we are the leader
+ becomeActive();
+ } else {
+ // we dont own the lock znode. so we are a standby.
+ becomeStandby();
+ }
+ // the watch set by us will notify about changes
+ return;
+ }
+
+ if (operationNodeDoesNotExist(code)) {
+ // the lock znode disappeared before we started monitoring it
+ enterNeutralMode();
+ joinElectionInternal();
+ return;
+ }
+
+ String errorMessage = "Received stat error from Zookeeper. code:"
+ + code.toString();
+ LOG.debug(errorMessage);
+
+ if (operationRetry(code)) {
+ if (statRetryCount < NUM_RETRIES) {
+ ++statRetryCount;
+ monitorNode();
+ return;
+ }
+ errorMessage = errorMessage
+ + ". Not retrying further znode monitoring connection errors.";
+ }
+
+ fatalError(errorMessage);
+ }
+
+ /**
+ * interface implementation of Zookeeper watch events (connection and node)
+ */
+ @Override
+ public synchronized void process(WatchedEvent event) {
+ Event.EventType eventType = event.getType();
+ LOG.debug("Watcher event type: " + eventType + " with state:"
+ + event.getState() + " for path:" + event.getPath()
+ + " connectionState: " + zkConnectionState);
+ if (zkClient == null) {
+ // zkClient is nulled before closing the connection
+ // this is the callback with session expired after we closed the session
+ return;
+ }
+
+ if (eventType == Event.EventType.None) {
+ // the connection state has changed
+ switch (event.getState()) {
+ case SyncConnected:
+ // if the listener was asked to move to safe state then it needs to
+ // be undone
+ ConnectionState prevConnectionState = zkConnectionState;
+ zkConnectionState = ConnectionState.CONNECTED;
+ if (prevConnectionState == ConnectionState.DISCONNECTED) {
+ monitorActiveStatus();
+ }
+ break;
+ case Disconnected:
+ // ask the app to move to safe state because zookeeper connection
+ // is not active and we dont know our state
+ zkConnectionState = ConnectionState.DISCONNECTED;
+ enterNeutralMode();
+ break;
+ case Expired:
+ // the connection got terminated because of session timeout
+ // call listener to reconnect
+ enterNeutralMode();
+ reJoinElection();
+ break;
+ default:
+ fatalError("Unexpected Zookeeper watch event state: "
+ + event.getState());
+ break;
+ }
+
+ return;
+ }
+
+ // a watch on lock path in zookeeper has fired. so something has changed on
+ // the lock. ideally we should check that the path is the same as the lock
+ // path but trusting zookeeper for now
+ String path = event.getPath();
+ if (path != null) {
+ switch (eventType) {
+ case NodeDeleted:
+ if (state == State.ACTIVE) {
+ enterNeutralMode();
+ }
+ joinElectionInternal();
+ break;
+ case NodeDataChanged:
+ monitorActiveStatus();
+ break;
+ default:
+ LOG.debug("Unexpected node event: " + eventType + " for path: " + path);
+ monitorActiveStatus();
+ }
+
+ return;
+ }
+
+ // some unexpected error has occurred
+ fatalError("Unexpected watch error from Zookeeper");
+ }
+
+ /**
+ * Get a new zookeeper client instance. protected so that test class can
+ * inherit and pass in a mock object for zookeeper
+ *
+ * @return new zookeeper client instance
+ * @throws IOException
+ */
+ protected synchronized ZooKeeper getNewZooKeeper() throws IOException {
+ return new ZooKeeper(zkHostPort, zkSessionTimeout, this);
+ }
+
+ private void fatalError(String errorMessage) {
+ reset();
+ appClient.notifyFatalError(errorMessage);
+ }
+
+ private void monitorActiveStatus() {
+ LOG.debug("Monitoring active leader");
+ statRetryCount = 0;
+ monitorNode();
+ }
+
+ private void joinElectionInternal() {
+ if (zkClient == null) {
+ if (!reEstablishSession()) {
+ fatalError("Failed to reEstablish connection with ZooKeeper");
+ return;
+ }
+ }
+
+ createRetryCount = 0;
+ createNode();
+ }
+
+ private void reJoinElection() {
+ LOG.debug("Trying to re-establish ZK session");
+ terminateConnection();
+ joinElectionInternal();
+ }
+
+ private boolean reEstablishSession() {
+ int connectionRetryCount = 0;
+ boolean success = false;
+ while(!success && connectionRetryCount < NUM_RETRIES) {
+ LOG.debug("Establishing zookeeper connection");
+ try {
+ createConnection();
+ success = true;
+ } catch(IOException e) {
+ LOG.warn(e);
+ try {
+ Thread.sleep(5000);
+ } catch(InterruptedException e1) {
+ LOG.warn(e1);
+ }
+ }
+ ++connectionRetryCount;
+ }
+ return success;
+ }
+
+ private void createConnection() throws IOException {
+ zkClient = getNewZooKeeper();
+ }
+
+ private void terminateConnection() {
+ if (zkClient == null) {
+ return;
+ }
+ LOG.debug("Terminating ZK connection");
+ ZooKeeper tempZk = zkClient;
+ zkClient = null;
+ try {
+ tempZk.close();
+ } catch(InterruptedException e) {
+ LOG.warn(e);
+ }
+ zkConnectionState = ConnectionState.TERMINATED;
+ }
+
+ private void reset() {
+ state = State.INIT;
+ terminateConnection();
+ }
+
+ private void becomeActive() {
+ if (state != State.ACTIVE) {
+ LOG.debug("Becoming active");
+ state = State.ACTIVE;
+ appClient.becomeActive();
+ }
+ }
+
+ private void becomeStandby() {
+ if (state != State.STANDBY) {
+ LOG.debug("Becoming standby");
+ state = State.STANDBY;
+ appClient.becomeStandby();
+ }
+ }
+
+ private void enterNeutralMode() {
+ if (state != State.NEUTRAL) {
+ LOG.debug("Entering neutral mode");
+ state = State.NEUTRAL;
+ appClient.enterNeutralMode();
+ }
+ }
+
+ private void createNode() {
+ zkClient.create(zkLockFilePath, appData, zkAcl, CreateMode.EPHEMERAL, this,
+ null);
+ }
+
+ private void monitorNode() {
+ zkClient.exists(zkLockFilePath, true, this, null);
+ }
+
+ private boolean operationSuccess(Code code) {
+ return (code == Code.OK);
+ }
+
+ private boolean operationNodeExists(Code code) {
+ return (code == Code.NODEEXISTS);
+ }
+
+ private boolean operationNodeDoesNotExist(Code code) {
+ return (code == Code.NONODE);
+ }
+
+ private boolean operationRetry(Code code) {
+ switch (code) {
+ case CONNECTIONLOSS:
+ case OPERATIONTIMEOUT:
+ return true;
+ }
+ return false;
+ }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/BadFencingConfigurationException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/BadFencingConfigurationException.java
new file mode 100644
index 0000000000..3d3b1ba53c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/BadFencingConfigurationException.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+
+/**
+ * Indicates that the operator has specified an invalid configuration
+ * for fencing methods.
+ */
+class BadFencingConfigurationException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+ public BadFencingConfigurationException(String msg) {
+ super(msg);
+ }
+
+ public BadFencingConfigurationException(String msg, Throwable cause) {
+ super(msg, cause);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java
new file mode 100644
index 0000000000..0960fb7cbd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverController.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The FailOverController is responsible for electing an active service
+ * on startup or when the current active is changing (eg due to failure),
+ * monitoring the health of a service, and performing a fail-over when a
+ * new active service is either manually selected by a user or elected.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class FailoverController {
+
+ private static final Log LOG = LogFactory.getLog(FailoverController.class);
+
+ /**
+ * Perform pre-failover checks on the given service we plan to
+ * failover to, eg to prevent failing over to a service (eg due
+ * to it being inaccessible, already active, not healthy, etc).
+ *
+ * An option to ignore toSvc if it claims it is not ready to
+ * become active is provided in case performing a failover will
+ * allow it to become active, eg because it triggers a log roll
+ * so the standby can learn about new blocks and leave safemode.
+ *
+ * @param toSvc service to make active
+ * @param toSvcName name of service to make active
+ * @param forceActive ignore toSvc if it reports that it is not ready
+ * @throws FailoverFailedException if we should avoid failover
+ */
+ private static void preFailoverChecks(HAServiceProtocol toSvc,
+ InetSocketAddress toSvcAddr,
+ boolean forceActive)
+ throws FailoverFailedException {
+ HAServiceState toSvcState;
+
+ try {
+ toSvcState = toSvc.getServiceState();
+ } catch (IOException e) {
+ String msg = "Unable to get service state for " + toSvcAddr;
+ LOG.error(msg, e);
+ throw new FailoverFailedException(msg, e);
+ }
+
+ if (!toSvcState.equals(HAServiceState.STANDBY)) {
+ throw new FailoverFailedException(
+ "Can't failover to an active service");
+ }
+
+ try {
+ HAServiceProtocolHelper.monitorHealth(toSvc);
+ } catch (HealthCheckFailedException hce) {
+ throw new FailoverFailedException(
+ "Can't failover to an unhealthy service", hce);
+ } catch (IOException e) {
+ throw new FailoverFailedException(
+ "Got an IO exception", e);
+ }
+
+ try {
+ if (!toSvc.readyToBecomeActive()) {
+ if (!forceActive) {
+ throw new FailoverFailedException(
+ toSvcAddr + " is not ready to become active");
+ }
+ }
+ } catch (IOException e) {
+ throw new FailoverFailedException(
+ "Got an IO exception", e);
+ }
+ }
+
+ /**
+ * Failover from service 1 to service 2. If the failover fails
+ * then try to failback.
+ *
+ * @param fromSvc currently active service
+ * @param fromSvcAddr addr of the currently active service
+ * @param toSvc service to make active
+ * @param toSvcAddr addr of the service to make active
+ * @param fencer for fencing fromSvc
+ * @param forceFence to fence fromSvc even if not strictly necessary
+ * @param forceActive try to make toSvc active even if it is not ready
+ * @throws FailoverFailedException if the failover fails
+ */
+ public static void failover(HAServiceProtocol fromSvc,
+ InetSocketAddress fromSvcAddr,
+ HAServiceProtocol toSvc,
+ InetSocketAddress toSvcAddr,
+ NodeFencer fencer,
+ boolean forceFence,
+ boolean forceActive)
+ throws FailoverFailedException {
+ Preconditions.checkArgument(fencer != null, "failover requires a fencer");
+ preFailoverChecks(toSvc, toSvcAddr, forceActive);
+
+ // Try to make fromSvc standby
+ boolean tryFence = true;
+ try {
+ HAServiceProtocolHelper.transitionToStandby(fromSvc);
+ // We should try to fence if we failed or it was forced
+ tryFence = forceFence ? true : false;
+ } catch (ServiceFailedException sfe) {
+ LOG.warn("Unable to make " + fromSvcAddr + " standby (" +
+ sfe.getMessage() + ")");
+ } catch (IOException ioe) {
+ LOG.warn("Unable to make " + fromSvcAddr +
+ " standby (unable to connect)", ioe);
+ }
+
+ // Fence fromSvc if it's required or forced by the user
+ if (tryFence) {
+ if (!fencer.fence(fromSvcAddr)) {
+ throw new FailoverFailedException("Unable to fence " +
+ fromSvcAddr + ". Fencing failed.");
+ }
+ }
+
+ // Try to make toSvc active
+ boolean failed = false;
+ Throwable cause = null;
+ try {
+ HAServiceProtocolHelper.transitionToActive(toSvc);
+ } catch (ServiceFailedException sfe) {
+ LOG.error("Unable to make " + toSvcAddr + " active (" +
+ sfe.getMessage() + "). Failing back.");
+ failed = true;
+ cause = sfe;
+ } catch (IOException ioe) {
+ LOG.error("Unable to make " + toSvcAddr +
+ " active (unable to connect). Failing back.", ioe);
+ failed = true;
+ cause = ioe;
+ }
+
+ // We failed to make toSvc active
+ if (failed) {
+ String msg = "Unable to failover to " + toSvcAddr;
+ // Only try to failback if we didn't fence fromSvc
+ if (!tryFence) {
+ try {
+ // Unconditionally fence toSvc in case it is still trying to
+ // become active, eg we timed out waiting for its response.
+ // Unconditionally force fromSvc to become active since it
+ // was previously active when we initiated failover.
+ failover(toSvc, toSvcAddr, fromSvc, fromSvcAddr, fencer, true, true);
+ } catch (FailoverFailedException ffe) {
+ msg += ". Failback to " + fromSvcAddr +
+ " failed (" + ffe.getMessage() + ")";
+ LOG.fatal(msg);
+ }
+ }
+ throw new FailoverFailedException(msg, cause);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverFailedException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverFailedException.java
new file mode 100644
index 0000000000..09982b4f7e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FailoverFailedException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Exception thrown to indicate service failover has failed.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class FailoverFailedException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public FailoverFailedException(final String message) {
+ super(message);
+ }
+
+ public FailoverFailedException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FenceMethod.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FenceMethod.java
new file mode 100644
index 0000000000..d8bda1402f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FenceMethod.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+
+/**
+ * A fencing method is a method by which one node can forcibly prevent
+ * another node from making continued progress. This might be implemented
+ * by killing a process on the other node, by denying the other node's
+ * access to shared storage, or by accessing a PDU to cut the other node's
+ * power.
+ *
+ * Since these methods are often vendor- or device-specific, operators
+ * may implement this interface in order to achieve fencing.
+ *
+ * Fencing is configured by the operator as an ordered list of methods to
+ * attempt. Each method will be tried in turn, and the next in the list
+ * will only be attempted if the previous one fails. See {@link NodeFencer}
+ * for more information.
+ *
+ * If an implementation also implements {@link Configurable} then its
+ * setConf method will be called upon instantiation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface FenceMethod {
+ /**
+ * Verify that the given fencing method's arguments are valid.
+ * @param args the arguments provided in the configuration. This may
+ * be null if the operator did not configure any arguments.
+ * @throws BadFencingConfigurationException if the arguments are invalid
+ */
+ public void checkArgs(String args) throws BadFencingConfigurationException;
+
+ /**
+ * Attempt to fence the target node.
+ * @param serviceAddr the address (host:ipcport) of the service to fence
+ * @param args the configured arguments, which were checked at startup by
+ * {@link #checkArgs(String)}
+ * @return true if fencing was successful, false if unsuccessful or
+ * indeterminate
+ * @throws BadFencingConfigurationException if the configuration was
+ * determined to be invalid only at runtime
+ */
+ public boolean tryFence(InetSocketAddress serviceAddr, String args)
+ throws BadFencingConfigurationException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
new file mode 100644
index 0000000000..3350692d68
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -0,0 +1,321 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.InetSocketAddress;
+import java.util.Map;
+
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.ParseException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * A command-line tool for making calls in the HAServiceProtocol.
+ * For example,. this can be used to force a service to standby or active
+ * mode, or to trigger a health-check.
+ */
+@InterfaceAudience.Private
+
+public abstract class HAAdmin extends Configured implements Tool {
+
+ private static final String FORCEFENCE = "forcefence";
+ private static final String FORCEACTIVE = "forceactive";
+
+ private static Map USAGE =
+ ImmutableMap.builder()
+ .put("-transitionToActive",
+ new UsageInfo("", "Transitions the service into Active state"))
+ .put("-transitionToStandby",
+ new UsageInfo("", "Transitions the service into Standby state"))
+ .put("-failover",
+ new UsageInfo("[--"+FORCEFENCE+"] [--"+FORCEACTIVE+"] ",
+ "Failover from the first service to the second.\n" +
+ "Unconditionally fence services if the "+FORCEFENCE+" option is used.\n" +
+ "Try to failover to the target service even if it is not ready if the " +
+ FORCEACTIVE + " option is used."))
+ .put("-getServiceState",
+ new UsageInfo("", "Returns the state of the service"))
+ .put("-checkHealth",
+ new UsageInfo("",
+ "Requests that the service perform a health check.\n" +
+ "The HAAdmin tool will exit with a non-zero exit code\n" +
+ "if the check fails."))
+ .put("-help",
+ new UsageInfo("", "Displays help on the specified command"))
+ .build();
+
+ /** Output stream for errors, for use in tests */
+ protected PrintStream errOut = System.err;
+ PrintStream out = System.out;
+
+ protected String getUsageString() {
+ return "Usage: HAAdmin";
+ }
+
+ protected void printUsage(PrintStream errOut) {
+ errOut.println(getUsageString());
+ for (Map.Entry e : USAGE.entrySet()) {
+ String cmd = e.getKey();
+ UsageInfo usage = e.getValue();
+
+ errOut.println(" [" + cmd + " " + usage.args + "]");
+ }
+ errOut.println();
+ ToolRunner.printGenericCommandUsage(errOut);
+ }
+
+ private static void printUsage(PrintStream errOut, String cmd) {
+ UsageInfo usage = USAGE.get(cmd);
+ if (usage == null) {
+ throw new RuntimeException("No usage for cmd " + cmd);
+ }
+ errOut.println("Usage: HAAdmin [" + cmd + " " + usage.args + "]");
+ }
+
+ private int transitionToActive(final String[] argv)
+ throws IOException, ServiceFailedException {
+ if (argv.length != 2) {
+ errOut.println("transitionToActive: incorrect number of arguments");
+ printUsage(errOut, "-transitionToActive");
+ return -1;
+ }
+
+ HAServiceProtocol proto = getProtocol(argv[1]);
+ HAServiceProtocolHelper.transitionToActive(proto);
+ return 0;
+ }
+
+ private int transitionToStandby(final String[] argv)
+ throws IOException, ServiceFailedException {
+ if (argv.length != 2) {
+ errOut.println("transitionToStandby: incorrect number of arguments");
+ printUsage(errOut, "-transitionToStandby");
+ return -1;
+ }
+
+ HAServiceProtocol proto = getProtocol(argv[1]);
+ HAServiceProtocolHelper.transitionToStandby(proto);
+ return 0;
+ }
+
+ private int failover(final String[] argv)
+ throws IOException, ServiceFailedException {
+ Configuration conf = getConf();
+ boolean forceFence = false;
+ boolean forceActive = false;
+
+ Options failoverOpts = new Options();
+ // "-failover" isn't really an option but we need to add
+ // it to appease CommandLineParser
+ failoverOpts.addOption("failover", false, "failover");
+ failoverOpts.addOption(FORCEFENCE, false, "force fencing");
+ failoverOpts.addOption(FORCEACTIVE, false, "force failover");
+
+ CommandLineParser parser = new GnuParser();
+ CommandLine cmd;
+
+ try {
+ cmd = parser.parse(failoverOpts, argv);
+ forceFence = cmd.hasOption(FORCEFENCE);
+ forceActive = cmd.hasOption(FORCEACTIVE);
+ } catch (ParseException pe) {
+ errOut.println("failover: incorrect arguments");
+ printUsage(errOut, "-failover");
+ return -1;
+ }
+
+ int numOpts = cmd.getOptions() == null ? 0 : cmd.getOptions().length;
+ final String[] args = cmd.getArgs();
+
+ if (numOpts > 2 || args.length != 2) {
+ errOut.println("failover: incorrect arguments");
+ printUsage(errOut, "-failover");
+ return -1;
+ }
+
+ NodeFencer fencer;
+ try {
+ fencer = NodeFencer.create(conf);
+ } catch (BadFencingConfigurationException bfce) {
+ errOut.println("failover: incorrect fencing configuration: " +
+ bfce.getLocalizedMessage());
+ return -1;
+ }
+ if (fencer == null) {
+ errOut.println("failover: no fencer configured");
+ return -1;
+ }
+
+ InetSocketAddress addr1 =
+ NetUtils.createSocketAddr(getServiceAddr(args[0]));
+ InetSocketAddress addr2 =
+ NetUtils.createSocketAddr(getServiceAddr(args[1]));
+ HAServiceProtocol proto1 = getProtocol(args[0]);
+ HAServiceProtocol proto2 = getProtocol(args[1]);
+
+ try {
+ FailoverController.failover(proto1, addr1, proto2, addr2,
+ fencer, forceFence, forceActive);
+ out.println("Failover from "+args[0]+" to "+args[1]+" successful");
+ } catch (FailoverFailedException ffe) {
+ errOut.println("Failover failed: " + ffe.getLocalizedMessage());
+ return -1;
+ }
+ return 0;
+ }
+
+ private int checkHealth(final String[] argv)
+ throws IOException, ServiceFailedException {
+ if (argv.length != 2) {
+ errOut.println("checkHealth: incorrect number of arguments");
+ printUsage(errOut, "-checkHealth");
+ return -1;
+ }
+
+ HAServiceProtocol proto = getProtocol(argv[1]);
+ try {
+ HAServiceProtocolHelper.monitorHealth(proto);
+ } catch (HealthCheckFailedException e) {
+ errOut.println("Health check failed: " + e.getLocalizedMessage());
+ return -1;
+ }
+ return 0;
+ }
+
+ private int getServiceState(final String[] argv)
+ throws IOException, ServiceFailedException {
+ if (argv.length != 2) {
+ errOut.println("getServiceState: incorrect number of arguments");
+ printUsage(errOut, "-getServiceState");
+ return -1;
+ }
+
+ HAServiceProtocol proto = getProtocol(argv[1]);
+ out.println(proto.getServiceState());
+ return 0;
+ }
+
+ /**
+ * Return the serviceId as is, we are assuming it was
+ * given as a service address of form .
+ */
+ protected String getServiceAddr(String serviceId) {
+ return serviceId;
+ }
+
+ /**
+ * Return a proxy to the specified target service.
+ */
+ protected HAServiceProtocol getProtocol(String serviceId)
+ throws IOException {
+ String serviceAddr = getServiceAddr(serviceId);
+ InetSocketAddress addr = NetUtils.createSocketAddr(serviceAddr);
+ return new HAServiceProtocolClientSideTranslatorPB(addr, getConf());
+ }
+
+ @Override
+ public int run(String[] argv) throws Exception {
+ try {
+ return runCmd(argv);
+ } catch (IllegalArgumentException iae) {
+ errOut.println("Illegal argument: " + iae.getLocalizedMessage());
+ return -1;
+ } catch (IOException ioe) {
+ errOut.println("Operation failed: " + ioe.getLocalizedMessage());
+ return -1;
+ }
+ }
+
+ protected int runCmd(String[] argv) throws Exception {
+ if (argv.length < 1) {
+ printUsage(errOut);
+ return -1;
+ }
+
+ String cmd = argv[0];
+
+ if (!cmd.startsWith("-")) {
+ errOut.println("Bad command '" + cmd + "': expected command starting with '-'");
+ printUsage(errOut);
+ return -1;
+ }
+
+ if ("-transitionToActive".equals(cmd)) {
+ return transitionToActive(argv);
+ } else if ("-transitionToStandby".equals(cmd)) {
+ return transitionToStandby(argv);
+ } else if ("-failover".equals(cmd)) {
+ return failover(argv);
+ } else if ("-getServiceState".equals(cmd)) {
+ return getServiceState(argv);
+ } else if ("-checkHealth".equals(cmd)) {
+ return checkHealth(argv);
+ } else if ("-help".equals(cmd)) {
+ return help(argv);
+ } else {
+ errOut.println(cmd.substring(1) + ": Unknown command");
+ printUsage(errOut);
+ return -1;
+ }
+ }
+
+ private int help(String[] argv) {
+ if (argv.length != 2) {
+ printUsage(errOut, "-help");
+ return -1;
+ }
+ String cmd = argv[1];
+ if (!cmd.startsWith("-")) {
+ cmd = "-" + cmd;
+ }
+ UsageInfo usageInfo = USAGE.get(cmd);
+ if (usageInfo == null) {
+ errOut.println(cmd + ": Unknown command");
+ printUsage(errOut);
+ return -1;
+ }
+
+ errOut.println(cmd + " [" + usageInfo.args + "]: " + usageInfo.help);
+ return 0;
+ }
+
+ private static class UsageInfo {
+ private final String args;
+ private final String help;
+
+ public UsageInfo(String args, String help) {
+ this.args = args;
+ this.help = help;
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java
new file mode 100644
index 0000000000..18b10f99c6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.KerberosInfo;
+
+import java.io.IOException;
+
+/**
+ * Protocol interface that provides High Availability related primitives to
+ * monitor and fail-over the service.
+ *
+ * This interface could be used by HA frameworks to manage the service.
+ */
+@KerberosInfo(
+ serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface HAServiceProtocol {
+ /**
+ * Initial version of the protocol
+ */
+ public static final long versionID = 1L;
+
+ /**
+ * An HA service may be in active or standby state. During
+ * startup, it is in an unknown INITIALIZING state.
+ */
+ public enum HAServiceState {
+ INITIALIZING("initializing"),
+ ACTIVE("active"),
+ STANDBY("standby");
+
+ private String name;
+
+ HAServiceState(String name) {
+ this.name = name;
+ }
+
+ public String toString() {
+ return name;
+ }
+ }
+
+ /**
+ * Monitor the health of service. This periodically called by the HA
+ * frameworks to monitor the health of the service.
+ *
+ * Service is expected to perform checks to ensure it is functional.
+ * If the service is not healthy due to failure or partial failure,
+ * it is expected to throw {@link HealthCheckFailedException}.
+ * The definition of service not healthy is left to the service.
+ *
+ * Note that when health check of an Active service fails,
+ * failover to standby may be done.
+ *
+ * @throws HealthCheckFailedException
+ * if the health check of a service fails.
+ * @throws AccessControlException
+ * if access is denied.
+ * @throws IOException
+ * if other errors happen
+ */
+ public void monitorHealth() throws HealthCheckFailedException,
+ AccessControlException,
+ IOException;
+
+ /**
+ * Request service to transition to active state. No operation, if the
+ * service is already in active state.
+ *
+ * @throws ServiceFailedException
+ * if transition from standby to active fails.
+ * @throws AccessControlException
+ * if access is denied.
+ * @throws IOException
+ * if other errors happen
+ */
+ public void transitionToActive() throws ServiceFailedException,
+ AccessControlException,
+ IOException;
+
+ /**
+ * Request service to transition to standby state. No operation, if the
+ * service is already in standby state.
+ *
+ * @throws ServiceFailedException
+ * if transition from active to standby fails.
+ * @throws AccessControlException
+ * if access is denied.
+ * @throws IOException
+ * if other errors happen
+ */
+ public void transitionToStandby() throws ServiceFailedException,
+ AccessControlException,
+ IOException;
+
+ /**
+ * Return the current state of the service.
+ *
+ * @throws AccessControlException
+ * if access is denied.
+ * @throws IOException
+ * if other errors happen
+ */
+ public HAServiceState getServiceState() throws AccessControlException,
+ IOException;
+
+ /**
+ * Return true if the service is capable and ready to transition
+ * from the standby state to the active state.
+ *
+ * @return true if the service is ready to become active, false otherwise.
+ * @throws AccessControlException
+ * if access is denied.
+ * @throws IOException
+ * if other errors happen
+ */
+ public boolean readyToBecomeActive() throws ServiceFailedException,
+ AccessControlException,
+ IOException;
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocolHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocolHelper.java
new file mode 100644
index 0000000000..b8ee717951
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocolHelper.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.RemoteException;
+
+/**
+ * Helper for making {@link HAServiceProtocol} RPC calls. This helper
+ * unwraps the {@link RemoteException} to specific exceptions.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HAServiceProtocolHelper {
+ public static void monitorHealth(HAServiceProtocol svc)
+ throws IOException {
+ try {
+ svc.monitorHealth();
+ } catch (RemoteException e) {
+ throw e.unwrapRemoteException(HealthCheckFailedException.class);
+ }
+ }
+
+ public static void transitionToActive(HAServiceProtocol svc)
+ throws IOException {
+ try {
+ svc.transitionToActive();
+ } catch (RemoteException e) {
+ throw e.unwrapRemoteException(ServiceFailedException.class);
+ }
+ }
+
+ public static void transitionToStandby(HAServiceProtocol svc)
+ throws IOException {
+ try {
+ svc.transitionToStandby();
+ } catch (RemoteException e) {
+ throw e.unwrapRemoteException(ServiceFailedException.class);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthCheckFailedException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthCheckFailedException.java
new file mode 100644
index 0000000000..e636adff3e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthCheckFailedException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Exception thrown to indicate that health check of a service failed.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HealthCheckFailedException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+ public HealthCheckFailedException(final String message) {
+ super(message);
+ }
+
+ public HealthCheckFailedException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java
new file mode 100644
index 0000000000..34a2c8b823
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+
+/**
+ * This class parses the configured list of fencing methods, and
+ * is responsible for trying each one in turn while logging informative
+ * output.
+ *
+ * The fencing methods are configured as a carriage-return separated list.
+ * Each line in the list is of the form:
+ * com.example.foo.MyMethod(arg string)
+ * or
+ * com.example.foo.MyMethod
+ * The class provided must implement the {@link FenceMethod} interface.
+ * The fencing methods that ship with Hadoop may also be referred to
+ * by shortened names:
+ *
+ *
shell(/path/to/some/script.sh args...)
+ *
sshfence(...) (see {@link SshFenceByTcpPort})
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class NodeFencer {
+ public static final String CONF_METHODS_KEY =
+ "dfs.ha.fencing.methods";
+
+ private static final String CLASS_RE = "([a-zA-Z0-9\\.\\$]+)";
+ private static final Pattern CLASS_WITH_ARGUMENT =
+ Pattern.compile(CLASS_RE + "\\((.+?)\\)");
+ private static final Pattern CLASS_WITHOUT_ARGUMENT =
+ Pattern.compile(CLASS_RE);
+ private static final Pattern HASH_COMMENT_RE =
+ Pattern.compile("#.*$");
+
+ private static final Log LOG = LogFactory.getLog(NodeFencer.class);
+
+ /**
+ * Standard fencing methods included with Hadoop.
+ */
+ private static final Map> STANDARD_METHODS =
+ ImmutableMap.>of(
+ "shell", ShellCommandFencer.class,
+ "sshfence", SshFenceByTcpPort.class);
+
+ private final List methods;
+
+ public NodeFencer(Configuration conf)
+ throws BadFencingConfigurationException {
+ this.methods = parseMethods(conf);
+ }
+
+ public static NodeFencer create(Configuration conf)
+ throws BadFencingConfigurationException {
+ String confStr = conf.get(CONF_METHODS_KEY);
+ if (confStr == null) {
+ return null;
+ }
+ return new NodeFencer(conf);
+ }
+
+ public boolean fence(InetSocketAddress serviceAddr) {
+ LOG.info("====== Beginning Service Fencing Process... ======");
+ int i = 0;
+ for (FenceMethodWithArg method : methods) {
+ LOG.info("Trying method " + (++i) + "/" + methods.size() +": " + method);
+
+ try {
+ if (method.method.tryFence(serviceAddr, method.arg)) {
+ LOG.info("====== Fencing successful by method " + method + " ======");
+ return true;
+ }
+ } catch (BadFencingConfigurationException e) {
+ LOG.error("Fencing method " + method + " misconfigured", e);
+ continue;
+ } catch (Throwable t) {
+ LOG.error("Fencing method " + method + " failed with an unexpected error.", t);
+ continue;
+ }
+ LOG.warn("Fencing method " + method + " was unsuccessful.");
+ }
+
+ LOG.error("Unable to fence service by any configured method.");
+ return false;
+ }
+
+ private static List parseMethods(Configuration conf)
+ throws BadFencingConfigurationException {
+ String confStr = conf.get(CONF_METHODS_KEY);
+ String[] lines = confStr.split("\\s*\n\\s*");
+
+ List methods = Lists.newArrayList();
+ for (String line : lines) {
+ line = HASH_COMMENT_RE.matcher(line).replaceAll("");
+ line = line.trim();
+ if (!line.isEmpty()) {
+ methods.add(parseMethod(conf, line));
+ }
+ }
+
+ return methods;
+ }
+
+ private static FenceMethodWithArg parseMethod(Configuration conf, String line)
+ throws BadFencingConfigurationException {
+ Matcher m;
+ if ((m = CLASS_WITH_ARGUMENT.matcher(line)).matches()) {
+ String className = m.group(1);
+ String arg = m.group(2);
+ return createFenceMethod(conf, className, arg);
+ } else if ((m = CLASS_WITHOUT_ARGUMENT.matcher(line)).matches()) {
+ String className = m.group(1);
+ return createFenceMethod(conf, className, null);
+ } else {
+ throw new BadFencingConfigurationException(
+ "Unable to parse line: '" + line + "'");
+ }
+ }
+
+ private static FenceMethodWithArg createFenceMethod(
+ Configuration conf, String clazzName, String arg)
+ throws BadFencingConfigurationException {
+
+ Class> clazz;
+ try {
+ // See if it's a short name for one of the built-in methods
+ clazz = STANDARD_METHODS.get(clazzName);
+ if (clazz == null) {
+ // Try to instantiate the user's custom method
+ clazz = Class.forName(clazzName);
+ }
+ } catch (Exception e) {
+ throw new BadFencingConfigurationException(
+ "Could not find configured fencing method " + clazzName,
+ e);
+ }
+
+ // Check that it implements the right interface
+ if (!FenceMethod.class.isAssignableFrom(clazz)) {
+ throw new BadFencingConfigurationException("Class " + clazzName +
+ " does not implement FenceMethod");
+ }
+
+ FenceMethod method = (FenceMethod)ReflectionUtils.newInstance(
+ clazz, conf);
+ method.checkArgs(arg);
+ return new FenceMethodWithArg(method, arg);
+ }
+
+ private static class FenceMethodWithArg {
+ private final FenceMethod method;
+ private final String arg;
+
+ private FenceMethodWithArg(FenceMethod method, String arg) {
+ this.method = method;
+ this.arg = arg;
+ }
+
+ public String toString() {
+ return method.getClass().getCanonicalName() + "(" + arg + ")";
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ServiceFailedException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ServiceFailedException.java
new file mode 100644
index 0000000000..6f3e444b39
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ServiceFailedException.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+
+/**
+ * Exception thrown to indicate that an operation performed
+ * to modify the state of a service or application failed.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ServiceFailedException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+ public ServiceFailedException(final String message) {
+ super(message);
+ }
+
+ public ServiceFailedException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java
new file mode 100644
index 0000000000..ca81f23a18
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java
@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Fencing method that runs a shell command. It should be specified
+ * in the fencing configuration like:
+ *
+ * shell(/path/to/my/script.sh arg1 arg2 ...)
+ *
+ * The string between '(' and ')' is passed directly to a bash shell and
+ * may not include any closing parentheses.
+ *
+ * The shell command will be run with an environment set up to contain
+ * all of the current Hadoop configuration variables, with the '_' character
+ * replacing any '.' characters in the configuration keys.
+ *
+ * If the shell command returns an exit code of 0, the fencing is
+ * determined to be successful. If it returns any other exit code, the
+ * fencing was not successful and the next fencing method in the list
+ * will be attempted.
+ *
+ * Note: this fencing method does not implement any timeout.
+ * If timeouts are necessary, they should be implemented in the shell
+ * script itself (eg by forking a subshell to kill its parent in
+ * some number of seconds).
+ */
+public class ShellCommandFencer
+ extends Configured implements FenceMethod {
+
+ /** Length at which to abbreviate command in long messages */
+ private static final int ABBREV_LENGTH = 20;
+
+ @VisibleForTesting
+ static Log LOG = LogFactory.getLog(
+ ShellCommandFencer.class);
+
+ @Override
+ public void checkArgs(String args) throws BadFencingConfigurationException {
+ if (args == null || args.isEmpty()) {
+ throw new BadFencingConfigurationException(
+ "No argument passed to 'shell' fencing method");
+ }
+ // Nothing else we can really check without actually running the command
+ }
+
+ @Override
+ public boolean tryFence(InetSocketAddress serviceAddr, String cmd) {
+ List cmdList = Arrays.asList(cmd.split("\\s+"));
+
+ // Create arg list with service as the first argument
+ List argList = new ArrayList();
+ argList.add(cmdList.get(0));
+ argList.add(serviceAddr.getHostName() + ":" + serviceAddr.getPort());
+ argList.addAll(cmdList.subList(1, cmdList.size()));
+ String cmdWithSvc = StringUtils.join(" ", argList);
+
+ ProcessBuilder builder = new ProcessBuilder(
+ "bash", "-e", "-c", cmdWithSvc);
+ setConfAsEnvVars(builder.environment());
+
+ Process p;
+ try {
+ p = builder.start();
+ p.getOutputStream().close();
+ } catch (IOException e) {
+ LOG.warn("Unable to execute " + cmd, e);
+ return false;
+ }
+
+ String pid = tryGetPid(p);
+ LOG.info("Launched fencing command '" + cmd + "' with "
+ + ((pid != null) ? ("pid " + pid) : "unknown pid"));
+
+ String logPrefix = abbreviate(cmd, ABBREV_LENGTH);
+ if (pid != null) {
+ logPrefix = "[PID " + pid + "] " + logPrefix;
+ }
+
+ // Pump logs to stderr
+ StreamPumper errPumper = new StreamPumper(
+ LOG, logPrefix, p.getErrorStream(),
+ StreamPumper.StreamType.STDERR);
+ errPumper.start();
+
+ StreamPumper outPumper = new StreamPumper(
+ LOG, logPrefix, p.getInputStream(),
+ StreamPumper.StreamType.STDOUT);
+ outPumper.start();
+
+ int rc;
+ try {
+ rc = p.waitFor();
+ errPumper.join();
+ outPumper.join();
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted while waiting for fencing command: " + cmd);
+ return false;
+ }
+
+ return rc == 0;
+ }
+
+ /**
+ * Abbreviate a string by putting '...' in the middle of it,
+ * in an attempt to keep logs from getting too messy.
+ * @param cmd the string to abbreviate
+ * @param len maximum length to abbreviate to
+ * @return abbreviated string
+ */
+ static String abbreviate(String cmd, int len) {
+ if (cmd.length() > len && len >= 5) {
+ int firstHalf = (len - 3) / 2;
+ int rem = len - firstHalf - 3;
+
+ return cmd.substring(0, firstHalf) +
+ "..." + cmd.substring(cmd.length() - rem);
+ } else {
+ return cmd;
+ }
+ }
+
+ /**
+ * Attempt to use evil reflection tricks to determine the
+ * pid of a launched process. This is helpful to ops
+ * if debugging a fencing process that might have gone
+ * wrong. If running on a system or JVM where this doesn't
+ * work, it will simply return null.
+ */
+ private static String tryGetPid(Process p) {
+ try {
+ Class extends Process> clazz = p.getClass();
+ if (clazz.getName().equals("java.lang.UNIXProcess")) {
+ Field f = clazz.getDeclaredField("pid");
+ f.setAccessible(true);
+ return String.valueOf(f.getInt(p));
+ } else {
+ LOG.trace("Unable to determine pid for " + p
+ + " since it is not a UNIXProcess");
+ return null;
+ }
+ } catch (Throwable t) {
+ LOG.trace("Unable to determine pid for " + p, t);
+ return null;
+ }
+ }
+
+ /**
+ * Set the environment of the subprocess to be the Configuration,
+ * with '.'s replaced by '_'s.
+ */
+ private void setConfAsEnvVars(Map env) {
+ for (Map.Entry pair : getConf()) {
+ env.put(pair.getKey().replace('.', '_'), pair.getValue());
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
new file mode 100644
index 0000000000..cec731cf20
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/SshFenceByTcpPort.java
@@ -0,0 +1,315 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configured;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.jcraft.jsch.ChannelExec;
+import com.jcraft.jsch.JSch;
+import com.jcraft.jsch.JSchException;
+import com.jcraft.jsch.Session;
+
+/**
+ * This fencing implementation sshes to the target node and uses
+ * fuser to kill the process listening on the service's
+ * TCP port. This is more accurate than using "jps" since it doesn't
+ * require parsing, and will work even if there are multiple service
+ * processes running on the same machine.
+ * It returns a successful status code if:
+ *
+ *
fuser indicates it successfully killed a process, or
+ *
nc -z indicates that nothing is listening on the target port
+ *
+ *
+ * This fencing mechanism is configured as following in the fencing method
+ * list:
+ * sshfence([[username][:ssh-port]])
+ * where the optional argument specifies the username and port to use
+ * with ssh.
+ *
+ * In order to achieve passwordless SSH, the operator must also configure
+ * dfs.ha.fencing.ssh.private-key-files to point to an
+ * SSH key that has passphrase-less access to the given username and host.
+ */
+public class SshFenceByTcpPort extends Configured
+ implements FenceMethod {
+
+ static final Log LOG = LogFactory.getLog(
+ SshFenceByTcpPort.class);
+
+ static final String CONF_CONNECT_TIMEOUT_KEY =
+ "dfs.ha.fencing.ssh.connect-timeout";
+ private static final int CONF_CONNECT_TIMEOUT_DEFAULT =
+ 30*1000;
+ static final String CONF_IDENTITIES_KEY =
+ "dfs.ha.fencing.ssh.private-key-files";
+
+ /**
+ * Verify that the argument, if given, in the conf is parseable.
+ */
+ @Override
+ public void checkArgs(String argStr) throws BadFencingConfigurationException {
+ if (argStr != null) {
+ // Use a dummy service when checking the arguments defined
+ // in the configuration are parseable.
+ new Args(new InetSocketAddress("localhost", 8020), argStr);
+ }
+ }
+
+ @Override
+ public boolean tryFence(InetSocketAddress serviceAddr, String argsStr)
+ throws BadFencingConfigurationException {
+
+ Args args = new Args(serviceAddr, argsStr);
+
+ Session session;
+ try {
+ session = createSession(args);
+ } catch (JSchException e) {
+ LOG.warn("Unable to create SSH session", e);
+ return false;
+ }
+
+ LOG.info("Connecting to " + args.host + "...");
+
+ try {
+ session.connect(getSshConnectTimeout());
+ } catch (JSchException e) {
+ LOG.warn("Unable to connect to " + args.host
+ + " as user " + args.user, e);
+ return false;
+ }
+ LOG.info("Connected to " + args.host);
+
+ try {
+ return doFence(session, args.targetPort);
+ } catch (JSchException e) {
+ LOG.warn("Unable to achieve fencing on remote host", e);
+ return false;
+ } finally {
+ session.disconnect();
+ }
+ }
+
+
+ private Session createSession(Args args) throws JSchException {
+ JSch jsch = new JSch();
+ for (String keyFile : getKeyFiles()) {
+ jsch.addIdentity(keyFile);
+ }
+ JSch.setLogger(new LogAdapter());
+
+ Session session = jsch.getSession(args.user, args.host, args.sshPort);
+ session.setConfig("StrictHostKeyChecking", "no");
+ return session;
+ }
+
+ private boolean doFence(Session session, int port) throws JSchException {
+ try {
+ LOG.info("Looking for process running on port " + port);
+ int rc = execCommand(session,
+ "PATH=$PATH:/sbin:/usr/sbin fuser -v -k -n tcp " + port);
+ if (rc == 0) {
+ LOG.info("Successfully killed process that was " +
+ "listening on port " + port);
+ // exit code 0 indicates the process was successfully killed.
+ return true;
+ } else if (rc == 1) {
+ // exit code 1 indicates either that the process was not running
+ // or that fuser didn't have root privileges in order to find it
+ // (eg running as a different user)
+ LOG.info(
+ "Indeterminate response from trying to kill service. " +
+ "Verifying whether it is running using nc...");
+ rc = execCommand(session, "nc -z localhost 8020");
+ if (rc == 0) {
+ // the service is still listening - we are unable to fence
+ LOG.warn("Unable to fence - it is running but we cannot kill it");
+ return false;
+ } else {
+ LOG.info("Verified that the service is down.");
+ return true;
+ }
+ } else {
+ // other
+ }
+ LOG.info("rc: " + rc);
+ return rc == 0;
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while trying to fence via ssh", e);
+ return false;
+ } catch (IOException e) {
+ LOG.warn("Unknown failure while trying to fence via ssh", e);
+ return false;
+ }
+ }
+
+ /**
+ * Execute a command through the ssh session, pumping its
+ * stderr and stdout to our own logs.
+ */
+ private int execCommand(Session session, String cmd)
+ throws JSchException, InterruptedException, IOException {
+ LOG.debug("Running cmd: " + cmd);
+ ChannelExec exec = null;
+ try {
+ exec = (ChannelExec)session.openChannel("exec");
+ exec.setCommand(cmd);
+ exec.setInputStream(null);
+ exec.connect();
+
+ // Pump stdout of the command to our WARN logs
+ StreamPumper outPumper = new StreamPumper(LOG, cmd + " via ssh",
+ exec.getInputStream(), StreamPumper.StreamType.STDOUT);
+ outPumper.start();
+
+ // Pump stderr of the command to our WARN logs
+ StreamPumper errPumper = new StreamPumper(LOG, cmd + " via ssh",
+ exec.getErrStream(), StreamPumper.StreamType.STDERR);
+ errPumper.start();
+
+ outPumper.join();
+ errPumper.join();
+ return exec.getExitStatus();
+ } finally {
+ cleanup(exec);
+ }
+ }
+
+ private static void cleanup(ChannelExec exec) {
+ if (exec != null) {
+ try {
+ exec.disconnect();
+ } catch (Throwable t) {
+ LOG.warn("Couldn't disconnect ssh channel", t);
+ }
+ }
+ }
+
+ private int getSshConnectTimeout() {
+ return getConf().getInt(
+ CONF_CONNECT_TIMEOUT_KEY, CONF_CONNECT_TIMEOUT_DEFAULT);
+ }
+
+ private Collection getKeyFiles() {
+ return getConf().getTrimmedStringCollection(CONF_IDENTITIES_KEY);
+ }
+
+ /**
+ * Container for the parsed arg line for this fencing method.
+ */
+ @VisibleForTesting
+ static class Args {
+ private static final Pattern USER_PORT_RE = Pattern.compile(
+ "([^:]+?)?(?:\\:(\\d+))?");
+
+ private static final int DEFAULT_SSH_PORT = 22;
+
+ String host;
+ int targetPort;
+ String user;
+ int sshPort;
+
+ public Args(InetSocketAddress serviceAddr, String arg)
+ throws BadFencingConfigurationException {
+ host = serviceAddr.getHostName();
+ targetPort = serviceAddr.getPort();
+ user = System.getProperty("user.name");
+ sshPort = DEFAULT_SSH_PORT;
+
+ // Parse optional user and ssh port
+ if (arg != null && !"".equals(arg)) {
+ Matcher m = USER_PORT_RE.matcher(arg);
+ if (!m.matches()) {
+ throw new BadFencingConfigurationException(
+ "Unable to parse user and SSH port: "+ arg);
+ }
+ if (m.group(1) != null) {
+ user = m.group(1);
+ }
+ if (m.group(2) != null) {
+ sshPort = parseConfiggedPort(m.group(2));
+ }
+ }
+ }
+
+ private Integer parseConfiggedPort(String portStr)
+ throws BadFencingConfigurationException {
+ try {
+ return Integer.valueOf(portStr);
+ } catch (NumberFormatException nfe) {
+ throw new BadFencingConfigurationException(
+ "Port number '" + portStr + "' invalid");
+ }
+ }
+ }
+
+ /**
+ * Adapter from JSch's logger interface to our log4j
+ */
+ private static class LogAdapter implements com.jcraft.jsch.Logger {
+ static final Log LOG = LogFactory.getLog(
+ SshFenceByTcpPort.class.getName() + ".jsch");
+
+ public boolean isEnabled(int level) {
+ switch (level) {
+ case com.jcraft.jsch.Logger.DEBUG:
+ return LOG.isDebugEnabled();
+ case com.jcraft.jsch.Logger.INFO:
+ return LOG.isInfoEnabled();
+ case com.jcraft.jsch.Logger.WARN:
+ return LOG.isWarnEnabled();
+ case com.jcraft.jsch.Logger.ERROR:
+ return LOG.isErrorEnabled();
+ case com.jcraft.jsch.Logger.FATAL:
+ return LOG.isFatalEnabled();
+ default:
+ return false;
+ }
+ }
+
+ public void log(int level, String message) {
+ switch (level) {
+ case com.jcraft.jsch.Logger.DEBUG:
+ LOG.debug(message);
+ break;
+ case com.jcraft.jsch.Logger.INFO:
+ LOG.info(message);
+ break;
+ case com.jcraft.jsch.Logger.WARN:
+ LOG.warn(message);
+ break;
+ case com.jcraft.jsch.Logger.ERROR:
+ LOG.error(message);
+ break;
+ case com.jcraft.jsch.Logger.FATAL:
+ LOG.fatal(message);
+ break;
+ }
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java
new file mode 100644
index 0000000000..8bc16af2af
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/StreamPumper.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import org.apache.commons.logging.Log;
+
+/**
+ * Class responsible for pumping the streams of the subprocess
+ * out to log4j. stderr is pumped to WARN level and stdout is
+ * pumped to INFO level
+ */
+class StreamPumper {
+ enum StreamType {
+ STDOUT, STDERR;
+ }
+
+ private final Log log;
+
+ final Thread thread;
+ final String logPrefix;
+ final StreamPumper.StreamType type;
+ private final InputStream stream;
+ private boolean started = false;
+
+ StreamPumper(final Log log, final String logPrefix,
+ final InputStream stream, final StreamType type) {
+ this.log = log;
+ this.logPrefix = logPrefix;
+ this.stream = stream;
+ this.type = type;
+
+ thread = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ pump();
+ } catch (Throwable t) {
+ ShellCommandFencer.LOG.warn(logPrefix +
+ ": Unable to pump output from " + type,
+ t);
+ }
+ }
+ }, logPrefix + ": StreamPumper for " + type);
+ thread.setDaemon(true);
+ }
+
+ void join() throws InterruptedException {
+ assert started;
+ thread.join();
+ }
+
+ void start() {
+ assert !started;
+ thread.start();
+ started = true;
+ }
+
+ protected void pump() throws IOException {
+ InputStreamReader inputStreamReader = new InputStreamReader(stream);
+ BufferedReader br = new BufferedReader(inputStreamReader);
+ String line = null;
+ while ((line = br.readLine()) != null) {
+ if (type == StreamType.STDOUT) {
+ log.info(logPrefix + ": " + line);
+ } else {
+ log.warn(logPrefix + ": " + line);
+ }
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolClientSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolClientSideTranslatorPB.java
new file mode 100644
index 0000000000..3bf4f6f013
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolClientSideTranslatorPB.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.GetServiceStateRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.ReadyToBecomeActiveRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyRequestProto;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link HAServiceProtocol} interfaces to the RPC server implementing
+ * {@link HAServiceProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class HAServiceProtocolClientSideTranslatorPB implements
+ HAServiceProtocol, Closeable {
+ /** RpcController is not used and hence is set to null */
+ private final static RpcController NULL_CONTROLLER = null;
+ private final static MonitorHealthRequestProto MONITOR_HEALTH_REQ =
+ MonitorHealthRequestProto.newBuilder().build();
+ private final static TransitionToActiveRequestProto TRANSITION_TO_ACTIVE_REQ =
+ TransitionToActiveRequestProto.newBuilder().build();
+ private final static TransitionToStandbyRequestProto TRANSITION_TO_STANDBY_REQ =
+ TransitionToStandbyRequestProto.newBuilder().build();
+ private final static GetServiceStateRequestProto GET_SERVICE_STATE_REQ =
+ GetServiceStateRequestProto.newBuilder().build();
+ private final static ReadyToBecomeActiveRequestProto ACTIVE_READY_REQ =
+ ReadyToBecomeActiveRequestProto.newBuilder().build();
+
+ private final HAServiceProtocolPB rpcProxy;
+
+ public HAServiceProtocolClientSideTranslatorPB(InetSocketAddress addr,
+ Configuration conf) throws IOException {
+ RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
+ ProtobufRpcEngine.class);
+ rpcProxy = RPC.getProxy(HAServiceProtocolPB.class,
+ RPC.getProtocolVersion(HAServiceProtocolPB.class), addr, conf);
+ }
+
+ @Override
+ public void monitorHealth() throws IOException {
+ try {
+ rpcProxy.monitorHealth(NULL_CONTROLLER, MONITOR_HEALTH_REQ);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public void transitionToActive() throws IOException {
+ try {
+ rpcProxy.transitionToActive(NULL_CONTROLLER, TRANSITION_TO_ACTIVE_REQ);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public void transitionToStandby() throws IOException {
+ try {
+ rpcProxy.transitionToStandby(NULL_CONTROLLER, TRANSITION_TO_STANDBY_REQ);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public HAServiceState getServiceState() throws IOException {
+ HAServiceStateProto state;
+ try {
+ state = rpcProxy.getServiceState(NULL_CONTROLLER,
+ GET_SERVICE_STATE_REQ).getState();
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ switch(state) {
+ case ACTIVE:
+ return HAServiceState.ACTIVE;
+ case STANDBY:
+ return HAServiceState.STANDBY;
+ case INITIALIZING:
+ default:
+ return HAServiceState.INITIALIZING;
+ }
+ }
+
+ @Override
+ public void close() {
+ RPC.stopProxy(rpcProxy);
+ }
+
+ @Override
+ public boolean readyToBecomeActive() throws IOException {
+ try {
+ return rpcProxy.readyToBecomeActive(NULL_CONTROLLER, ACTIVE_READY_REQ)
+ .getReadyToBecomeActive();
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolPB.java
new file mode 100644
index 0000000000..57eefce54a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolPB.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceProtocolService;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+
+@KerberosInfo(
+ serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
+@ProtocolInfo(protocolName = "org.apache.hadoop.ha.HAServiceProtocol",
+ protocolVersion = 1)
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface HAServiceProtocolPB extends
+ HAServiceProtocolService.BlockingInterface, VersionedProtocol {
+ /**
+ * If any methods need annotation, it can be added here
+ */
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolServerSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolServerSideTranslatorPB.java
new file mode 100644
index 0000000000..3655a4e712
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/protocolPB/HAServiceProtocolServerSideTranslatorPB.java
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ha.protocolPB;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.GetServiceStateRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.GetServiceStateResponseProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthResponseProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.ReadyToBecomeActiveRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.ReadyToBecomeActiveResponseProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveResponseProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyRequestProto;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyResponseProto;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is used on the server side. Calls come across the wire for the
+ * for protocol {@link HAServiceProtocolPB}.
+ * This class translates the PB data types
+ * to the native data types used inside the NN as specified in the generic
+ * ClientProtocol.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class HAServiceProtocolServerSideTranslatorPB implements
+ HAServiceProtocolPB {
+ private final HAServiceProtocol server;
+ private static final MonitorHealthResponseProto MONITOR_HEALTH_RESP =
+ MonitorHealthResponseProto.newBuilder().build();
+ private static final TransitionToActiveResponseProto TRANSITION_TO_ACTIVE_RESP =
+ TransitionToActiveResponseProto.newBuilder().build();
+ private static final TransitionToStandbyResponseProto TRANSITION_TO_STANDBY_RESP =
+ TransitionToStandbyResponseProto.newBuilder().build();
+
+ public HAServiceProtocolServerSideTranslatorPB(HAServiceProtocol server) {
+ this.server = server;
+ }
+
+ @Override
+ public MonitorHealthResponseProto monitorHealth(RpcController controller,
+ MonitorHealthRequestProto request) throws ServiceException {
+ try {
+ server.monitorHealth();
+ return MONITOR_HEALTH_RESP;
+ } catch(IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public TransitionToActiveResponseProto transitionToActive(
+ RpcController controller, TransitionToActiveRequestProto request)
+ throws ServiceException {
+ try {
+ server.transitionToActive();
+ return TRANSITION_TO_ACTIVE_RESP;
+ } catch(IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public TransitionToStandbyResponseProto transitionToStandby(
+ RpcController controller, TransitionToStandbyRequestProto request)
+ throws ServiceException {
+ try {
+ server.transitionToStandby();
+ return TRANSITION_TO_STANDBY_RESP;
+ } catch(IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public GetServiceStateResponseProto getServiceState(RpcController controller,
+ GetServiceStateRequestProto request) throws ServiceException {
+ HAServiceState s;
+ try {
+ s = server.getServiceState();
+ } catch(IOException e) {
+ throw new ServiceException(e);
+ }
+
+ HAServiceStateProto ret;
+ switch (s) {
+ case ACTIVE:
+ ret = HAServiceStateProto.ACTIVE;
+ break;
+ case STANDBY:
+ ret = HAServiceStateProto.STANDBY;
+ break;
+ case INITIALIZING:
+ default:
+ ret = HAServiceStateProto.INITIALIZING;
+ break;
+ }
+ return GetServiceStateResponseProto.newBuilder().setState(ret).build();
+ }
+
+ @Override
+ public long getProtocolVersion(String protocol, long clientVersion)
+ throws IOException {
+ return RPC.getProtocolVersion(HAServiceProtocolPB.class);
+ }
+
+ @Override
+ public ProtocolSignature getProtocolSignature(String protocol,
+ long clientVersion, int clientMethodsHash) throws IOException {
+ if (!protocol.equals(RPC.getProtocolName(HAServiceProtocolPB.class))) {
+ throw new IOException("Serverside implements " +
+ RPC.getProtocolName(HAServiceProtocolPB.class) +
+ ". The following requested protocol is unknown: " + protocol);
+ }
+
+ return ProtocolSignature.getProtocolSignature(clientMethodsHash,
+ RPC.getProtocolVersion(HAServiceProtocolPB.class),
+ HAServiceProtocolPB.class);
+ }
+
+ @Override
+ public ReadyToBecomeActiveResponseProto readyToBecomeActive(
+ RpcController controller, ReadyToBecomeActiveRequestProto request)
+ throws ServiceException {
+ try {
+ return ReadyToBecomeActiveResponseProto.newBuilder()
+ .setReadyToBecomeActive(server.readyToBecomeActive()).build();
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
index 812a46e02b..ae37d0bed4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
@@ -27,28 +27,28 @@ import org.apache.hadoop.ipc.RPC;
* event of failover, and always returns the same proxy object.
*/
@InterfaceStability.Evolving
-public class DefaultFailoverProxyProvider implements FailoverProxyProvider {
+public class DefaultFailoverProxyProvider implements FailoverProxyProvider {
- private Object proxy;
- private Class> iface;
+ private T proxy;
+ private Class iface;
- public DefaultFailoverProxyProvider(Class> iface, Object proxy) {
+ public DefaultFailoverProxyProvider(Class iface, T proxy) {
this.proxy = proxy;
this.iface = iface;
}
@Override
- public Class> getInterface() {
+ public Class getInterface() {
return iface;
}
@Override
- public Object getProxy() {
+ public T getProxy() {
return proxy;
}
@Override
- public void performFailover(Object currentProxy) {
+ public void performFailover(T currentProxy) {
// Nothing to do.
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
index 707a40d888..ba7d29f0d5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
* {@link RetryPolicy}.
*/
@InterfaceStability.Evolving
-public interface FailoverProxyProvider extends Closeable {
+public interface FailoverProxyProvider extends Closeable {
/**
* Get the proxy object which should be used until the next failover event
@@ -37,7 +37,7 @@ public interface FailoverProxyProvider extends Closeable {
*
* @return the proxy object to invoke methods upon
*/
- public Object getProxy();
+ public T getProxy();
/**
* Called whenever the associated {@link RetryPolicy} determines that an error
@@ -46,7 +46,7 @@ public interface FailoverProxyProvider extends Closeable {
* @param currentProxy the proxy object which was being used before this
* failover event
*/
- public void performFailover(Object currentProxy);
+ public void performFailover(T currentProxy);
/**
* Return a reference to the interface this provider's proxy objects actually
@@ -58,5 +58,5 @@ public interface FailoverProxyProvider extends Closeable {
* @return the interface implemented by the proxy objects returned by
* {@link FailoverProxyProvider#getProxy()}
*/
- public Class> getInterface();
+ public Class getInterface();
}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
index 0dad53b59b..323542cbd3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
@@ -20,14 +20,15 @@ package org.apache.hadoop.io.retry;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
import java.util.Collections;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
+import org.apache.hadoop.util.ThreadUtil;
import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcInvocationHandler;
class RetryInvocationHandler implements RpcInvocationHandler {
@@ -38,6 +39,7 @@ class RetryInvocationHandler implements RpcInvocationHandler {
* The number of times the associated proxyProvider has ever been failed over.
*/
private long proxyProviderFailoverCount = 0;
+ private volatile boolean hasMadeASuccessfulCall = false;
private RetryPolicy defaultPolicy;
private Map methodNameToPolicyMap;
@@ -78,47 +80,82 @@ class RetryInvocationHandler implements RpcInvocationHandler {
invocationAttemptFailoverCount = proxyProviderFailoverCount;
}
try {
- return invokeMethod(method, args);
+ Object ret = invokeMethod(method, args);
+ hasMadeASuccessfulCall = true;
+ return ret;
} catch (Exception e) {
boolean isMethodIdempotent = proxyProvider.getInterface()
.getMethod(method.getName(), method.getParameterTypes())
.isAnnotationPresent(Idempotent.class);
RetryAction action = policy.shouldRetry(e, retries++, invocationFailoverCount,
isMethodIdempotent);
- if (action == RetryAction.FAIL) {
- LOG.warn("Exception while invoking " + method.getName()
- + " of " + currentProxy.getClass() + ". Not retrying.", e);
- if (!method.getReturnType().equals(Void.TYPE)) {
- throw e; // non-void methods can't fail without an exception
+ if (action.action == RetryAction.RetryDecision.FAIL) {
+ if (action.reason != null) {
+ LOG.warn("Exception while invoking " +
+ currentProxy.getClass() + "." + method.getName() +
+ ". Not retrying because " + action.reason, e);
}
- return null;
- } else if (action == RetryAction.FAILOVER_AND_RETRY) {
- LOG.warn("Exception while invoking " + method.getName()
- + " of " + currentProxy.getClass()
- + " after " + invocationFailoverCount + " fail over attempts."
- + " Trying to fail over.", e);
- // Make sure that concurrent failed method invocations only cause a
- // single actual fail over.
- synchronized (proxyProvider) {
- if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
- proxyProvider.performFailover(currentProxy);
- proxyProviderFailoverCount++;
- currentProxy = proxyProvider.getProxy();
+ throw e;
+ } else { // retry or failover
+ // avoid logging the failover if this is the first call on this
+ // proxy object, and we successfully achieve the failover without
+ // any flip-flopping
+ boolean worthLogging =
+ !(invocationFailoverCount == 0 && !hasMadeASuccessfulCall);
+ worthLogging |= LOG.isDebugEnabled();
+ if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY &&
+ worthLogging) {
+ String msg = "Exception while invoking " + method.getName()
+ + " of class " + currentProxy.getClass().getSimpleName();
+ if (invocationFailoverCount > 0) {
+ msg += " after " + invocationFailoverCount + " fail over attempts";
+ }
+ msg += ". Trying to fail over " + formatSleepMessage(action.delayMillis);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(msg, e);
} else {
- LOG.warn("A failover has occurred since the start of this method"
- + " invocation attempt.");
+ LOG.warn(msg);
+ }
+ } else {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Exception while invoking " + method.getName()
+ + " of class " + currentProxy.getClass().getSimpleName() +
+ ". Retrying " + formatSleepMessage(action.delayMillis), e);
}
}
- invocationFailoverCount++;
- }
- if(LOG.isDebugEnabled()) {
- LOG.debug("Exception while invoking " + method.getName()
- + " of " + currentProxy.getClass() + ". Retrying.", e);
+
+ if (action.delayMillis > 0) {
+ ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis);
+ }
+
+ if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
+ // Make sure that concurrent failed method invocations only cause a
+ // single actual fail over.
+ synchronized (proxyProvider) {
+ if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
+ proxyProvider.performFailover(currentProxy);
+ proxyProviderFailoverCount++;
+ currentProxy = proxyProvider.getProxy();
+ } else {
+ LOG.warn("A failover has occurred since the start of this method"
+ + " invocation attempt.");
+ }
+ }
+ invocationFailoverCount++;
+ }
}
}
}
}
-
+
+ private static String formatSleepMessage(long millis) {
+ if (millis > 0) {
+ return "after sleeping for " + millis + "ms.";
+ } else {
+ return "immediately.";
+ }
+ }
+
private Object invokeMethod(Method method, Object[] args) throws Throwable {
try {
if (!method.isAccessible()) {
@@ -137,9 +174,7 @@ class RetryInvocationHandler implements RpcInvocationHandler {
@Override //RpcInvocationHandler
public ConnectionId getConnectionId() {
- RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
- .getInvocationHandler(currentProxy);
- return inv.getConnectionId();
+ return RPC.getConnectionIdForProxy(currentProxy);
}
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
index 3634e18673..2be8b75999 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
@@ -33,6 +33,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
+import com.google.common.annotations.VisibleForTesting;
+
/**
*
* A collection of useful implementations of {@link RetryPolicy}.
@@ -42,6 +44,8 @@ public class RetryPolicies {
public static final Log LOG = LogFactory.getLog(RetryPolicies.class);
+ private static final Random RAND = new Random();
+
/**
*
* Try once, and fail by re-throwing the exception.
@@ -50,14 +54,6 @@ public class RetryPolicies {
*/
public static final RetryPolicy TRY_ONCE_THEN_FAIL = new TryOnceThenFail();
- /**
- *
- * Try once, and fail silently for void methods, or by
- * re-throwing the exception for non-void methods.
- *
- */
- public static final RetryPolicy TRY_ONCE_DONT_FAIL = new TryOnceDontFail();
-
/**
*
* Keep trying forever.
@@ -137,16 +133,17 @@ public class RetryPolicies {
public static final RetryPolicy failoverOnNetworkException(
RetryPolicy fallbackPolicy, int maxFailovers) {
- return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers);
+ return failoverOnNetworkException(fallbackPolicy, maxFailovers, 0, 0);
+ }
+
+ public static final RetryPolicy failoverOnNetworkException(
+ RetryPolicy fallbackPolicy, int maxFailovers, long delayMillis,
+ long maxDelayBase) {
+ return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers,
+ delayMillis, maxDelayBase);
}
static class TryOnceThenFail implements RetryPolicy {
- public RetryAction shouldRetry(Exception e, int retries, int failovers,
- boolean isMethodIdempotent) throws Exception {
- throw e;
- }
- }
- static class TryOnceDontFail implements RetryPolicy {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
return RetryAction.FAIL;
@@ -174,14 +171,10 @@ public class RetryPolicies {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
if (retries >= maxRetries) {
- throw e;
+ return RetryAction.FAIL;
}
- try {
- timeUnit.sleep(calculateSleepTime(retries));
- } catch (InterruptedException ie) {
- // retry
- }
- return RetryAction.RETRY;
+ return new RetryAction(RetryAction.RetryDecision.RETRY,
+ timeUnit.toMillis(calculateSleepTime(retries)));
}
protected abstract long calculateSleepTime(int retries);
@@ -268,7 +261,7 @@ public class RetryPolicies {
}
static class ExponentialBackoffRetry extends RetryLimited {
- private Random r = new Random();
+
public ExponentialBackoffRetry(
int maxRetries, long sleepTime, TimeUnit timeUnit) {
super(maxRetries, sleepTime, timeUnit);
@@ -276,16 +269,19 @@ public class RetryPolicies {
@Override
protected long calculateSleepTime(int retries) {
- return sleepTime*r.nextInt(1<<(retries+1));
+ return calculateExponentialTime(sleepTime, retries + 1);
}
}
- /*
+ /**
* Fail over and retry in the case of:
* Remote StandbyException (server is up, but is not the active server)
* Immediate socket exceptions (e.g. no route to host, econnrefused)
* Socket exceptions after initial connection when operation is idempotent
*
+ * The first failover is immediate, while all subsequent failovers wait an
+ * exponentially-increasing random amount of time.
+ *
* Fail immediately in the case of:
* Socket exceptions after initial connection when operation is not idempotent
*
@@ -295,33 +291,49 @@ public class RetryPolicies {
private RetryPolicy fallbackPolicy;
private int maxFailovers;
+ private long delayMillis;
+ private long maxDelayBase;
public FailoverOnNetworkExceptionRetry(RetryPolicy fallbackPolicy,
int maxFailovers) {
+ this(fallbackPolicy, maxFailovers, 0, 0);
+ }
+
+ public FailoverOnNetworkExceptionRetry(RetryPolicy fallbackPolicy,
+ int maxFailovers, long delayMillis, long maxDelayBase) {
this.fallbackPolicy = fallbackPolicy;
this.maxFailovers = maxFailovers;
+ this.delayMillis = delayMillis;
+ this.maxDelayBase = maxDelayBase;
}
@Override
public RetryAction shouldRetry(Exception e, int retries,
int failovers, boolean isMethodIdempotent) throws Exception {
if (failovers >= maxFailovers) {
- LOG.info("Failovers (" + failovers + ") exceeded maximum allowed ("
+ return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
+ "failovers (" + failovers + ") exceeded maximum allowed ("
+ maxFailovers + ")");
- return RetryAction.FAIL;
}
if (e instanceof ConnectException ||
e instanceof NoRouteToHostException ||
e instanceof UnknownHostException ||
- e instanceof StandbyException) {
- return RetryAction.FAILOVER_AND_RETRY;
+ e instanceof StandbyException ||
+ isWrappedStandbyException(e)) {
+ return new RetryAction(
+ RetryAction.RetryDecision.FAILOVER_AND_RETRY,
+ // retry immediately if this is our first failover, sleep otherwise
+ failovers == 0 ? 0 :
+ calculateExponentialTime(delayMillis, failovers, maxDelayBase));
} else if (e instanceof SocketException ||
- e instanceof IOException) {
+ (e instanceof IOException && !(e instanceof RemoteException))) {
if (isMethodIdempotent) {
return RetryAction.FAILOVER_AND_RETRY;
} else {
- return RetryAction.FAIL;
+ return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
+ "the invoked method is not idempotent, and unable to determine " +
+ "whether it was invoked");
}
} else {
return fallbackPolicy.shouldRetry(e, retries, failovers,
@@ -330,4 +342,34 @@ public class RetryPolicies {
}
}
+
+ /**
+ * Return a value which is time increasing exponentially as a
+ * function of retries, +/- 0%-50% of that value, chosen
+ * randomly.
+ *
+ * @param time the base amount of time to work with
+ * @param retries the number of retries that have so occurred so far
+ * @param cap value at which to cap the base sleep time
+ * @return an amount of time to sleep
+ */
+ @VisibleForTesting
+ public static long calculateExponentialTime(long time, int retries,
+ long cap) {
+ long baseTime = Math.min(time * ((long)1 << retries), cap);
+ return (long) (baseTime * (RAND.nextFloat() + 0.5));
+ }
+
+ private static long calculateExponentialTime(long time, int retries) {
+ return calculateExponentialTime(time, retries, Long.MAX_VALUE);
+ }
+
+ private static boolean isWrappedStandbyException(Exception e) {
+ if (!(e instanceof RemoteException)) {
+ return false;
+ }
+ Exception unwrapped = ((RemoteException)e).unwrapRemoteException(
+ StandbyException.class);
+ return unwrapped instanceof StandbyException;
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java
index 4c4534ffb7..ed673e950f 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicy.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.io.retry;
import org.apache.hadoop.classification.InterfaceStability;
-
/**
*
* Specifies a policy for retrying method failures.
@@ -33,10 +32,39 @@ public interface RetryPolicy {
* Returned by {@link RetryPolicy#shouldRetry(Exception, int, int, boolean)}.
*/
@InterfaceStability.Evolving
- public enum RetryAction {
- FAIL,
- RETRY,
- FAILOVER_AND_RETRY
+ public static class RetryAction {
+
+ // A few common retry policies, with no delays.
+ public static final RetryAction FAIL =
+ new RetryAction(RetryDecision.FAIL);
+ public static final RetryAction RETRY =
+ new RetryAction(RetryDecision.RETRY);
+ public static final RetryAction FAILOVER_AND_RETRY =
+ new RetryAction(RetryDecision.FAILOVER_AND_RETRY);
+
+ public final RetryDecision action;
+ public final long delayMillis;
+ public final String reason;
+
+ public RetryAction(RetryDecision action) {
+ this(action, 0, null);
+ }
+
+ public RetryAction(RetryDecision action, long delayTime) {
+ this(action, delayTime, null);
+ }
+
+ public RetryAction(RetryDecision action, long delayTime, String reason) {
+ this.action = action;
+ this.delayMillis = delayTime;
+ this.reason = reason;
+ }
+
+ public enum RetryDecision {
+ FAIL,
+ RETRY,
+ FAILOVER_AND_RETRY
+ }
}
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index f09600d4b3..e5a2d7f15a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -227,6 +227,8 @@ public class Client {
private int maxIdleTime; //connections will be culled if it was idle for
//maxIdleTime msecs
private int maxRetries; //the max. no. of retries for socket connections
+ // the max. no. of retries for socket connections on time out exceptions
+ private int maxRetriesOnSocketTimeouts;
private boolean tcpNoDelay; // if T then disable Nagle's Algorithm
private boolean doPing; //do we need to send ping message
private int pingInterval; // how often sends ping to the server in msecs
@@ -250,6 +252,7 @@ public class Client {
this.rpcTimeout = remoteId.getRpcTimeout();
this.maxIdleTime = remoteId.getMaxIdleTime();
this.maxRetries = remoteId.getMaxRetries();
+ this.maxRetriesOnSocketTimeouts = remoteId.getMaxRetriesOnSocketTimeouts();
this.tcpNoDelay = remoteId.getTcpNoDelay();
this.doPing = remoteId.getDoPing();
this.pingInterval = remoteId.getPingInterval();
@@ -478,11 +481,8 @@ public class Client {
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
- /*
- * The max number of retries is 45, which amounts to 20s*45 = 15
- * minutes retries.
- */
- handleConnectionFailure(timeoutFailures++, 45, toe);
+ handleConnectionFailure(timeoutFailures++,
+ maxRetriesOnSocketTimeouts, toe);
} catch (IOException ie) {
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
@@ -1286,6 +1286,8 @@ public class Client {
private final int maxIdleTime; //connections will be culled if it was idle for
//maxIdleTime msecs
private final int maxRetries; //the max. no. of retries for socket connections
+ // the max. no. of retries for socket connections on time out exceptions
+ private final int maxRetriesOnSocketTimeouts;
private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
private final boolean doPing; //do we need to send ping message
private final int pingInterval; // how often sends ping to the server in msecs
@@ -1293,8 +1295,8 @@ public class Client {
ConnectionId(InetSocketAddress address, Class> protocol,
UserGroupInformation ticket, int rpcTimeout,
String serverPrincipal, int maxIdleTime,
- int maxRetries, boolean tcpNoDelay,
- boolean doPing, int pingInterval) {
+ int maxRetries, int maxRetriesOnSocketTimeouts,
+ boolean tcpNoDelay, boolean doPing, int pingInterval) {
this.protocol = protocol;
this.address = address;
this.ticket = ticket;
@@ -1302,6 +1304,7 @@ public class Client {
this.serverPrincipal = serverPrincipal;
this.maxIdleTime = maxIdleTime;
this.maxRetries = maxRetries;
+ this.maxRetriesOnSocketTimeouts = maxRetriesOnSocketTimeouts;
this.tcpNoDelay = tcpNoDelay;
this.doPing = doPing;
this.pingInterval = pingInterval;
@@ -1335,6 +1338,11 @@ public class Client {
return maxRetries;
}
+ /** max connection retries on socket time outs */
+ public int getMaxRetriesOnSocketTimeouts() {
+ return maxRetriesOnSocketTimeouts;
+ }
+
boolean getTcpNoDelay() {
return tcpNoDelay;
}
@@ -1369,6 +1377,9 @@ public class Client {
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT),
conf.getInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_DEFAULT),
+ conf.getInt(
+ CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+ CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT),
conf.getBoolean(CommonConfigurationKeysPublic.IPC_CLIENT_TCPNODELAY_KEY,
CommonConfigurationKeysPublic.IPC_CLIENT_TCPNODELAY_DEFAULT),
doPing,
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolTranslator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolTranslator.java
new file mode 100644
index 0000000000..5bf9dbaed1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolTranslator.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ipc;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * An interface implemented by client-side protocol translators to get the
+ * underlying proxy object the translator is operating on.
+ */
+@InterfaceAudience.Private
+public interface ProtocolTranslator {
+
+ /**
+ * Return the proxy object underlying this protocol translator.
+ * @return the proxy object underlying this protocol translator.
+ */
+ public Object getUnderlyingProxyObject();
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
index 4f85e905cd..eee364ccde 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
@@ -40,6 +40,7 @@ import javax.net.SocketFactory;
import org.apache.commons.logging.*;
import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
import org.apache.hadoop.net.NetUtils;
@@ -530,9 +531,24 @@ public class RPC {
* Returns the server address for a given proxy.
*/
public static InetSocketAddress getServerAddress(Object proxy) {
+ return getConnectionIdForProxy(proxy).getAddress();
+ }
+
+ /**
+ * Return the connection ID of the given object. If the provided object is in
+ * fact a protocol translator, we'll get the connection ID of the underlying
+ * proxy object.
+ *
+ * @param proxy the proxy object to get the connection ID of.
+ * @return the connection ID for the provided proxy object.
+ */
+ public static ConnectionId getConnectionIdForProxy(Object proxy) {
+ if (proxy instanceof ProtocolTranslator) {
+ proxy = ((ProtocolTranslator)proxy).getUnderlyingProxyObject();
+ }
RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
.getInvocationHandler(proxy);
- return inv.getConnectionId().getAddress();
+ return inv.getConnectionId();
}
/**
@@ -564,6 +580,12 @@ public class RPC {
* @param proxy the RPC proxy object to be stopped
*/
public static void stopProxy(Object proxy) {
+ if (proxy instanceof ProtocolTranslator) {
+ RPC.stopProxy(((ProtocolTranslator)proxy)
+ .getUnderlyingProxyObject());
+ return;
+ }
+
InvocationHandler invocationHandler = null;
try {
invocationHandler = Proxy.getInvocationHandler(proxy);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 2b35598738..5f642c4f69 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -1671,6 +1671,10 @@ public abstract class Server {
// on the server side, as opposed to just a normal exceptional
// result.
LOG.warn(logMsg, e);
+ } else if (e instanceof StandbyException) {
+ // Don't log the whole stack trace of these exceptions.
+ // Way too noisy!
+ LOG.info(logMsg);
} else {
LOG.info(logMsg, e);
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/StandbyException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/StandbyException.java
index 49f4fadfd5..7a168619af 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/StandbyException.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/StandbyException.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.ipc;
+import java.io.IOException;
+
import org.apache.hadoop.classification.InterfaceStability;
/**
@@ -24,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
* set of servers in which only a subset may be active.
*/
@InterfaceStability.Evolving
-public class StandbyException extends Exception {
+public class StandbyException extends IOException {
static final long serialVersionUID = 0x12308AD010L;
public StandbyException(String msg) {
super(msg);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
index 2685887464..43132d263a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
@@ -23,6 +23,7 @@ import java.net.URI;
import java.net.URL;
import java.net.UnknownHostException;
import java.security.AccessController;
+import java.security.PrivilegedAction;
import java.util.Arrays;
import java.util.List;
import java.util.ServiceLoader;
@@ -448,6 +449,27 @@ public class SecurityUtil {
return buildTokenService(NetUtils.createSocketAddr(uri.getAuthority()));
}
+ /**
+ * Perform the given action as the daemon's login user. If the login
+ * user cannot be determined, this will log a FATAL error and exit
+ * the whole JVM.
+ */
+ public static T doAsLoginUserOrFatal(PrivilegedAction action) {
+ if (UserGroupInformation.isSecurityEnabled()) {
+ UserGroupInformation ugi = null;
+ try {
+ ugi = UserGroupInformation.getLoginUser();
+ } catch (IOException e) {
+ LOG.fatal("Exception while getting login user", e);
+ e.printStackTrace();
+ Runtime.getRuntime().exit(-1);
+ }
+ return ugi.doAs(action);
+ } else {
+ return action.run();
+ }
+ }
+
/**
* Resolves a host subject to the security requirements determined by
* hadoop.security.token.service.use_ip.
@@ -597,5 +619,5 @@ public class SecurityUtil {
void setSearchDomains(String ... domains) {
searchDomains = Arrays.asList(domains);
}
- }
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
index 3c2e666a39..11df9811b2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.util.Daemon;
+import com.google.common.base.Preconditions;
+
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
public abstract
@@ -84,6 +86,12 @@ extends AbstractDelegationTokenIdentifier>
private Thread tokenRemoverThread;
protected volatile boolean running;
+ /**
+ * If the delegation token update thread holds this lock, it will
+ * not get interrupted.
+ */
+ protected Object noInterruptsLock = new Object();
+
public AbstractDelegationTokenSecretManager(long delegationKeyUpdateInterval,
long delegationTokenMaxLifetime, long delegationTokenRenewInterval,
long delegationTokenRemoverScanInterval) {
@@ -95,6 +103,7 @@ extends AbstractDelegationTokenIdentifier>
/** should be called before this object is used */
public void startThreads() throws IOException {
+ Preconditions.checkState(!running);
updateCurrentKey();
synchronized (this) {
running = true;
@@ -354,12 +363,21 @@ extends AbstractDelegationTokenIdentifier>
}
}
- public synchronized void stopThreads() {
+ public void stopThreads() {
if (LOG.isDebugEnabled())
LOG.debug("Stopping expired delegation token remover thread");
running = false;
+
if (tokenRemoverThread != null) {
- tokenRemoverThread.interrupt();
+ synchronized (noInterruptsLock) {
+ tokenRemoverThread.interrupt();
+ }
+ try {
+ tokenRemoverThread.join();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(
+ "Unable to join on token removal thread", e);
+ }
}
}
@@ -395,7 +413,7 @@ extends AbstractDelegationTokenIdentifier>
lastTokenCacheCleanup = now;
}
try {
- Thread.sleep(5000); // 5 seconds
+ Thread.sleep(Math.min(5000, keyUpdateInterval)); // 5 seconds
} catch (InterruptedException ie) {
LOG
.error("InterruptedExcpetion recieved for ExpiredTokenRemover thread "
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ThreadUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ThreadUtil.java
new file mode 100644
index 0000000000..6e4dfafdf7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ThreadUtil.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceStability.Evolving
+public class ThreadUtil {
+
+ private static final Log LOG = LogFactory.getLog(ThreadUtil.class);
+
+ /**
+ * Cause the current thread to sleep as close as possible to the provided
+ * number of milliseconds. This method will log and ignore any
+ * {@link InterruptedException} encountered.
+ *
+ * @param millis the number of milliseconds for the current thread to sleep
+ */
+ public static void sleepAtLeastIgnoreInterrupts(long millis) {
+ long start = System.currentTimeMillis();
+ while (System.currentTimeMillis() - start < millis) {
+ long timeToSleep = millis -
+ (System.currentTimeMillis() - start);
+ try {
+ Thread.sleep(timeToSleep);
+ } catch (InterruptedException ie) {
+ LOG.warn("interrupted while sleeping", ie);
+ }
+ }
+ }
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml
index b3e12d14e2..2fd9f8d2a9 100644
--- a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml
+++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml
@@ -216,6 +216,13 @@
group list is separated by a blank. For e.g. "alice,bob users,wheel".
A special value of "*" means all users are allowed.
+
+
+ security.ha.service.protocol.acl
+ *
+ ACL for HAService protocol used by HAAdmin to manage the
+ active and stand-by states of namenode.
+ security.mrhs.client.protocol.acl
diff --git a/hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto b/hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto
new file mode 100644
index 0000000000..a3fd86c040
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.hadoop.ha.proto";
+option java_outer_classname = "HAServiceProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+enum HAServiceStateProto {
+ INITIALIZING = 0;
+ ACTIVE = 1;
+ STANDBY = 2;
+}
+
+/**
+ * void request
+ */
+message MonitorHealthRequestProto {
+}
+
+/**
+ * void response
+ */
+message MonitorHealthResponseProto {
+}
+
+/**
+ * void request
+ */
+message TransitionToActiveRequestProto {
+}
+
+/**
+ * void response
+ */
+message TransitionToActiveResponseProto {
+}
+
+/**
+ * void request
+ */
+message TransitionToStandbyRequestProto {
+}
+
+/**
+ * void response
+ */
+message TransitionToStandbyResponseProto {
+}
+
+/**
+ * void request
+ */
+message GetServiceStateRequestProto {
+}
+
+/**
+ * Returns the state of the service
+ */
+message GetServiceStateResponseProto {
+ required HAServiceStateProto state = 1;
+}
+
+/**
+ * void request
+ */
+message ReadyToBecomeActiveRequestProto {
+}
+
+/**
+ * Returns true if service is ready to become active
+ */
+message ReadyToBecomeActiveResponseProto {
+ required bool readyToBecomeActive = 1;
+}
+
+/**
+ * Protocol interface provides High availability related
+ * primitives to monitor and failover a service.
+ *
+ * For details see o.a.h.ha.HAServiceProtocol.
+ */
+service HAServiceProtocolService {
+ /**
+ * Monitor the health of a service.
+ */
+ rpc monitorHealth(MonitorHealthRequestProto)
+ returns(MonitorHealthResponseProto);
+
+ /**
+ * Request service to tranisition to active state.
+ */
+ rpc transitionToActive(TransitionToActiveRequestProto)
+ returns(TransitionToActiveResponseProto);
+
+ /**
+ * Request service to transition to standby state.
+ */
+ rpc transitionToStandby(TransitionToStandbyRequestProto)
+ returns(TransitionToStandbyResponseProto);
+
+ /**
+ * Get the current state of the service.
+ */
+ rpc getServiceState(GetServiceStateRequestProto)
+ returns(GetServiceStateResponseProto);
+
+ /**
+ * Check if the service is ready to become active
+ */
+ rpc readyToBecomeActive(ReadyToBecomeActiveRequestProto)
+ returns(ReadyToBecomeActiveResponseProto);
+}
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 8fc45c5efc..a9684000b6 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -487,6 +487,14 @@
+
+ ipc.client.connect.max.retries.on.timeouts
+ 45
+ Indicates the number of retries a client will make on socket timeout
+ to establish a server connection.
+
+
+
ipc.server.listen.queue.size128
@@ -849,4 +857,30 @@
+
+ dfs.ha.fencing.methods
+
+
+ List of fencing methods to use for service fencing. May contain
+ builtin methods (eg shell and sshfence) or user-defined method.
+
+
+
+
+ dfs.ha.fencing.ssh.connect-timeout
+ 30000
+
+ SSH connection timeout, in milliseconds, to use with the builtin
+ sshfence fencer.
+
+
+
+
+ dfs.ha.fencing.ssh.private-key-files
+
+
+ The SSH private key files to use with the builtin sshfence fencer.
+
+
+
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElector.java
new file mode 100644
index 0000000000..fec350d3bc
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElector.java
@@ -0,0 +1,527 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ha;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.Watcher.Event;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.Assert;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
+import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
+
+public class TestActiveStandbyElector {
+
+ static ZooKeeper mockZK;
+ static int count;
+ static ActiveStandbyElectorCallback mockApp;
+ static final byte[] data = new byte[8];
+
+ ActiveStandbyElectorTester elector;
+
+ class ActiveStandbyElectorTester extends ActiveStandbyElector {
+ ActiveStandbyElectorTester(String hostPort, int timeout, String parent,
+ List acl, ActiveStandbyElectorCallback app) throws IOException {
+ super(hostPort, timeout, parent, acl, app);
+ }
+
+ @Override
+ public ZooKeeper getNewZooKeeper() {
+ ++TestActiveStandbyElector.count;
+ return TestActiveStandbyElector.mockZK;
+ }
+
+ }
+
+ private static final String zkParentName = "/zookeeper";
+ private static final String zkLockPathName = "/zookeeper/"
+ + ActiveStandbyElector.LOCKFILENAME;
+
+ @Before
+ public void init() throws IOException {
+ count = 0;
+ mockZK = Mockito.mock(ZooKeeper.class);
+ mockApp = Mockito.mock(ActiveStandbyElectorCallback.class);
+ elector = new ActiveStandbyElectorTester("hostPort", 1000, zkParentName,
+ Ids.OPEN_ACL_UNSAFE, mockApp);
+ }
+
+ /**
+ * verify that joinElection checks for null data
+ */
+ @Test(expected = HadoopIllegalArgumentException.class)
+ public void testJoinElectionException() {
+ elector.joinElection(null);
+ }
+
+ /**
+ * verify that joinElection tries to create ephemeral lock znode
+ */
+ @Test
+ public void testJoinElection() {
+ elector.joinElection(data);
+ Mockito.verify(mockZK, Mockito.times(1)).create(zkLockPathName, data,
+ Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+ }
+
+ /**
+ * verify that successful znode create result becomes active and monitoring is
+ * started
+ */
+ @Test
+ public void testCreateNodeResultBecomeActive() {
+ elector.joinElection(data);
+ elector.processResult(Code.OK.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+
+ // monitor callback verifies the leader is ephemeral owner of lock but does
+ // not call becomeActive since its already active
+ Stat stat = new Stat();
+ stat.setEphemeralOwner(1L);
+ Mockito.when(mockZK.getSessionId()).thenReturn(1L);
+ elector.processResult(Code.OK.intValue(), zkLockPathName, null, stat);
+ // should not call neutral mode/standby/active
+ Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
+ Mockito.verify(mockApp, Mockito.times(0)).becomeStandby();
+ Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
+ // another joinElection not called.
+ Mockito.verify(mockZK, Mockito.times(1)).create(zkLockPathName, data,
+ Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+ // no new monitor called
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+ }
+
+ /**
+ * verify that znode create for existing node and no retry becomes standby and
+ * monitoring is started
+ */
+ @Test
+ public void testCreateNodeResultBecomeStandby() {
+ elector.joinElection(data);
+
+ elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+ }
+
+ /**
+ * verify that znode create error result in fatal error
+ */
+ @Test
+ public void testCreateNodeResultError() {
+ elector.joinElection(data);
+
+ elector.processResult(Code.APIERROR.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
+ "Received create error from Zookeeper. code:APIERROR");
+ }
+
+ /**
+ * verify that retry of network errors verifies master by session id and
+ * becomes active if they match. monitoring is started.
+ */
+ @Test
+ public void testCreateNodeResultRetryBecomeActive() {
+ elector.joinElection(data);
+
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ // 4 errors results in fatalError
+ Mockito
+ .verify(mockApp, Mockito.times(1))
+ .notifyFatalError(
+ "Received create error from Zookeeper. code:CONNECTIONLOSS. "+
+ "Not retrying further znode create connection errors.");
+
+ elector.joinElection(data);
+ // recreate connection via getNewZooKeeper
+ Assert.assertEquals(2, TestActiveStandbyElector.count);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+
+ Stat stat = new Stat();
+ stat.setEphemeralOwner(1L);
+ Mockito.when(mockZK.getSessionId()).thenReturn(1L);
+ elector.processResult(Code.OK.intValue(), zkLockPathName, null, stat);
+ Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+ Mockito.verify(mockZK, Mockito.times(6)).create(zkLockPathName, data,
+ Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+ }
+
+ /**
+ * verify that retry of network errors verifies active by session id and
+ * becomes standby if they dont match. monitoring is started.
+ */
+ @Test
+ public void testCreateNodeResultRetryBecomeStandby() {
+ elector.joinElection(data);
+
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+
+ Stat stat = new Stat();
+ stat.setEphemeralOwner(0);
+ Mockito.when(mockZK.getSessionId()).thenReturn(1L);
+ elector.processResult(Code.OK.intValue(), zkLockPathName, null, stat);
+ Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+ }
+
+ /**
+ * verify that if create znode results in nodeexists and that znode is deleted
+ * before exists() watch is set then the return of the exists() method results
+ * in attempt to re-create the znode and become active
+ */
+ @Test
+ public void testCreateNodeResultRetryNoNode() {
+ elector.joinElection(data);
+
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
+ zkLockPathName);
+ Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+ elector, null);
+
+ elector.processResult(Code.NONODE.intValue(), zkLockPathName, null,
+ (Stat) null);
+ Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
+ Mockito.verify(mockZK, Mockito.times(4)).create(zkLockPathName, data,
+ Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+ }
+
+ /**
+ * verify that more than 3 network error retries result fatalError
+ */
+ @Test
+ public void testStatNodeRetry() {
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ (Stat) null);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ (Stat) null);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ (Stat) null);
+ elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
+ (Stat) null);
+ Mockito
+ .verify(mockApp, Mockito.times(1))
+ .notifyFatalError(
+ "Received stat error from Zookeeper. code:CONNECTIONLOSS. "+
+ "Not retrying further znode monitoring connection errors.");
+ }
+
+ /**
+ * verify error in exists() callback results in fatal error
+ */
+ @Test
+ public void testStatNodeError() {
+ elector.processResult(Code.RUNTIMEINCONSISTENCY.intValue(), zkLockPathName,
+ null, (Stat) null);
+ Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
+ Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
+ "Received stat error from Zookeeper. code:RUNTIMEINCONSISTENCY");
+ }
+
+ /**
+ * verify behavior of watcher.process callback with non-node event
+ */
+ @Test
+ public void testProcessCallbackEventNone() {
+ elector.joinElection(data);
+
+ WatchedEvent mockEvent = Mockito.mock(WatchedEvent.class);
+ Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.None);
+
+ // first SyncConnected should not do anything
+ Mockito.when(mockEvent.getState()).thenReturn(
+ Event.KeeperState.SyncConnected);
+ elector.process(mockEvent);
+ Mockito.verify(mockZK, Mockito.times(0)).exists(Mockito.anyString(),
+ Mockito.anyBoolean(), Mockito. anyObject(),
+ Mockito.
+
+
+
+
+
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 0b4da80e8a..3f85de096b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -387,6 +387,7 @@
CHANGES.txt
+ CHANGES.HDFS-1623.txt.idea/**src/main/conf/*src/main/docs/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
index 707182ec5c..636471a450 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
@@ -129,6 +129,12 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
return null;
}
+ // TODO(HA): Test this.
+ @Override
+ public boolean isInProgress() {
+ return true;
+ }
+
/**
* Input stream implementation which can be used by
* FSEditLogOp.Reader
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
index 7fa90269ec..047efd51f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
@@ -312,8 +312,10 @@ public class BookKeeperJournalManager implements JournalManager {
}
}
+ // TODO(HA): Handle inProgressOk
@Override
- public EditLogInputStream getInputStream(long fromTxnId) throws IOException {
+ public EditLogInputStream getInputStream(long fromTxnId, boolean inProgressOk)
+ throws IOException {
for (EditLogLedgerMetadata l : getLedgerList()) {
if (l.getFirstTxId() == fromTxnId) {
try {
@@ -329,8 +331,10 @@ public class BookKeeperJournalManager implements JournalManager {
throw new IOException("No ledger for fromTxnId " + fromTxnId + " found.");
}
+ // TODO(HA): Handle inProgressOk
@Override
- public long getNumberOfTransactions(long fromTxnId) throws IOException {
+ public long getNumberOfTransactions(long fromTxnId, boolean inProgressOk)
+ throws IOException {
long count = 0;
long expectedStart = 0;
for (EditLogLedgerMetadata l : getLedgerList()) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java
index b949bc200e..5937fa8295 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java
@@ -195,7 +195,7 @@ public class TestBookKeeperJournalManager {
out.close();
bkjm.finalizeLogSegment(1, 100);
- long numTrans = bkjm.getNumberOfTransactions(1);
+ long numTrans = bkjm.getNumberOfTransactions(1, true);
assertEquals(100, numTrans);
}
@@ -218,17 +218,17 @@ public class TestBookKeeperJournalManager {
}
zkc.delete(bkjm.finalizedLedgerZNode(DEFAULT_SEGMENT_SIZE+1, DEFAULT_SEGMENT_SIZE*2), -1);
- long numTrans = bkjm.getNumberOfTransactions(1);
+ long numTrans = bkjm.getNumberOfTransactions(1, true);
assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
try {
- numTrans = bkjm.getNumberOfTransactions(DEFAULT_SEGMENT_SIZE+1);
+ numTrans = bkjm.getNumberOfTransactions(DEFAULT_SEGMENT_SIZE+1, true);
fail("Should have thrown corruption exception by this point");
} catch (JournalManager.CorruptionException ce) {
// if we get here, everything is going good
}
- numTrans = bkjm.getNumberOfTransactions((DEFAULT_SEGMENT_SIZE*2)+1);
+ numTrans = bkjm.getNumberOfTransactions((DEFAULT_SEGMENT_SIZE*2)+1, true);
assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
}
@@ -262,7 +262,7 @@ public class TestBookKeeperJournalManager {
out.abort();
out.close();
- long numTrans = bkjm.getNumberOfTransactions(1);
+ long numTrans = bkjm.getNumberOfTransactions(1, true);
assertEquals((txid-1), numTrans);
}
@@ -357,7 +357,7 @@ public class TestBookKeeperJournalManager {
bkjm.finalizeLogSegment(1, numTransactions);
- EditLogInputStream in = bkjm.getInputStream(1);
+ EditLogInputStream in = bkjm.getInputStream(1, true);
try {
assertEquals(numTransactions,
FSEditLogTestUtil.countTransactionsInStream(in));
@@ -392,4 +392,4 @@ public class TestBookKeeperJournalManager {
assertNotNull(zkc.exists(bkjm.finalizedLedgerZNode(1, 100), false));
assertNull(zkc.exists(bkjm.inprogressZNode(), false));
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 4c56bb3f14..a01c939e42 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -32,6 +32,7 @@ function print_usage(){
echo " namenode run the DFS namenode"
echo " datanode run a DFS datanode"
echo " dfsadmin run a DFS admin client"
+ echo " haadmin run a DFS HA admin client"
echo " fsck run a DFS filesystem checking utility"
echo " balancer run a cluster balancing utility"
echo " jmxget get JMX exported values from NameNode or DataNode."
@@ -86,6 +87,10 @@ elif [ "$COMMAND" = "dfs" ] ; then
elif [ "$COMMAND" = "dfsadmin" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.DFSAdmin
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
+elif [ "$COMMAND" = "haadmin" ] ; then
+ CLASS=org.apache.hadoop.hdfs.tools.DFSHAAdmin
+ CLASSPATH=${CLASSPATH}:${TOOL_PATH}
+ HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
elif [ "$COMMAND" = "fsck" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.DFSck
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
index 5a45f51ee5..82d0c3663c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
@@ -80,8 +80,7 @@ public class Hdfs extends AbstractFileSystem {
throw new IOException("Incomplete HDFS URI, no host: " + theUri);
}
- InetSocketAddress namenode = NameNode.getAddress(theUri.getAuthority());
- this.dfs = new DFSClient(namenode, conf, getStatistics());
+ this.dfs = new DFSClient(theUri, conf, getStatistics());
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 359fd47a6c..88b36b73b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -1,4 +1,3 @@
-
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@@ -30,6 +29,8 @@ import java.net.InetSocketAddress;
import java.net.NetworkInterface;
import java.net.Socket;
import java.net.SocketException;
+import java.net.URI;
+import java.net.URISyntaxException;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
@@ -60,6 +61,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@@ -83,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@@ -105,7 +108,8 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+
+import com.google.common.base.Preconditions;
/********************************************************
* DFSClient can connect to a Hadoop Filesystem and
@@ -124,7 +128,9 @@ public class DFSClient implements java.io.Closeable {
public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
final ClientProtocol namenode;
- private final InetSocketAddress nnAddress;
+ /* The service used for delegation tokens */
+ private Text dtService;
+
final UserGroupInformation ugi;
volatile boolean clientRunning = true;
private volatile FsServerDefaults serverDefaults;
@@ -143,6 +149,9 @@ public class DFSClient implements java.io.Closeable {
* DFSClient configuration
*/
static class Conf {
+ final int maxFailoverAttempts;
+ final int failoverSleepBaseMillis;
+ final int failoverSleepMaxMillis;
final int maxBlockAcquireFailures;
final int confTime;
final int ioBufferSize;
@@ -164,6 +173,16 @@ public class DFSClient implements java.io.Closeable {
final boolean useLegacyBlockReader;
Conf(Configuration conf) {
+ maxFailoverAttempts = conf.getInt(
+ DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
+ DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
+ failoverSleepBaseMillis = conf.getInt(
+ DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
+ DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
+ failoverSleepMaxMillis = conf.getInt(
+ DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
+ DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
+
maxBlockAcquireFailures = conf.getInt(
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
@@ -236,6 +255,7 @@ public class DFSClient implements java.io.Closeable {
*/
private final Map filesBeingWritten
= new HashMap();
+
private boolean shortCircuitLocalReads;
/**
@@ -247,59 +267,69 @@ public class DFSClient implements java.io.Closeable {
public DFSClient(Configuration conf) throws IOException {
this(NameNode.getAddress(conf), conf);
}
+
+ public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
+ this(NameNode.getUri(address), conf);
+ }
/**
- * Same as this(nameNodeAddr, conf, null);
+ * Same as this(nameNodeUri, conf, null);
* @see #DFSClient(InetSocketAddress, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
*/
- public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf
+ public DFSClient(URI nameNodeUri, Configuration conf
) throws IOException {
- this(nameNodeAddr, conf, null);
+ this(nameNodeUri, conf, null);
}
/**
- * Same as this(nameNodeAddr, null, conf, stats);
+ * Same as this(nameNodeUri, null, conf, stats);
* @see #DFSClient(InetSocketAddress, ClientProtocol, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
*/
- public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
+ public DFSClient(URI nameNodeUri, Configuration conf,
FileSystem.Statistics stats)
throws IOException {
- this(nameNodeAddr, null, conf, stats);
+ this(nameNodeUri, null, conf, stats);
}
-
+
/**
- * Create a new DFSClient connected to the given nameNodeAddr or rpcNamenode.
- * Exactly one of nameNodeAddr or rpcNamenode must be null.
+ * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
+ * Exactly one of nameNodeUri or rpcNamenode must be null.
*/
- DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
+ DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
Configuration conf, FileSystem.Statistics stats)
throws IOException {
// Copy only the required DFSClient configuration
this.dfsClientConf = new Conf(conf);
this.conf = conf;
this.stats = stats;
- this.nnAddress = nameNodeAddr;
this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
// The hdfsTimeout is currently the same as the ipc timeout
this.hdfsTimeout = Client.getTimeout(conf);
this.ugi = UserGroupInformation.getCurrentUser();
- final String authority = nameNodeAddr == null? "null":
- nameNodeAddr.getHostName() + ":" + nameNodeAddr.getPort();
+
+ final String authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
this.leaserenewer = LeaseRenewer.getInstance(authority, ugi, this);
this.clientName = leaserenewer.getClientName(dfsClientConf.taskId);
+
this.socketCache = new SocketCache(dfsClientConf.socketCacheCapacity);
- if (nameNodeAddr != null && rpcNamenode == null) {
- this.namenode = DFSUtil.createNamenode(nameNodeAddr, conf, ugi);
- } else if (nameNodeAddr == null && rpcNamenode != null) {
- //This case is used for testing.
+
+
+ if (rpcNamenode != null) {
+ // This case is used for testing.
+ Preconditions.checkArgument(nameNodeUri == null);
this.namenode = rpcNamenode;
+ dtService = null;
} else {
- throw new IllegalArgumentException(
- "Expecting exactly one of nameNodeAddr and rpcNamenode being null: "
- + "nameNodeAddr=" + nameNodeAddr + ", rpcNamenode=" + rpcNamenode);
+ Preconditions.checkArgument(nameNodeUri != null,
+ "null URI");
+ NameNodeProxies.ProxyAndInfo proxyInfo =
+ NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class);
+ this.dtService = proxyInfo.getDelegationTokenService();
+ this.namenode = proxyInfo.getProxy();
}
+
// read directly from the block file if configured.
this.shortCircuitLocalReads = conf.getBoolean(
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
@@ -388,20 +418,8 @@ public class DFSClient implements java.io.Closeable {
/**
* Close connections the Namenode.
- * The namenode variable is either a rpcProxy passed by a test or
- * created using the protocolTranslator which is closeable.
- * If closeable then call close, else close using RPC.stopProxy().
*/
void closeConnectionToNamenode() {
- if (namenode instanceof Closeable) {
- try {
- ((Closeable) namenode).close();
- return;
- } catch (IOException e) {
- // fall through - lets try the stopProxy
- LOG.warn("Exception closing namenode, stopping the proxy");
- }
- }
RPC.stopProxy(namenode);
}
@@ -491,11 +509,13 @@ public class DFSClient implements java.io.Closeable {
*/
public Token getDelegationToken(Text renewer)
throws IOException {
- Token result =
+ assert dtService != null;
+ Token token =
namenode.getDelegationToken(renewer);
- SecurityUtil.setTokenService(result, nnAddress);
- LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(result));
- return result;
+ token.setService(this.dtService);
+
+ LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+ return token;
}
/**
@@ -625,13 +645,8 @@ public class DFSClient implements java.io.Closeable {
@Override
public long renew(Token> token, Configuration conf) throws IOException {
Token delToken =
- (Token) token;
- LOG.info("Renewing " +
- DelegationTokenIdentifier.stringifyToken(delToken));
- ClientProtocol nn =
- DFSUtil.createNamenode
- (SecurityUtil.getTokenServiceAddr(delToken),
- conf, UserGroupInformation.getCurrentUser());
+ (Token) token;
+ ClientProtocol nn = getNNProxy(delToken, conf);
try {
return nn.renewDelegationToken(delToken);
} catch (RemoteException re) {
@@ -647,9 +662,7 @@ public class DFSClient implements java.io.Closeable {
(Token) token;
LOG.info("Cancelling " +
DelegationTokenIdentifier.stringifyToken(delToken));
- ClientProtocol nn = DFSUtil.createNamenode(
- SecurityUtil.getTokenServiceAddr(delToken), conf,
- UserGroupInformation.getCurrentUser());
+ ClientProtocol nn = getNNProxy(delToken, conf);
try {
nn.cancelDelegationToken(delToken);
} catch (RemoteException re) {
@@ -657,6 +670,31 @@ public class DFSClient implements java.io.Closeable {
AccessControlException.class);
}
}
+
+ private static ClientProtocol getNNProxy(
+ Token token, Configuration conf)
+ throws IOException {
+ URI uri = HAUtil.getServiceUriFromToken(token);
+ if (HAUtil.isTokenForLogicalUri(token) &&
+ !HAUtil.isLogicalUri(conf, uri)) {
+ // If the token is for a logical nameservice, but the configuration
+ // we have disagrees about that, we can't actually renew it.
+ // This can be the case in MR, for example, if the RM doesn't
+ // have all of the HA clusters configured in its configuration.
+ throw new IOException("Unable to map logical nameservice URI '" +
+ uri + "' to a NameNode. Local configuration does not have " +
+ "a failover proxy provider configured.");
+ }
+
+ NameNodeProxies.ProxyAndInfo info =
+ NameNodeProxies.createProxy(conf, uri, ClientProtocol.class);
+ assert info.getDelegationTokenService().equals(token.getService()) :
+ "Returned service '" + info.getDelegationTokenService().toString() +
+ "' doesn't match expected service '" +
+ token.getService().toString() + "'";
+
+ return info.getProxy();
+ }
@Override
public boolean isManaged(Token> token) throws IOException {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index fef6d8b9ca..4187f1c5c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -48,6 +48,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT = "DEFAULT";
public static final String DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY = "dfs.client.socketcache.capacity";
public static final int DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT = 16;
+
+ // HA related configuration
+ public static final String DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX = "dfs.client.failover.proxy.provider";
+ public static final String DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY = "dfs.client.failover.max.attempts";
+ public static final int DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
+ public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY = "dfs.client.failover.sleep.base.millis";
+ public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
+ public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.client.failover.sleep.max.millis";
+ public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
+ public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY = "dfs.client.failover.connection.retries";
+ public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT = 0;
+ public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = "dfs.client.failover.connection.retries.on.timeouts";
+ public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
public static final String DFS_NAMENODE_BACKUP_ADDRESS_KEY = "dfs.namenode.backup.address";
public static final String DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT = "localhost:50100";
@@ -120,6 +133,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_WEBHDFS_ENABLED_DEFAULT = false;
public static final String DFS_PERMISSIONS_ENABLED_KEY = "dfs.permissions.enabled";
public static final boolean DFS_PERMISSIONS_ENABLED_DEFAULT = true;
+ public static final String DFS_PERSIST_BLOCKS_KEY = "dfs.persist.blocks";
+ public static final boolean DFS_PERSIST_BLOCKS_DEFAULT = false;
public static final String DFS_PERMISSIONS_SUPERUSERGROUP_KEY = "dfs.permissions.superusergroup";
public static final String DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT = "supergroup";
public static final String DFS_ADMIN = "dfs.cluster.administrators";
@@ -131,6 +146,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT = true;
public static final String DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY = "dfs.namenode.num.checkpoints.retained";
public static final int DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
+ public static final String DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
+ public static final int DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
+
public static final String DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
public static final int DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
@@ -150,6 +168,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final long DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT = 24*60*60*1000; // 1 day
public static final String DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY = "dfs.namenode.delegation.token.max-lifetime";
public static final long DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT = 7*24*60*60*1000; // 7 days
+ public static final String DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY = "dfs.namenode.delegation.token.always-use"; // for tests
+ public static final boolean DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT = false;
//Filesystem limit keys
public static final String DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY = "dfs.namenode.fs-limits.max-component-length";
@@ -165,6 +185,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTPS_PORT_DEFAULT;
public static final String DFS_NAMENODE_NAME_DIR_KEY = "dfs.namenode.name.dir";
public static final String DFS_NAMENODE_EDITS_DIR_KEY = "dfs.namenode.edits.dir";
+ public static final String DFS_NAMENODE_SHARED_EDITS_DIR_KEY = "dfs.namenode.shared.edits.dir";
public static final String DFS_NAMENODE_EDITS_PLUGIN_PREFIX = "dfs.namenode.edits.journal-plugin";
public static final String DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY = "dfs.namenode.edits.dir.required";
public static final String DFS_CLIENT_READ_PREFETCH_SIZE_KEY = "dfs.client.read.prefetch.size";
@@ -298,8 +319,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY = "dfs.namenode.name.cache.threshold";
public static final int DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT = 10;
- public static final String DFS_FEDERATION_NAMESERVICES = "dfs.federation.nameservices";
- public static final String DFS_FEDERATION_NAMESERVICE_ID = "dfs.federation.nameservice.id";
+ public static final String DFS_FEDERATION_NAMESERVICES = "dfs.federation.nameservices";
+ public static final String DFS_FEDERATION_NAMESERVICE_ID = "dfs.federation.nameservice.id";
public static final String DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY = "dfs.namenode.resource.check.interval";
public static final int DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT = 5000;
public static final String DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved";
@@ -309,5 +330,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_DEFAULT = 1;
public static final String DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY = "dfs.web.authentication.kerberos.principal";
public static final String DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY = "dfs.web.authentication.kerberos.keytab";
+
public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
+
+ // HA related configuration
+ public static final String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
+ public static final String DFS_HA_NAMENODE_ID_KEY = "dfs.ha.namenode.id";
+ public static final String DFS_HA_STANDBY_CHECKPOINTS_KEY = "dfs.ha.standby.checkpoints";
+ public static final boolean DFS_HA_STANDBY_CHECKPOINTS_DEFAULT = true;
+ public static final String DFS_HA_LOGROLL_PERIOD_KEY = "dfs.ha.log-roll.period";
+ public static final int DFS_HA_LOGROLL_PERIOD_DEFAULT = 2 * 60; // 2m
+ public static final String DFS_HA_TAILEDITS_PERIOD_KEY = "dfs.ha.tail-edits.period";
+ public static final int DFS_HA_TAILEDITS_PERIOD_DEFAULT = 60; // 1m
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7064616780..cbc0f0ea23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -18,24 +18,21 @@
package org.apache.hadoop.hdfs;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
-
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.security.SecureRandom;
-import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.Comparator;
+import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Random;
+import java.util.Set;
import java.util.StringTokenizer;
import javax.net.SocketFactory;
@@ -46,9 +43,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
@@ -59,11 +56,19 @@ import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import com.google.protobuf.BlockingService;
@InterfaceAudience.Private
public class DFSUtil {
+ public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
+
+ private DFSUtil() { /* Hidden constructor */ }
private static final ThreadLocal RANDOM = new ThreadLocal() {
@Override
protected Random initialValue() {
@@ -101,13 +106,20 @@ public class DFSUtil {
a.isDecommissioned() ? 1 : -1;
}
};
+ /**
+ * Address matcher for matching an address to local address
+ */
+ static final AddressMatcher LOCAL_ADDRESS_MATCHER = new AddressMatcher() {
+ public boolean match(InetSocketAddress s) {
+ return NetUtils.isLocalAddress(s.getAddress());
+ };
+ };
/**
* Whether the pathname is valid. Currently prohibits relative paths,
* and names which contain a ":" or "/"
*/
public static boolean isValidName(String src) {
-
// Path must be absolute.
if (!src.startsWith(Path.SEPARATOR)) {
return false;
@@ -304,12 +316,38 @@ public class DFSUtil {
/**
* Returns collection of nameservice Ids from the configuration.
* @param conf configuration
- * @return collection of nameservice Ids
+ * @return collection of nameservice Ids, or null if not specified
*/
public static Collection getNameServiceIds(Configuration conf) {
- return conf.getStringCollection(DFS_FEDERATION_NAMESERVICES);
+ return conf.getTrimmedStringCollection(DFS_FEDERATION_NAMESERVICES);
}
+ /**
+ * @return coll if it is non-null and non-empty. Otherwise,
+ * returns a list with a single null value.
+ */
+ private static Collection emptyAsSingletonNull(Collection coll) {
+ if (coll == null || coll.isEmpty()) {
+ return Collections.singletonList(null);
+ } else {
+ return coll;
+ }
+ }
+
+ /**
+ * Namenode HighAvailability related configuration.
+ * Returns collection of namenode Ids from the configuration. One logical id
+ * for each namenode in the in the HA setup.
+ *
+ * @param conf configuration
+ * @param nsId the nameservice ID to look at, or null for non-federated
+ * @return collection of namenode Ids
+ */
+ public static Collection getNameNodeIds(Configuration conf, String nsId) {
+ String key = addSuffix(DFS_HA_NAMENODES_KEY_PREFIX, nsId);
+ return conf.getTrimmedStringCollection(key);
+ }
+
/**
* Given a list of keys in the order of preference, returns a value
* for the key in the given order from the configuration.
@@ -323,9 +361,7 @@ public class DFSUtil {
Configuration conf, String... keys) {
String value = null;
for (String key : keys) {
- if (keySuffix != null) {
- key += "." + keySuffix;
- }
+ key = addSuffix(key, keySuffix);
value = conf.get(key);
if (value != null) {
break;
@@ -337,36 +373,84 @@ public class DFSUtil {
return value;
}
+ /** Add non empty and non null suffix to a key */
+ private static String addSuffix(String key, String suffix) {
+ if (suffix == null || suffix.isEmpty()) {
+ return key;
+ }
+ assert !suffix.startsWith(".") :
+ "suffix '" + suffix + "' should not already have '.' prepended.";
+ return key + "." + suffix;
+ }
+
+ /** Concatenate list of suffix strings '.' separated */
+ private static String concatSuffixes(String... suffixes) {
+ if (suffixes == null) {
+ return null;
+ }
+ return Joiner.on(".").skipNulls().join(suffixes);
+ }
+
/**
- * Returns list of InetSocketAddress for a given set of keys.
- * @param conf configuration
- * @param defaultAddress default address to return in case key is not found
- * @param keys Set of keys to look for in the order of preference
- * @return list of InetSocketAddress corresponding to the key
+ * Return configuration key of format key.suffix1.suffix2...suffixN
*/
- private static List getAddresses(Configuration conf,
+ public static String addKeySuffixes(String key, String... suffixes) {
+ String keySuffix = concatSuffixes(suffixes);
+ return addSuffix(key, keySuffix);
+ }
+
+ /**
+ * Returns the configured address for all NameNodes in the cluster.
+ * @param conf configuration
+ * @param defaultAddress default address to return in case key is not found.
+ * @param keys Set of keys to look for in the order of preference
+ * @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
+ */
+ private static Map>
+ getAddresses(Configuration conf,
String defaultAddress, String... keys) {
Collection nameserviceIds = getNameServiceIds(conf);
- List isas = new ArrayList();
-
- // Configuration with a single namenode
- if (nameserviceIds == null || nameserviceIds.isEmpty()) {
- String address = getConfValue(defaultAddress, null, conf, keys);
- if (address == null) {
- return null;
- }
- isas.add(NetUtils.createSocketAddr(address));
- } else {
- // Get the namenodes for all the configured nameServiceIds
- for (String nameserviceId : nameserviceIds) {
- String address = getConfValue(null, nameserviceId, conf, keys);
- if (address == null) {
- return null;
- }
- isas.add(NetUtils.createSocketAddr(address));
+
+ // Look for configurations of the form [.][.]
+ // across all of the configured nameservices and namenodes.
+ Map> ret = Maps.newHashMap();
+ for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
+ Map isas =
+ getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
+ if (!isas.isEmpty()) {
+ ret.put(nsId, isas);
}
}
- return isas;
+ return ret;
+ }
+
+ private static Map getAddressesForNameserviceId(
+ Configuration conf, String nsId, String defaultValue,
+ String[] keys) {
+ Collection nnIds = getNameNodeIds(conf, nsId);
+ Map ret = Maps.newHashMap();
+ for (String nnId : emptyAsSingletonNull(nnIds)) {
+ String suffix = concatSuffixes(nsId, nnId);
+ String address = getConfValue(defaultValue, suffix, conf, keys);
+ if (address != null) {
+ InetSocketAddress isa = NetUtils.createSocketAddr(address);
+ ret.put(nnId, isa);
+ }
+ }
+ return ret;
+ }
+
+ /**
+ * Returns list of InetSocketAddress corresponding to HA NN RPC addresses from
+ * the configuration.
+ *
+ * @param conf configuration
+ * @return list of InetSocketAddresses
+ * @throws IOException if no addresses are configured
+ */
+ public static Map> getHaNnRpcAddresses(
+ Configuration conf) {
+ return getAddresses(conf, null, DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
}
/**
@@ -377,11 +461,11 @@ public class DFSUtil {
* @return list of InetSocketAddresses
* @throws IOException on error
*/
- public static List getBackupNodeAddresses(
+ public static Map> getBackupNodeAddresses(
Configuration conf) throws IOException {
- List addressList = getAddresses(conf,
+ Map> addressList = getAddresses(conf,
null, DFS_NAMENODE_BACKUP_ADDRESS_KEY);
- if (addressList == null) {
+ if (addressList.isEmpty()) {
throw new IOException("Incorrect configuration: backup node address "
+ DFS_NAMENODE_BACKUP_ADDRESS_KEY + " is not configured.");
}
@@ -396,11 +480,11 @@ public class DFSUtil {
* @return list of InetSocketAddresses
* @throws IOException on error
*/
- public static List getSecondaryNameNodeAddresses(
+ public static Map> getSecondaryNameNodeAddresses(
Configuration conf) throws IOException {
- List addressList = getAddresses(conf, null,
+ Map> addressList = getAddresses(conf, null,
DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY);
- if (addressList == null) {
+ if (addressList.isEmpty()) {
throw new IOException("Incorrect configuration: secondary namenode address "
+ DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY + " is not configured.");
}
@@ -420,7 +504,7 @@ public class DFSUtil {
* @return list of InetSocketAddress
* @throws IOException on error
*/
- public static List getNNServiceRpcAddresses(
+ public static Map> getNNServiceRpcAddresses(
Configuration conf) throws IOException {
// Use default address as fall back
String defaultAddress;
@@ -430,9 +514,10 @@ public class DFSUtil {
defaultAddress = null;
}
- List addressList = getAddresses(conf, defaultAddress,
+ Map> addressList =
+ getAddresses(conf, defaultAddress,
DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
- if (addressList == null) {
+ if (addressList.isEmpty()) {
throw new IOException("Incorrect configuration: namenode address "
+ DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
+ DFS_NAMENODE_RPC_ADDRESS_KEY
@@ -442,10 +527,154 @@ public class DFSUtil {
}
/**
- * Given the InetSocketAddress for any configured communication with a
- * namenode, this method returns the corresponding nameservice ID,
- * by doing a reverse lookup on the list of nameservices until it
- * finds a match.
+ * Flatten the given map, as returned by other functions in this class,
+ * into a flat list of {@link ConfiguredNNAddress} instances.
+ */
+ public static List flattenAddressMap(
+ Map> map) {
+ List ret = Lists.newArrayList();
+
+ for (Map.Entry> entry :
+ map.entrySet()) {
+ String nsId = entry.getKey();
+ Map nnMap = entry.getValue();
+ for (Map.Entry e2 : nnMap.entrySet()) {
+ String nnId = e2.getKey();
+ InetSocketAddress addr = e2.getValue();
+
+ ret.add(new ConfiguredNNAddress(nsId, nnId, addr));
+ }
+ }
+ return ret;
+ }
+
+ /**
+ * Format the given map, as returned by other functions in this class,
+ * into a string suitable for debugging display. The format of this string
+ * should not be considered an interface, and is liable to change.
+ */
+ public static String addressMapToString(
+ Map> map) {
+ StringBuilder b = new StringBuilder();
+ for (Map.Entry> entry :
+ map.entrySet()) {
+ String nsId = entry.getKey();
+ Map nnMap = entry.getValue();
+ b.append("Nameservice <").append(nsId).append(">:").append("\n");
+ for (Map.Entry e2 : nnMap.entrySet()) {
+ b.append(" NN ID ").append(e2.getKey())
+ .append(" => ").append(e2.getValue()).append("\n");
+ }
+ }
+ return b.toString();
+ }
+
+ public static String nnAddressesAsString(Configuration conf) {
+ Map> addresses =
+ getHaNnRpcAddresses(conf);
+ return addressMapToString(addresses);
+ }
+
+ /**
+ * Represent one of the NameNodes configured in the cluster.
+ */
+ public static class ConfiguredNNAddress {
+ private final String nameserviceId;
+ private final String namenodeId;
+ private final InetSocketAddress addr;
+
+ private ConfiguredNNAddress(String nameserviceId, String namenodeId,
+ InetSocketAddress addr) {
+ this.nameserviceId = nameserviceId;
+ this.namenodeId = namenodeId;
+ this.addr = addr;
+ }
+
+ public String getNameserviceId() {
+ return nameserviceId;
+ }
+
+ public String getNamenodeId() {
+ return namenodeId;
+ }
+
+ public InetSocketAddress getAddress() {
+ return addr;
+ }
+
+ @Override
+ public String toString() {
+ return "ConfiguredNNAddress[nsId=" + nameserviceId + ";" +
+ "nnId=" + namenodeId + ";addr=" + addr + "]";
+ }
+ }
+
+ /**
+ * Get a URI for each configured nameservice. If a nameservice is
+ * HA-enabled, then the logical URI of the nameservice is returned. If the
+ * nameservice is not HA-enabled, then a URI corresponding to an RPC address
+ * of the single NN for that nameservice is returned, preferring the service
+ * RPC address over the client RPC address.
+ *
+ * @param conf configuration
+ * @return a collection of all configured NN URIs, preferring service
+ * addresses
+ */
+ public static Collection getNsServiceRpcUris(Configuration conf) {
+ return getNameServiceUris(conf,
+ DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+ DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
+ }
+
+ /**
+ * Get a URI for each configured nameservice. If a nameservice is
+ * HA-enabled, then the logical URI of the nameservice is returned. If the
+ * nameservice is not HA-enabled, then a URI corresponding to the address of
+ * the single NN for that nameservice is returned.
+ *
+ * @param conf configuration
+ * @param keys configuration keys to try in order to get the URI for non-HA
+ * nameservices
+ * @return a collection of all configured NN URIs
+ */
+ public static Collection getNameServiceUris(Configuration conf,
+ String... keys) {
+ Set ret = new HashSet();
+ for (String nsId : getNameServiceIds(conf)) {
+ if (HAUtil.isHAEnabled(conf, nsId)) {
+ // Add the logical URI of the nameservice.
+ try {
+ ret.add(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + nsId));
+ } catch (URISyntaxException ue) {
+ throw new IllegalArgumentException(ue);
+ }
+ } else {
+ // Add the URI corresponding to the address of the NN.
+ for (String key : keys) {
+ String addr = conf.get(concatSuffixes(key, nsId));
+ if (addr != null) {
+ ret.add(createUri(HdfsConstants.HDFS_URI_SCHEME,
+ NetUtils.createSocketAddr(addr)));
+ break;
+ }
+ }
+ }
+ }
+ // Add the generic configuration keys.
+ for (String key : keys) {
+ String addr = conf.get(key);
+ if (addr != null) {
+ ret.add(createUri("hdfs", NetUtils.createSocketAddr(addr)));
+ break;
+ }
+ }
+ return ret;
+ }
+
+ /**
+ * Given the InetSocketAddress this method returns the nameservice Id
+ * corresponding to the key with matching address, by doing a reverse
+ * lookup on the list of nameservices until it finds a match.
*
* Since the process of resolving URIs to Addresses is slightly expensive,
* this utility method should not be used in performance-critical routines.
@@ -463,91 +692,109 @@ public class DFSUtil {
* not the NameServiceId-suffixed keys.
* @return nameserviceId, or null if no match found
*/
- public static String getNameServiceIdFromAddress(Configuration conf,
- InetSocketAddress address, String... keys) {
- Collection nameserviceIds = getNameServiceIds(conf);
-
+ public static String getNameServiceIdFromAddress(final Configuration conf,
+ final InetSocketAddress address, String... keys) {
// Configuration with a single namenode and no nameserviceId
- if (nameserviceIds == null || nameserviceIds.isEmpty()) {
- return null;
- }
- // Get the candidateAddresses for all the configured nameServiceIds
- for (String nameserviceId : nameserviceIds) {
- for (String key : keys) {
- String candidateAddress = conf.get(
- getNameServiceIdKey(key, nameserviceId));
- if (candidateAddress != null
- && address.equals(NetUtils.createSocketAddr(candidateAddress)))
- return nameserviceId;
- }
- }
- // didn't find a match
- return null;
+ String[] ids = getSuffixIDs(conf, address, keys);
+ return (ids != null) ? ids[0] : null;
}
-
+
/**
- * return server http or https address from the configuration
+ * return server http or https address from the configuration for a
+ * given namenode rpc address.
* @param conf
- * @param namenode - namenode address
+ * @param namenodeAddr - namenode RPC address
* @param httpsAddress -If true, and if security is enabled, returns server
* https address. If false, returns server http address.
* @return server http or https address
*/
public static String getInfoServer(
- InetSocketAddress namenode, Configuration conf, boolean httpsAddress) {
- String httpAddress = null;
-
- String httpAddressKey = (UserGroupInformation.isSecurityEnabled()
- && httpsAddress) ? DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY
- : DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
- String httpAddressDefault = (UserGroupInformation.isSecurityEnabled()
- && httpsAddress) ? DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT
- : DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_DEFAULT;
- if(namenode != null) {
+ InetSocketAddress namenodeAddr, Configuration conf, boolean httpsAddress) {
+ boolean securityOn = UserGroupInformation.isSecurityEnabled();
+ String httpAddressKey = (securityOn && httpsAddress) ?
+ DFS_NAMENODE_HTTPS_ADDRESS_KEY : DFS_NAMENODE_HTTP_ADDRESS_KEY;
+ String httpAddressDefault = (securityOn && httpsAddress) ?
+ DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT : DFS_NAMENODE_HTTP_ADDRESS_DEFAULT;
+
+ String suffixes[];
+ if (namenodeAddr != null) {
// if non-default namenode, try reverse look up
// the nameServiceID if it is available
- String nameServiceId = DFSUtil.getNameServiceIdFromAddress(
- conf, namenode,
+ suffixes = getSuffixIDs(conf, namenodeAddr,
DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
-
- if (nameServiceId != null) {
- httpAddress = conf.get(DFSUtil.getNameServiceIdKey(
- httpAddressKey, nameServiceId));
- }
- }
- // else - Use non-federation style configuration
- if (httpAddress == null) {
- httpAddress = conf.get(httpAddressKey, httpAddressDefault);
+ } else {
+ suffixes = new String[2];
}
- return httpAddress;
+ return getSuffixedConf(conf, httpAddressKey, httpAddressDefault, suffixes);
}
+
/**
- * @return key specific to a nameserviceId from a generic key
+ * Substitute a default host in the case that an address has been configured
+ * with a wildcard. This is used, for example, when determining the HTTP
+ * address of the NN -- if it's configured to bind to 0.0.0.0, we want to
+ * substitute the hostname from the filesystem URI rather than trying to
+ * connect to 0.0.0.0.
+ * @param configuredAddress the address found in the configuration
+ * @param defaultHost the host to substitute with, if configuredAddress
+ * is a local/wildcard address.
+ * @return the substituted address
+ * @throws IOException if it is a wildcard address and security is enabled
*/
- public static String getNameServiceIdKey(String key, String nameserviceId) {
- return key + "." + nameserviceId;
+ public static String substituteForWildcardAddress(String configuredAddress,
+ String defaultHost) throws IOException {
+ InetSocketAddress sockAddr = NetUtils.createSocketAddr(configuredAddress);
+ if (sockAddr.getAddress().isAnyLocalAddress()) {
+ if(UserGroupInformation.isSecurityEnabled()) {
+ throw new IOException("Cannot use a wildcard address with security. " +
+ "Must explicitly set bind address for Kerberos");
+ }
+ return defaultHost + ":" + sockAddr.getPort();
+ } else {
+ return configuredAddress;
+ }
+ }
+
+ private static String getSuffixedConf(Configuration conf,
+ String key, String defaultVal, String[] suffixes) {
+ String ret = conf.get(DFSUtil.addKeySuffixes(key, suffixes));
+ if (ret != null) {
+ return ret;
+ }
+ return conf.get(key, defaultVal);
}
/**
* Sets the node specific setting into generic configuration key. Looks up
- * value of "key.nameserviceId" and if found sets that value into generic key
- * in the conf. Note that this only modifies the runtime conf.
+ * value of "key.nameserviceId.namenodeId" and if found sets that value into
+ * generic key in the conf. If this is not found, falls back to
+ * "key.nameserviceId" and then the unmodified key.
+ *
+ * Note that this only modifies the runtime conf.
*
* @param conf
* Configuration object to lookup specific key and to set the value
* to the key passed. Note the conf object is modified.
* @param nameserviceId
- * nameservice Id to construct the node specific key.
+ * nameservice Id to construct the node specific key. Pass null if
+ * federation is not configuration.
+ * @param nnId
+ * namenode Id to construct the node specific key. Pass null if
+ * HA is not configured.
* @param keys
* The key for which node specific value is looked up
*/
public static void setGenericConf(Configuration conf,
- String nameserviceId, String... keys) {
+ String nameserviceId, String nnId, String... keys) {
for (String key : keys) {
- String value = conf.get(getNameServiceIdKey(key, nameserviceId));
+ String value = conf.get(addKeySuffixes(key, nameserviceId, nnId));
+ if (value != null) {
+ conf.set(key, value);
+ continue;
+ }
+ value = conf.get(addKeySuffixes(key, nameserviceId));
if (value != null) {
conf.set(key, value);
}
@@ -572,34 +819,7 @@ public class DFSUtil {
public static int roundBytesToGB(long bytes) {
return Math.round((float)bytes/ 1024 / 1024 / 1024);
}
-
-
- /** Create a {@link NameNode} proxy */
- public static ClientProtocol createNamenode(Configuration conf)
- throws IOException {
- return createNamenode(NameNode.getAddress(conf), conf);
- }
-
- /** Create a {@link NameNode} proxy */
- public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
- Configuration conf) throws IOException {
- return createNamenode(nameNodeAddr, conf,
- UserGroupInformation.getCurrentUser());
- }
-
- /** Create a {@link NameNode} proxy */
- public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
- Configuration conf, UserGroupInformation ugi) throws IOException {
- /**
- * Currently we have simply burnt-in support for a SINGLE
- * protocol - protocolPB. This will be replaced
- * by a way to pick the right protocol based on the
- * version of the target server.
- */
- return new org.apache.hadoop.hdfs.protocolPB.
- ClientNamenodeProtocolTranslatorPB(nameNodeAddr, conf, ugi);
- }
-
+
/** Create a {@link ClientDatanodeProtocol} proxy */
public static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
DatanodeID datanodeid, Configuration conf, int socketTimeout,
@@ -622,9 +842,9 @@ public class DFSUtil {
SocketFactory factory) throws IOException {
return new ClientDatanodeProtocolTranslatorPB(addr, ticket, conf, factory);
}
-
+
/**
- * Get name service Id for the {@link NameNode} based on namenode RPC address
+ * Get nameservice Id for the {@link NameNode} based on namenode RPC address
* matching the local node address.
*/
public static String getNamenodeNameServiceId(Configuration conf) {
@@ -632,7 +852,7 @@ public class DFSUtil {
}
/**
- * Get name service Id for the BackupNode based on backup node RPC address
+ * Get nameservice Id for the BackupNode based on backup node RPC address
* matching the local node address.
*/
public static String getBackupNameServiceId(Configuration conf) {
@@ -640,7 +860,7 @@ public class DFSUtil {
}
/**
- * Get name service Id for the secondary node based on secondary http address
+ * Get nameservice Id for the secondary node based on secondary http address
* matching the local node address.
*/
public static String getSecondaryNameServiceId(Configuration conf) {
@@ -652,13 +872,14 @@ public class DFSUtil {
* the address of the local node.
*
* If {@link DFSConfigKeys#DFS_FEDERATION_NAMESERVICE_ID} is not specifically
- * configured, this method determines the nameservice Id by matching the local
- * nodes address with the configured addresses. When a match is found, it
- * returns the nameservice Id from the corresponding configuration key.
+ * configured, and more than one nameservice Id is configured, this method
+ * determines the nameservice Id by matching the local node's address with the
+ * configured addresses. When a match is found, it returns the nameservice Id
+ * from the corresponding configuration key.
*
* @param conf Configuration
* @param addressKey configuration key to get the address.
- * @return name service Id on success, null on failure.
+ * @return nameservice Id on success, null if federation is not configured.
* @throws HadoopIllegalArgumentException on error
*/
private static String getNameServiceId(Configuration conf, String addressKey) {
@@ -666,34 +887,106 @@ public class DFSUtil {
if (nameserviceId != null) {
return nameserviceId;
}
-
- Collection ids = getNameServiceIds(conf);
- if (ids == null || ids.size() == 0) {
- // Not federation configuration, hence no nameservice Id
- return null;
+ Collection nsIds = getNameServiceIds(conf);
+ if (1 == nsIds.size()) {
+ return nsIds.toArray(new String[1])[0];
}
+ String nnId = conf.get(DFS_HA_NAMENODE_ID_KEY);
- // Match the rpc address with that of local address
+ return getSuffixIDs(conf, addressKey, null, nnId, LOCAL_ADDRESS_MATCHER)[0];
+ }
+
+ /**
+ * Returns nameservice Id and namenode Id when the local host matches the
+ * configuration parameter {@code addressKey}..
+ *
+ * @param conf Configuration
+ * @param addressKey configuration key corresponding to the address.
+ * @param knownNsId only look at configs for the given nameservice, if not-null
+ * @param knownNNId only look at configs for the given namenode, if not null
+ * @param matcher matching criteria for matching the address
+ * @return Array with nameservice Id and namenode Id on success. First element
+ * in the array is nameservice Id and second element is namenode Id.
+ * Null value indicates that the configuration does not have the the
+ * Id.
+ * @throws HadoopIllegalArgumentException on error
+ */
+ static String[] getSuffixIDs(final Configuration conf, final String addressKey,
+ String knownNsId, String knownNNId,
+ final AddressMatcher matcher) {
+ String nameserviceId = null;
+ String namenodeId = null;
int found = 0;
- for (String id : ids) {
- String addr = conf.get(getNameServiceIdKey(addressKey, id));
- InetSocketAddress s = NetUtils.createSocketAddr(addr);
- if (NetUtils.isLocalAddress(s.getAddress())) {
- nameserviceId = id;
- found++;
+
+ Collection nsIds = getNameServiceIds(conf);
+ for (String nsId : emptyAsSingletonNull(nsIds)) {
+ if (knownNsId != null && !knownNsId.equals(nsId)) {
+ continue;
+ }
+
+ Collection nnIds = getNameNodeIds(conf, nsId);
+ for (String nnId : emptyAsSingletonNull(nnIds)) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(String.format("addressKey: %s nsId: %s nnId: %s",
+ addressKey, nsId, nnId));
+ }
+ if (knownNNId != null && !knownNNId.equals(nnId)) {
+ continue;
+ }
+ String key = addKeySuffixes(addressKey, nsId, nnId);
+ String addr = conf.get(key);
+ if (addr == null) {
+ continue;
+ }
+ InetSocketAddress s = null;
+ try {
+ s = NetUtils.createSocketAddr(addr);
+ } catch (Exception e) {
+ LOG.warn("Exception in creating socket address " + addr, e);
+ continue;
+ }
+ if (!s.isUnresolved() && matcher.match(s)) {
+ nameserviceId = nsId;
+ namenodeId = nnId;
+ found++;
+ }
}
}
if (found > 1) { // Only one address must match the local address
- throw new HadoopIllegalArgumentException(
- "Configuration has multiple RPC addresses that matches "
- + "the local node's address. Please configure the system with "
- + "the parameter " + DFS_FEDERATION_NAMESERVICE_ID);
+ String msg = "Configuration has multiple addresses that match "
+ + "local node's address. Please configure the system with "
+ + DFS_FEDERATION_NAMESERVICE_ID + " and "
+ + DFS_HA_NAMENODE_ID_KEY;
+ throw new HadoopIllegalArgumentException(msg);
}
- if (found == 0) {
- throw new HadoopIllegalArgumentException("Configuration address "
- + addressKey + " is missing in configuration with name service Id");
+ return new String[] { nameserviceId, namenodeId };
+ }
+
+ /**
+ * For given set of {@code keys} adds nameservice Id and or namenode Id
+ * and returns {nameserviceId, namenodeId} when address match is found.
+ * @see #getSuffixIDs(Configuration, String, AddressMatcher)
+ */
+ static String[] getSuffixIDs(final Configuration conf,
+ final InetSocketAddress address, final String... keys) {
+ AddressMatcher matcher = new AddressMatcher() {
+ @Override
+ public boolean match(InetSocketAddress s) {
+ return address.equals(s);
+ }
+ };
+
+ for (String key : keys) {
+ String[] ids = getSuffixIDs(conf, key, null, null, matcher);
+ if (ids != null && (ids [0] != null || ids[1] != null)) {
+ return ids;
+ }
}
- return nameserviceId;
+ return null;
+ }
+
+ private interface AddressMatcher {
+ public boolean match(InetSocketAddress s);
}
/** Create a URI from the scheme and address */
@@ -719,4 +1012,39 @@ public class DFSUtil {
RPC.setProtocolEngine(conf, protocol, ProtobufRpcEngine.class);
server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
}
+
+ /**
+ * Map a logical namenode ID to its service address. Use the given
+ * nameservice if specified, or the configured one if none is given.
+ *
+ * @param conf Configuration
+ * @param nsId which nameservice nnId is a part of, optional
+ * @param nnId the namenode ID to get the service addr for
+ * @return the service addr, null if it could not be determined
+ */
+ public static String getNamenodeServiceAddr(final Configuration conf,
+ String nsId, String nnId) {
+
+ if (nsId == null) {
+ Collection nsIds = getNameServiceIds(conf);
+ if (1 == nsIds.size()) {
+ nsId = nsIds.toArray(new String[1])[0];
+ } else {
+ // No nameservice ID was given and more than one is configured
+ return null;
+ }
+ }
+
+ String serviceAddrKey = concatSuffixes(
+ DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsId, nnId);
+
+ String addrKey = concatSuffixes(
+ DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnId);
+
+ String serviceRpcAddr = conf.get(serviceAddrKey);
+ if (serviceRpcAddr == null) {
+ serviceRpcAddr = conf.get(addrKey);
+ }
+ return serviceRpcAddr;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 119bca9b55..8dfced350a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -106,8 +106,7 @@ public class DistributedFileSystem extends FileSystem {
throw new IOException("Incomplete HDFS URI, no host: "+ uri);
}
- InetSocketAddress namenode = NameNode.getAddress(uri.getAuthority());
- this.dfs = new DFSClient(namenode, conf, statistics);
+ this.dfs = new DFSClient(uri, conf, statistics);
this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
this.workingDir = getHomeDirectory();
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
new file mode 100644
index 0000000000..34e9d2e9dd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
@@ -0,0 +1,261 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class HAUtil {
+
+ private static final Log LOG =
+ LogFactory.getLog(HAUtil.class);
+
+ private HAUtil() { /* Hidden constructor */ }
+
+ /**
+ * Returns true if HA for namenode is configured for the given nameservice
+ *
+ * @param conf Configuration
+ * @param nsId nameservice, or null if no federated NS is configured
+ * @return true if HA is configured in the configuration; else false.
+ */
+ public static boolean isHAEnabled(Configuration conf, String nsId) {
+ Map> addresses =
+ DFSUtil.getHaNnRpcAddresses(conf);
+ if (addresses == null) return false;
+ Map nnMap = addresses.get(nsId);
+ return nnMap != null && nnMap.size() > 1;
+ }
+
+ /**
+ * Returns true if HA is using a shared edits directory.
+ *
+ * @param conf Configuration
+ * @return true if HA config is using a shared edits dir, false otherwise.
+ */
+ public static boolean usesSharedEditsDir(Configuration conf) {
+ return null != conf.get(DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
+ }
+
+ /**
+ * Get the namenode Id by matching the {@code addressKey}
+ * with the the address of the local node.
+ *
+ * If {@link DFSConfigKeys#DFS_HA_NAMENODE_ID_KEY} is not specifically
+ * configured, this method determines the namenode Id by matching the local
+ * node's address with the configured addresses. When a match is found, it
+ * returns the namenode Id from the corresponding configuration key.
+ *
+ * @param conf Configuration
+ * @return namenode Id on success, null on failure.
+ * @throws HadoopIllegalArgumentException on error
+ */
+ public static String getNameNodeId(Configuration conf, String nsId) {
+ String namenodeId = conf.getTrimmed(DFS_HA_NAMENODE_ID_KEY);
+ if (namenodeId != null) {
+ return namenodeId;
+ }
+
+ String suffixes[] = DFSUtil.getSuffixIDs(conf, DFS_NAMENODE_RPC_ADDRESS_KEY,
+ nsId, null, DFSUtil.LOCAL_ADDRESS_MATCHER);
+ if (suffixes == null) {
+ String msg = "Configuration " + DFS_NAMENODE_RPC_ADDRESS_KEY +
+ " must be suffixed with nameservice and namenode ID for HA " +
+ "configuration.";
+ throw new HadoopIllegalArgumentException(msg);
+ }
+
+ return suffixes[1];
+ }
+
+ /**
+ * Similar to
+ * {@link DFSUtil#getNameServiceIdFromAddress(Configuration,
+ * InetSocketAddress, String...)}
+ */
+ public static String getNameNodeIdFromAddress(final Configuration conf,
+ final InetSocketAddress address, String... keys) {
+ // Configuration with a single namenode and no nameserviceId
+ String[] ids = DFSUtil.getSuffixIDs(conf, address, keys);
+ if (ids != null && ids.length > 1) {
+ return ids[1];
+ }
+ return null;
+ }
+
+ /**
+ * Given the configuration for this node, return a Configuration object for
+ * the other node in an HA setup.
+ *
+ * @param myConf the configuration of this node
+ * @return the configuration of the other node in an HA setup
+ */
+ public static Configuration getConfForOtherNode(
+ Configuration myConf) {
+
+ String nsId = DFSUtil.getNamenodeNameServiceId(myConf);
+ Preconditions.checkArgument(nsId != null,
+ "Could not determine namespace id. Please ensure that this " +
+ "machine is one of the machines listed as a NN RPC address, " +
+ "or configure " + DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID);
+
+ Collection nnIds = DFSUtil.getNameNodeIds(myConf, nsId);
+ String myNNId = myConf.get(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY);
+ Preconditions.checkArgument(nnIds != null,
+ "Could not determine namenode ids in namespace '%s'. " +
+ "Please configure " +
+ DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
+ nsId),
+ nsId);
+ Preconditions.checkArgument(nnIds.size() == 2,
+ "Expected exactly 2 NameNodes in namespace '%s'. " +
+ "Instead, got only %s (NN ids were '%s'",
+ nsId, nnIds.size(), Joiner.on("','").join(nnIds));
+ Preconditions.checkState(myNNId != null && !myNNId.isEmpty(),
+ "Could not determine own NN ID in namespace '%s'. Please " +
+ "ensure that this node is one of the machines listed as an " +
+ "NN RPC address, or configure " + DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY,
+ nsId);
+
+ ArrayList nnSet = Lists.newArrayList(nnIds);
+ nnSet.remove(myNNId);
+ assert nnSet.size() == 1;
+ String activeNN = nnSet.get(0);
+
+ // Look up the address of the active NN.
+ Configuration confForOtherNode = new Configuration(myConf);
+ NameNode.initializeGenericKeys(confForOtherNode, nsId, activeNN);
+ return confForOtherNode;
+ }
+
+ /**
+ * This is used only by tests at the moment.
+ * @return true if the NN should allow read operations while in standby mode.
+ */
+ public static boolean shouldAllowStandbyReads(Configuration conf) {
+ return conf.getBoolean("dfs.ha.allow.stale.reads", false);
+ }
+
+ public static void setAllowStandbyReads(Configuration conf, boolean val) {
+ conf.setBoolean("dfs.ha.allow.stale.reads", val);
+ }
+
+ /**
+ * @return true if the given nameNodeUri appears to be a logical URI.
+ * This is the case if there is a failover proxy provider configured
+ * for it in the given configuration.
+ */
+ public static boolean isLogicalUri(
+ Configuration conf, URI nameNodeUri) {
+ String host = nameNodeUri.getHost();
+ String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
+ + host;
+ return conf.get(configKey) != null;
+ }
+
+ /**
+ * Parse the HDFS URI out of the provided token.
+ * @throws IOException if the token is invalid
+ */
+ public static URI getServiceUriFromToken(
+ Token token)
+ throws IOException {
+ String tokStr = token.getService().toString();
+
+ if (tokStr.startsWith(HA_DT_SERVICE_PREFIX)) {
+ tokStr = tokStr.replaceFirst(HA_DT_SERVICE_PREFIX, "");
+ }
+
+ try {
+ return new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
+ tokStr);
+ } catch (URISyntaxException e) {
+ throw new IOException("Invalid token contents: '" +
+ tokStr + "'");
+ }
+ }
+
+ /**
+ * Get the service name used in the delegation token for the given logical
+ * HA service.
+ * @param uri the logical URI of the cluster
+ * @return the service name
+ */
+ public static Text buildTokenServiceForLogicalUri(URI uri) {
+ return new Text(HA_DT_SERVICE_PREFIX + uri.getHost());
+ }
+
+ /**
+ * @return true if this token corresponds to a logical nameservice
+ * rather than a specific namenode.
+ */
+ public static boolean isTokenForLogicalUri(
+ Token token) {
+ return token.getService().toString().startsWith(HA_DT_SERVICE_PREFIX);
+ }
+
+ /**
+ * Locate a delegation token associated with the given HA cluster URI, and if
+ * one is found, clone it to also represent the underlying namenode address.
+ * @param ugi the UGI to modify
+ * @param haUri the logical URI for the cluster
+ * @param singleNNAddr one of the NNs in the cluster to which the token
+ * applies
+ */
+ public static void cloneDelegationTokenForLogicalUri(
+ UserGroupInformation ugi, URI haUri,
+ InetSocketAddress singleNNAddr) {
+ Text haService = buildTokenServiceForLogicalUri(haUri);
+ Token haToken =
+ DelegationTokenSelector.selectHdfsDelegationToken(haService, ugi);
+ if (haToken == null) {
+ // no token
+ return;
+ }
+ Token specificToken =
+ new Token(haToken);
+ specificToken.setService(SecurityUtil.buildTokenService(singleNNAddr));
+ ugi.addToken(specificToken);
+ LOG.debug("Mapped HA service delegation token for logical URI " +
+ haUri + " to namenode " + singleNNAddr);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
index 27702b5795..6e212458d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -44,6 +45,8 @@ public class HDFSPolicyProvider extends PolicyProvider {
new Service("security.inter.datanode.protocol.acl",
InterDatanodeProtocol.class),
new Service("security.namenode.protocol.acl", NamenodeProtocol.class),
+ new Service(CommonConfigurationKeys.SECURITY_HA_SERVICE_PROTOCOL_ACL,
+ HAServiceProtocol.class),
new Service(
CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_POLICY,
RefreshAuthorizationPolicyProtocol.class),
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
new file mode 100644
index 0000000000..650c313c0a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -0,0 +1,333 @@
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSClient.Conf;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolPB;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+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.io.Text;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Create proxy objects to communicate with a remote NN. All remote access to an
+ * NN should be funneled through this class. Most of the time you'll want to use
+ * {@link NameNodeProxies#createProxy(Configuration, URI, Class)}, which will
+ * create either an HA- or non-HA-enabled client proxy as appropriate.
+ */
+public class NameNodeProxies {
+
+ private static final Log LOG = LogFactory.getLog(NameNodeProxies.class);
+
+ /**
+ * Wrapper for a client proxy as well as its associated service ID.
+ * This is simply used as a tuple-like return type for
+ * {@link NameNodeProxies#createProxy} and
+ * {@link NameNodeProxies#createNonHAProxy}.
+ */
+ public static class ProxyAndInfo {
+ private final PROXYTYPE proxy;
+ private final Text dtService;
+
+ public ProxyAndInfo(PROXYTYPE proxy, Text dtService) {
+ this.proxy = proxy;
+ this.dtService = dtService;
+ }
+
+ public PROXYTYPE getProxy() {
+ return proxy;
+ }
+
+ public Text getDelegationTokenService() {
+ return dtService;
+ }
+ }
+
+ /**
+ * Creates the namenode proxy with the passed protocol. This will handle
+ * creation of either HA- or non-HA-enabled proxy objects, depending upon
+ * if the provided URI is a configured logical URI.
+ *
+ * @param conf the configuration containing the required IPC
+ * properties, client failover configurations, etc.
+ * @param nameNodeUri the URI pointing either to a specific NameNode
+ * or to a logical nameservice.
+ * @param xface the IPC interface which should be created
+ * @return an object containing both the proxy and the associated
+ * delegation token service it corresponds to
+ * @throws IOException if there is an error creating the proxy
+ **/
+ @SuppressWarnings("unchecked")
+ public static ProxyAndInfo createProxy(Configuration conf,
+ URI nameNodeUri, Class xface) throws IOException {
+ Class> failoverProxyProviderClass =
+ getFailoverProxyProviderClass(conf, nameNodeUri, xface);
+
+ if (failoverProxyProviderClass == null) {
+ // Non-HA case
+ return createNonHAProxy(conf, NameNode.getAddress(nameNodeUri), xface,
+ UserGroupInformation.getCurrentUser(), true);
+ } else {
+ // HA case
+ FailoverProxyProvider failoverProxyProvider = NameNodeProxies
+ .createFailoverProxyProvider(conf, failoverProxyProviderClass, xface,
+ nameNodeUri);
+ Conf config = new Conf(conf);
+ T proxy = (T) RetryProxy.create(xface, failoverProxyProvider, RetryPolicies
+ .failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+ config.maxFailoverAttempts, config.failoverSleepBaseMillis,
+ config.failoverSleepMaxMillis));
+
+ Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri);
+ return new ProxyAndInfo(proxy, dtService);
+ }
+ }
+
+ /**
+ * Creates an explicitly non-HA-enabled proxy object. Most of the time you
+ * don't want to use this, and should instead use {@link NameNodeProxies#createProxy}.
+ *
+ * @param conf the configuration object
+ * @param nnAddr address of the remote NN to connect to
+ * @param xface the IPC interface which should be created
+ * @param ugi the user who is making the calls on the proxy object
+ * @param withRetries certain interfaces have a non-standard retry policy
+ * @return an object containing both the proxy and the associated
+ * delegation token service it corresponds to
+ * @throws IOException
+ */
+ @SuppressWarnings("unchecked")
+ public static ProxyAndInfo createNonHAProxy(
+ Configuration conf, InetSocketAddress nnAddr, Class xface,
+ UserGroupInformation ugi, boolean withRetries) throws IOException {
+ Text dtService = SecurityUtil.buildTokenService(nnAddr);
+
+ T proxy;
+ if (xface == ClientProtocol.class) {
+ proxy = (T) createNNProxyWithClientProtocol(nnAddr, conf, ugi,
+ withRetries);
+ } else if (xface == JournalProtocol.class) {
+ proxy = (T) createNNProxyWithJournalProtocol(nnAddr, conf, ugi);
+ } else if (xface == NamenodeProtocol.class) {
+ proxy = (T) createNNProxyWithNamenodeProtocol(nnAddr, conf, ugi,
+ withRetries);
+ } else if (xface == GetUserMappingsProtocol.class) {
+ proxy = (T) createNNProxyWithGetUserMappingsProtocol(nnAddr, conf, ugi);
+ } else if (xface == RefreshUserMappingsProtocol.class) {
+ proxy = (T) createNNProxyWithRefreshUserMappingsProtocol(nnAddr, conf, ugi);
+ } else if (xface == RefreshAuthorizationPolicyProtocol.class) {
+ proxy = (T) createNNProxyWithRefreshAuthorizationPolicyProtocol(nnAddr,
+ conf, ugi);
+ } else {
+ String message = "Upsupported protocol found when creating the proxy " +
+ "connection to NameNode: " +
+ ((xface != null) ? xface.getClass().getName() : "null");
+ LOG.error(message);
+ throw new IllegalStateException(message);
+ }
+ return new ProxyAndInfo(proxy, dtService);
+ }
+
+ private static JournalProtocol createNNProxyWithJournalProtocol(
+ InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
+ throws IOException {
+ JournalProtocolPB proxy = (JournalProtocolPB) createNameNodeProxy(address,
+ conf, ugi, JournalProtocolPB.class);
+ return new JournalProtocolTranslatorPB(proxy);
+ }
+
+ private static RefreshAuthorizationPolicyProtocol
+ createNNProxyWithRefreshAuthorizationPolicyProtocol(InetSocketAddress address,
+ Configuration conf, UserGroupInformation ugi) throws IOException {
+ RefreshAuthorizationPolicyProtocolPB proxy = (RefreshAuthorizationPolicyProtocolPB)
+ createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class);
+ return new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(proxy);
+ }
+
+ private static RefreshUserMappingsProtocol
+ createNNProxyWithRefreshUserMappingsProtocol(InetSocketAddress address,
+ Configuration conf, UserGroupInformation ugi) throws IOException {
+ RefreshUserMappingsProtocolPB proxy = (RefreshUserMappingsProtocolPB)
+ createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class);
+ return new RefreshUserMappingsProtocolClientSideTranslatorPB(proxy);
+ }
+
+ private static GetUserMappingsProtocol createNNProxyWithGetUserMappingsProtocol(
+ InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
+ throws IOException {
+ GetUserMappingsProtocolPB proxy = (GetUserMappingsProtocolPB)
+ createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class);
+ return new GetUserMappingsProtocolClientSideTranslatorPB(proxy);
+ }
+
+ private static NamenodeProtocol createNNProxyWithNamenodeProtocol(
+ InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
+ boolean withRetries) throws IOException {
+ NamenodeProtocolPB proxy = (NamenodeProtocolPB) createNameNodeProxy(
+ address, conf, ugi, NamenodeProtocolPB.class);
+ if (withRetries) { // create the proxy with retries
+ RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(5, 200,
+ TimeUnit.MILLISECONDS);
+ Map, RetryPolicy> exceptionToPolicyMap
+ = new HashMap, RetryPolicy>();
+ RetryPolicy methodPolicy = RetryPolicies.retryByException(timeoutPolicy,
+ exceptionToPolicyMap);
+ Map methodNameToPolicyMap
+ = new HashMap();
+ methodNameToPolicyMap.put("getBlocks", methodPolicy);
+ methodNameToPolicyMap.put("getAccessKeys", methodPolicy);
+ proxy = (NamenodeProtocolPB) RetryProxy.create(NamenodeProtocolPB.class,
+ proxy, methodNameToPolicyMap);
+ }
+ return new NamenodeProtocolTranslatorPB(proxy);
+ }
+
+ private static ClientProtocol createNNProxyWithClientProtocol(
+ InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
+ boolean withRetries) throws IOException {
+ ClientNamenodeProtocolPB proxy = (ClientNamenodeProtocolPB) NameNodeProxies
+ .createNameNodeProxy(address, conf, ugi, ClientNamenodeProtocolPB.class);
+ if (withRetries) { // create the proxy with retries
+ RetryPolicy createPolicy = RetryPolicies
+ .retryUpToMaximumCountWithFixedSleep(5,
+ HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
+
+ Map, RetryPolicy> remoteExceptionToPolicyMap
+ = new HashMap, RetryPolicy>();
+ remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
+ createPolicy);
+
+ Map, RetryPolicy> exceptionToPolicyMap
+ = new HashMap, RetryPolicy>();
+ exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
+ .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+ remoteExceptionToPolicyMap));
+ RetryPolicy methodPolicy = RetryPolicies.retryByException(
+ RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+ Map methodNameToPolicyMap
+ = new HashMap();
+
+ methodNameToPolicyMap.put("create", methodPolicy);
+
+ proxy = (ClientNamenodeProtocolPB) RetryProxy
+ .create(ClientNamenodeProtocolPB.class, proxy, methodNameToPolicyMap);
+ }
+ return new ClientNamenodeProtocolTranslatorPB(proxy);
+ }
+
+ @SuppressWarnings("unchecked")
+ private static Object createNameNodeProxy(InetSocketAddress address,
+ Configuration conf, UserGroupInformation ugi, Class xface)
+ throws IOException {
+ RPC.setProtocolEngine(conf, xface, ProtobufRpcEngine.class);
+ Object proxy = RPC.getProxy(xface, RPC.getProtocolVersion(xface), address,
+ ugi, conf, NetUtils.getDefaultSocketFactory(conf));
+ return proxy;
+ }
+
+ /** Gets the configured Failover proxy provider's class */
+ private static Class> getFailoverProxyProviderClass(
+ Configuration conf, URI nameNodeUri, Class xface) throws IOException {
+ if (nameNodeUri == null) {
+ return null;
+ }
+ String host = nameNodeUri.getHost();
+
+ String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
+ + host;
+ try {
+ @SuppressWarnings("unchecked")
+ Class> ret = (Class>) conf
+ .getClass(configKey, null, FailoverProxyProvider.class);
+ if (ret != null) {
+ // If we found a proxy provider, then this URI should be a logical NN.
+ // Given that, it shouldn't have a non-default port number.
+ int port = nameNodeUri.getPort();
+ if (port > 0 && port != NameNode.DEFAULT_PORT) {
+ throw new IOException("Port " + port + " specified in URI "
+ + nameNodeUri + " but host '" + host
+ + "' is a logical (HA) namenode"
+ + " and does not use port information.");
+ }
+ }
+ return ret;
+ } catch (RuntimeException e) {
+ if (e.getCause() instanceof ClassNotFoundException) {
+ throw new IOException("Could not load failover proxy provider class "
+ + conf.get(configKey) + " which is configured for authority "
+ + nameNodeUri, e);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ /** Creates the Failover proxy provider instance*/
+ @SuppressWarnings("unchecked")
+ private static FailoverProxyProvider createFailoverProxyProvider(
+ Configuration conf, Class> failoverProxyProviderClass,
+ Class xface, URI nameNodeUri) throws IOException {
+ Preconditions.checkArgument(
+ xface.isAssignableFrom(NamenodeProtocols.class),
+ "Interface %s is not a NameNode protocol", xface);
+ try {
+ Constructor> ctor = failoverProxyProviderClass
+ .getConstructor(Configuration.class, URI.class, Class.class);
+ FailoverProxyProvider> provider = ctor.newInstance(conf, nameNodeUri,
+ xface);
+ return (FailoverProxyProvider) provider;
+ } catch (Exception e) {
+ String message = "Couldn't create proxy provider " + failoverProxyProviderClass;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(message, e);
+ }
+ if (e.getCause() instanceof IOException) {
+ throw (IOException) e.getCause();
+ } else {
+ throw new IOException(message, e);
+ }
+ }
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
index e1006a65d4..58af5fd50c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
* - followed by the invalid replica represented with three -1s;
* - followed by the under-construction replica list where each replica is
* represented by 4 longs: three for the block id, length, generation
- * stamp, and the forth for the replica state.
+ * stamp, and the fourth for the replica state.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@@ -304,4 +304,16 @@ public class BlockListAsLongs implements Iterable {
blockList[idx+1] = -1;
blockList[idx+2] = -1;
}
+
+ public long getMaxGsInBlockList() {
+ long maxGs = -1;
+ Iterator iter = getBlockReportIterator();
+ while (iter.hasNext()) {
+ Block b = iter.next();
+ if (b.getGenerationStamp() > maxGs) {
+ maxGs = b.getGenerationStamp();
+ }
+ }
+ return maxGs;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index ab6babcce3..099fd284ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.token.Token;
@@ -114,6 +115,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public LocatedBlocks getBlockLocations(String src,
long offset,
long length)
@@ -125,6 +127,7 @@ public interface ClientProtocol {
* @return a set of server default configuration values
* @throws IOException
*/
+ @Idempotent
public FsServerDefaults getServerDefaults() throws IOException;
/**
@@ -228,6 +231,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
@@ -242,6 +246,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public void setPermission(String src, FsPermission permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
@@ -259,12 +264,13 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
/**
- * The client can give up on a blcok by calling abandonBlock().
+ * The client can give up on a block by calling abandonBlock().
* The client can then
* either obtain a new block, or complete or abandon the file.
* Any partial writes to the block will be discarded.
@@ -331,6 +337,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
final int numAdditionalNodes, final String clientName
@@ -368,6 +375,7 @@ public interface ClientProtocol {
* locations on datanodes).
* @param blocks Array of located blocks to report
*/
+ @Idempotent
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
///////////////////////////////////////
@@ -482,6 +490,7 @@ public interface ClientProtocol {
* RunTimeExceptions:
* @throws InvalidPathException If src is invalid
*/
+ @Idempotent
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
@@ -502,6 +511,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public DirectoryListing getListing(String src,
byte[] startAfter,
boolean needLocation)
@@ -531,6 +541,7 @@ public interface ClientProtocol {
* @throws AccessControlException permission denied
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public void renewLease(String clientName) throws AccessControlException,
IOException;
@@ -543,6 +554,7 @@ public interface ClientProtocol {
* @return true if the file is already closed
* @throws IOException
*/
+ @Idempotent
public boolean recoverLease(String src, String clientName) throws IOException;
public int GET_STATS_CAPACITY_IDX = 0;
@@ -554,7 +566,7 @@ public interface ClientProtocol {
/**
* Get a set of statistics about the filesystem.
- * Right now, only three values are returned.
+ * Right now, only seven values are returned.
*
*
[0] contains the total storage capacity of the system, in bytes.
*
[1] contains the total used space of the system, in bytes.
@@ -567,6 +579,7 @@ public interface ClientProtocol {
* Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
* actual numbers to index into the array.
*/
+ @Idempotent
public long[] getStats() throws IOException;
/**
@@ -575,6 +588,7 @@ public interface ClientProtocol {
* Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
* otherwise all datanodes if type is ALL.
*/
+ @Idempotent
public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
throws IOException;
@@ -585,6 +599,7 @@ public interface ClientProtocol {
* @throws IOException
* @throws UnresolvedLinkException if the path contains a symlink.
*/
+ @Idempotent
public long getPreferredBlockSize(String filename)
throws IOException, UnresolvedLinkException;
@@ -700,9 +715,9 @@ public interface ClientProtocol {
* all corrupt files, call this method repeatedly and each time pass in the
* cookie returned from the previous call.
*/
- public CorruptFileBlocks
- listCorruptFileBlocks(String path, String cookie)
- throws IOException;
+ @Idempotent
+ public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+ throws IOException;
/**
* Dumps namenode data structures into specified file. If the file
@@ -719,6 +734,7 @@ public interface ClientProtocol {
* @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
* @throws IOException
*/
+ @Idempotent
public void setBalancerBandwidth(long bandwidth) throws IOException;
/**
@@ -732,6 +748,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if the path contains a symlink.
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException;
@@ -747,6 +764,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if src contains a symlink
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public HdfsFileStatus getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException;
@@ -759,6 +777,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if path contains a symlink.
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public ContentSummary getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@@ -784,6 +803,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if the path contains a symlink.
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@@ -799,6 +819,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if src contains a symlink.
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public void fsync(String src, String client)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@@ -818,6 +839,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if src contains a symlink.
* @throws IOException If an I/O error occurred
*/
+ @Idempotent
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@@ -858,6 +880,7 @@ public interface ClientProtocol {
* @throws IOException If the given path does not refer to a symlink
* or an I/O error occurred
*/
+ @Idempotent
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException;
@@ -873,6 +896,7 @@ public interface ClientProtocol {
* @return a located block with a new generation stamp and an access token
* @throws IOException if any error occurs
*/
+ @Idempotent
public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
String clientName) throws IOException;
@@ -896,6 +920,7 @@ public interface ClientProtocol {
* @return Token
* @throws IOException
*/
+ @Idempotent
public Token getDelegationToken(Text renewer)
throws IOException;
@@ -906,6 +931,7 @@ public interface ClientProtocol {
* @return the new expiration time
* @throws IOException
*/
+ @Idempotent
public long renewDelegationToken(Token token)
throws IOException;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 6b4835facc..da64b9e764 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -99,6 +99,14 @@ public class HdfsConstants {
*/
public static final String HDFS_URI_SCHEME = "hdfs";
+ /**
+ * A prefix put before the namenode URI inside the "service" field
+ * of a delgation token, indicating that the URI is a logical (HA)
+ * URI.
+ */
+ public static final String HA_DT_SERVICE_PREFIX = "ha-hdfs:";
+
+
/**
* Please see {@link LayoutVersion} on adding new layout version.
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
index 729748f302..3680ee54aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
@@ -91,7 +91,10 @@ public class LayoutVersion {
STORED_TXIDS(-37, "Transaction IDs are stored in edits log and image files"),
TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs"),
EDITLOG_OP_OPTIMIZATION(-39,
- "Use LongWritable and ShortWritable directly instead of ArrayWritable of UTF8");
+ "Use LongWritable and ShortWritable directly instead of ArrayWritable of UTF8"),
+ OPTIMIZE_PERSIST_BLOCKS(-40,
+ "Serialize block lists with delta-encoded variable length ints, " +
+ "add OP_UPDATE_BLOCKS");
final int lv;
final int ancestorLV;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index f6a63fba8e..7382543397 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
@@ -63,7 +64,8 @@ import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientDatanodeProtocolTranslatorPB implements
- ProtocolMetaInterface, ClientDatanodeProtocol, Closeable {
+ ProtocolMetaInterface, ClientDatanodeProtocol,
+ ProtocolTranslator, Closeable {
public static final Log LOG = LogFactory
.getLog(ClientDatanodeProtocolTranslatorPB.class);
@@ -198,4 +200,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
}
+
+ @Override
+ public Object getUnderlyingProxyObject() {
+ return rpcProxy;
+ }
}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 369158439f..46a3c825cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -20,15 +20,10 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
-import java.net.InetSocketAddress;
import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -49,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
+import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -137,52 +133,14 @@ import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientNamenodeProtocolTranslatorPB implements
- ProtocolMetaInterface, ClientProtocol, Closeable {
+ ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
final private ClientNamenodeProtocolPB rpcProxy;
- private static ClientNamenodeProtocolPB createNamenode(
- InetSocketAddress nameNodeAddr, Configuration conf,
- UserGroupInformation ugi) throws IOException {
- RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
- ProtobufRpcEngine.class);
- return RPC.getProxy(ClientNamenodeProtocolPB.class,
- RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), nameNodeAddr, ugi, conf,
- NetUtils.getSocketFactory(conf, ClientNamenodeProtocolPB.class));
+ public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy)
+ throws IOException {
+ rpcProxy = proxy;
}
-
- /** Create a {@link NameNode} proxy */
- static ClientNamenodeProtocolPB createNamenodeWithRetry(
- ClientNamenodeProtocolPB rpcNamenode) {
- RetryPolicy createPolicy = RetryPolicies
- .retryUpToMaximumCountWithFixedSleep(5,
- HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
-
- Map, RetryPolicy> remoteExceptionToPolicyMap
- = new HashMap, RetryPolicy>();
- remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
- createPolicy);
-
- Map, RetryPolicy> exceptionToPolicyMap =
- new HashMap, RetryPolicy>();
- exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
- .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
- remoteExceptionToPolicyMap));
- RetryPolicy methodPolicy = RetryPolicies.retryByException(
- RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
- Map methodNameToPolicyMap = new HashMap();
-
- methodNameToPolicyMap.put("create", methodPolicy);
-
- return (ClientNamenodeProtocolPB) RetryProxy.create(
- ClientNamenodeProtocolPB.class, rpcNamenode, methodNameToPolicyMap);
- }
-
- public ClientNamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
- Configuration conf, UserGroupInformation ugi) throws IOException {
-
- rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
- }
-
+
public void close() {
RPC.stopProxy(rpcProxy);
}
@@ -866,4 +824,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
}
+
+ @Override
+ public Object getUnderlyingProxyObject() {
+ return rpcProxy;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index d47eac2295..2a661c0fc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeComm
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -161,7 +163,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
}
@Override
- public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
+ public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
StorageReport[] reports, int xmitsInProgress, int xceiverCount,
int failedVolumes) throws IOException {
HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
@@ -184,7 +186,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
cmds[index] = PBHelper.convert(p);
index++;
}
- return cmds;
+ return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus()));
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 413bd3aabf..c653daa1ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -103,7 +104,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
@Override
public HeartbeatResponseProto sendHeartbeat(RpcController controller,
HeartbeatRequestProto request) throws ServiceException {
- DatanodeCommand[] cmds = null;
+ HeartbeatResponse response;
try {
List list = request.getReportsList();
StorageReport[] report = new StorageReport[list.size()];
@@ -113,7 +114,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
p.getBlockPoolUsed());
}
- cmds = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
+ response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
report, request.getXmitsInProgress(), request.getXceiverCount(),
request.getFailedVolumes());
} catch (IOException e) {
@@ -121,6 +122,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
}
HeartbeatResponseProto.Builder builder = HeartbeatResponseProto
.newBuilder();
+ DatanodeCommand[] cmds = response.getCommands();
if (cmds != null) {
for (int i = 0; i < cmds.length; i++) {
if (cmds[i] != null) {
@@ -128,6 +130,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
}
}
}
+ builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState()));
return builder.build();
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
index c29595e159..01bd88ebbf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java
@@ -20,22 +20,15 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.tools.GetUserMappingsProtocol;
import com.google.protobuf.RpcController;
@@ -47,16 +40,10 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final GetUserMappingsProtocolPB rpcProxy;
-
+
public GetUserMappingsProtocolClientSideTranslatorPB(
- InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
- Configuration conf) throws IOException {
- RPC.setProtocolEngine(conf, GetUserMappingsProtocolPB.class,
- ProtobufRpcEngine.class);
- rpcProxy = RPC.getProxy(GetUserMappingsProtocolPB.class,
- RPC.getProtocolVersion(GetUserMappingsProtocolPB.class),
- NameNode.getAddress(conf), ugi, conf,
- NetUtils.getSocketFactory(conf, GetUserMappingsProtocol.class));
+ GetUserMappingsProtocolPB rpcProxy) {
+ this.rpcProxy = rpcProxy;
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
index 0735cfdbb5..76ca46f4e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/JournalProtocolTranslatorPB.java
@@ -19,17 +19,14 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
-import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
@@ -52,12 +49,9 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final JournalProtocolPB rpcProxy;
-
- public JournalProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
- Configuration conf) throws IOException {
- RPC.setProtocolEngine(conf, JournalProtocolPB.class, ProtobufRpcEngine.class);
- rpcProxy = RPC.getProxy(JournalProtocolPB.class,
- RPC.getProtocolVersion(JournalProtocolPB.class), nameNodeAddr, conf);
+
+ public JournalProtocolTranslatorPB(JournalProtocolPB rpcProxy) {
+ this.rpcProxy = rpcProxy;
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index f2ec7ba2a9..7de2c0e461 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -19,11 +19,9 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
-import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
@@ -47,14 +45,11 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@@ -84,15 +79,6 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
VersionRequestProto.newBuilder().build();
final private NamenodeProtocolPB rpcProxy;
-
- public NamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
- Configuration conf, UserGroupInformation ugi) throws IOException {
- RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
- ProtobufRpcEngine.class);
- rpcProxy = RPC.getProxy(NamenodeProtocolPB.class,
- RPC.getProtocolVersion(NamenodeProtocolPB.class), nameNodeAddr, ugi,
- conf, NetUtils.getSocketFactory(conf, NamenodeProtocolPB.class));
- }
public NamenodeProtocolTranslatorPB(NamenodeProtocolPB rpcProxy) {
this.rpcProxy = rpcProxy;
@@ -137,7 +123,6 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
}
@Override
- @SuppressWarnings("deprecation")
public CheckpointSignature rollEditLog() throws IOException {
try {
return PBHelper.convert(rpcProxy.rollEditLog(NULL_CONTROLLER,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index fab9f1f1c9..b1e7be0a0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStor
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto.StorageState;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
@@ -119,7 +120,9 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@@ -819,6 +822,23 @@ public class PBHelper {
ReceivedDeletedBlockInfoProto.Builder builder =
ReceivedDeletedBlockInfoProto.newBuilder();
+ ReceivedDeletedBlockInfoProto.BlockStatus status;
+ switch (receivedDeletedBlockInfo.getStatus()) {
+ case RECEIVING_BLOCK:
+ status = ReceivedDeletedBlockInfoProto.BlockStatus.RECEIVING;
+ break;
+ case RECEIVED_BLOCK:
+ status = ReceivedDeletedBlockInfoProto.BlockStatus.RECEIVED;
+ break;
+ case DELETED_BLOCK:
+ status = ReceivedDeletedBlockInfoProto.BlockStatus.DELETED;
+ break;
+ default:
+ throw new IllegalArgumentException("Bad status: " +
+ receivedDeletedBlockInfo.getStatus());
+ }
+ builder.setStatus(status);
+
if (receivedDeletedBlockInfo.getDelHints() != null) {
builder.setDeleteHint(receivedDeletedBlockInfo.getDelHints());
}
@@ -850,7 +870,21 @@ public class PBHelper {
public static ReceivedDeletedBlockInfo convert(
ReceivedDeletedBlockInfoProto proto) {
- return new ReceivedDeletedBlockInfo(PBHelper.convert(proto.getBlock()),
+ ReceivedDeletedBlockInfo.BlockStatus status = null;
+ switch (proto.getStatus()) {
+ case RECEIVING:
+ status = BlockStatus.RECEIVING_BLOCK;
+ break;
+ case RECEIVED:
+ status = BlockStatus.RECEIVED_BLOCK;
+ break;
+ case DELETED:
+ status = BlockStatus.DELETED_BLOCK;
+ break;
+ }
+ return new ReceivedDeletedBlockInfo(
+ PBHelper.convert(proto.getBlock()),
+ status,
proto.hasDeleteHint() ? proto.getDeleteHint() : null);
}
@@ -1245,6 +1279,37 @@ public class PBHelper {
build();
}
+ public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
+ if (s == null) return null;
+ switch (s.getState()) {
+ case ACTIVE:
+ return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.ACTIVE, s.getTxid());
+ case STANDBY:
+ return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.STANDBY, s.getTxid());
+ default:
+ throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" + s.getState());
+ }
+ }
+
+ public static NNHAStatusHeartbeatProto convert(NNHAStatusHeartbeat hb) {
+ if (hb == null) return null;
+ NNHAStatusHeartbeatProto.Builder builder =
+ NNHAStatusHeartbeatProto.newBuilder();
+ switch (hb.getState()) {
+ case ACTIVE:
+ builder.setState(NNHAStatusHeartbeatProto.State.ACTIVE);
+ break;
+ case STANDBY:
+ builder.setState(NNHAStatusHeartbeatProto.State.STANDBY);
+ break;
+ default:
+ throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" +
+ hb.getState());
+ }
+ builder.setTxid(hb.getTxId());
+ return builder.build();
+ }
+
public static DatanodeStorageProto convert(DatanodeStorage s) {
return DatanodeStorageProto.newBuilder()
.setState(PBHelper.convert(s.getState()))
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
index 0fcf424497..96ba2cf7a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshAuthorizationPolicyProtocolClientSideTranslatorPB.java
@@ -20,21 +20,15 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
-import java.net.InetSocketAddress;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclRequestProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import com.google.protobuf.RpcController;
@@ -46,16 +40,10 @@ public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final RefreshAuthorizationPolicyProtocolPB rpcProxy;
-
+
public RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
- InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
- Configuration conf) throws IOException {
- RPC.setProtocolEngine(conf, RefreshAuthorizationPolicyProtocolPB.class,
- ProtobufRpcEngine.class);
- rpcProxy = RPC.getProxy(RefreshAuthorizationPolicyProtocolPB.class,
- RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
- NameNode.getAddress(conf), ugi, conf,
- NetUtils.getSocketFactory(conf, RefreshAuthorizationPolicyProtocol.class));
+ RefreshAuthorizationPolicyProtocolPB rpcProxy) {
+ this.rpcProxy = rpcProxy;
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
index eb8e059e4a..6f07617bab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java
@@ -20,23 +20,17 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
-import java.net.InetSocketAddress;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
-import org.apache.hadoop.security.UserGroupInformation;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@@ -47,16 +41,10 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final RefreshUserMappingsProtocolPB rpcProxy;
-
+
public RefreshUserMappingsProtocolClientSideTranslatorPB(
- InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
- Configuration conf) throws IOException {
- RPC.setProtocolEngine(conf, RefreshUserMappingsProtocolPB.class,
- ProtobufRpcEngine.class);
- rpcProxy = RPC.getProxy(RefreshUserMappingsProtocolPB.class,
- RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
- NameNode.getAddress(conf), ugi, conf,
- NetUtils.getSocketFactory(conf, RefreshUserMappingsProtocol.class));
+ RefreshUserMappingsProtocolPB rpcProxy) {
+ this.rpcProxy = rpcProxy;
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
index b5f24d1855..ba62a2c225 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.security.token.delegation;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
+import java.io.InterruptedIOException;
import java.net.InetSocketAddress;
import java.util.Iterator;
@@ -283,7 +284,18 @@ public class DelegationTokenSecretManager
@Override //AbstractDelegationTokenManager
protected void logUpdateMasterKey(DelegationKey key)
throws IOException {
- namesystem.logUpdateMasterKey(key);
+ synchronized (noInterruptsLock) {
+ // The edit logging code will fail catastrophically if it
+ // is interrupted during a logSync, since the interrupt
+ // closes the edit log files. Doing this inside the
+ // above lock and then checking interruption status
+ // prevents this bug.
+ if (Thread.interrupted()) {
+ throw new InterruptedIOException(
+ "Interrupted before updating master key");
+ }
+ namesystem.logUpdateMasterKey(key);
+ }
}
/** A utility method for creating credentials. */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
index 1822b27a1c..4f73b85164 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
@@ -59,6 +59,11 @@ public class DelegationTokenSelector
new InetSocketAddress(nnAddr.getHostName(), nnRpcPort));
return INSTANCE.selectToken(serviceName, ugi.getTokens());
}
+
+ public static Token selectHdfsDelegationToken(
+ Text serviceName, UserGroupInformation ugi) {
+ return INSTANCE.selectToken(serviceName, ugi.getTokens());
+ }
public DelegationTokenSelector() {
super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index a0146e75a8..e808af623c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -24,8 +24,8 @@ import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.net.InetSocketAddress;
import java.net.Socket;
+import java.net.URI;
import java.text.DateFormat;
import java.util.ArrayList;
import java.util.Arrays;
@@ -1379,7 +1379,7 @@ public class Balancer {
* for each namenode,
* execute a {@link Balancer} to work through all datanodes once.
*/
- static int run(List namenodes, final Parameters p,
+ static int run(Collection namenodes, final Parameters p,
Configuration conf) throws IOException, InterruptedException {
final long sleeptime = 2000*conf.getLong(
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
@@ -1393,8 +1393,8 @@ public class Balancer {
final List connectors
= new ArrayList(namenodes.size());
try {
- for(InetSocketAddress isa : namenodes) {
- connectors.add(new NameNodeConnector(isa, conf));
+ for (URI uri : namenodes) {
+ connectors.add(new NameNodeConnector(uri, conf));
}
boolean done = false;
@@ -1476,7 +1476,7 @@ public class Balancer {
try {
checkReplicationPolicyCompatibility(conf);
- final List namenodes = DFSUtil.getNNServiceRpcAddresses(conf);
+ final Collection namenodes = DFSUtil.getNsServiceRpcUris(conf);
return Balancer.run(namenodes, parse(args), conf);
} catch (IOException e) {
System.out.println(e + ". Exiting ...");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
index 83822e4c31..c4208b7951 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
@@ -21,38 +21,25 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.net.InetAddress;
-import java.net.InetSocketAddress;
+import java.net.URI;
import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Daemon;
@@ -64,7 +51,7 @@ class NameNodeConnector {
private static final Log LOG = Balancer.LOG;
private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
- final InetSocketAddress namenodeAddress;
+ final URI nameNodeUri;
final String blockpoolID;
final NamenodeProtocol namenode;
@@ -78,12 +65,17 @@ class NameNodeConnector {
private BlockTokenSecretManager blockTokenSecretManager;
private Daemon keyupdaterthread; // AccessKeyUpdater thread
- NameNodeConnector(InetSocketAddress namenodeAddress, Configuration conf
- ) throws IOException {
- this.namenodeAddress = namenodeAddress;
- this.namenode = createNamenode(namenodeAddress, conf);
- this.client = DFSUtil.createNamenode(conf);
- this.fs = FileSystem.get(NameNode.getUri(namenodeAddress), conf);
+ NameNodeConnector(URI nameNodeUri,
+ Configuration conf) throws IOException {
+ this.nameNodeUri = nameNodeUri;
+
+ this.namenode =
+ NameNodeProxies.createProxy(conf, nameNodeUri, NamenodeProtocol.class)
+ .getProxy();
+ this.client =
+ NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class)
+ .getProxy();
+ this.fs = FileSystem.get(nameNodeUri, conf);
final NamespaceInfo namespaceinfo = namenode.versionRequest();
this.blockpoolID = namespaceinfo.getBlockPoolID();
@@ -188,38 +180,11 @@ class NameNodeConnector {
@Override
public String toString() {
- return getClass().getSimpleName() + "[namenodeAddress=" + namenodeAddress
+ return getClass().getSimpleName() + "[namenodeUri=" + nameNodeUri
+ ", id=" + blockpoolID
+ "]";
}
- /** Build a NamenodeProtocol connection to the namenode and
- * set up the retry policy
- */
- private static NamenodeProtocol createNamenode(InetSocketAddress address,
- Configuration conf) throws IOException {
- RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(
- 5, 200, TimeUnit.MILLISECONDS);
- Map,RetryPolicy> exceptionToPolicyMap =
- new HashMap, RetryPolicy>();
- RetryPolicy methodPolicy = RetryPolicies.retryByException(
- timeoutPolicy, exceptionToPolicyMap);
- Map methodNameToPolicyMap =
- new HashMap();
- methodNameToPolicyMap.put("getBlocks", methodPolicy);
- methodNameToPolicyMap.put("getAccessKeys", methodPolicy);
-
- RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
- ProtobufRpcEngine.class);
- NamenodeProtocolPB proxy = RPC.getProxy(NamenodeProtocolPB.class,
- RPC.getProtocolVersion(NamenodeProtocolPB.class), address,
- UserGroupInformation.getCurrentUser(), conf,
- NetUtils.getDefaultSocketFactory(conf));
- NamenodeProtocolPB retryProxy = (NamenodeProtocolPB) RetryProxy.create(
- NamenodeProtocolPB.class, proxy, methodNameToPolicyMap);
- return new NamenodeProtocolTranslatorPB(retryProxy);
- }
-
/**
* Periodically updates access keys.
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index 58725a6b32..ce3ff8b3ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -183,7 +183,7 @@ public class BlockInfo extends Block implements
/**
* Count the number of data-nodes the block belongs to.
*/
- int numNodes() {
+ public int numNodes() {
assert this.triplets != null : "BlockInfo is not initialized";
assert triplets.length % 3 == 0 : "Malformed BlockInfo";
for(int idx = getCapacity()-1; idx >= 0; idx--) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index c8f36a01d9..1c9b2aad4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -28,6 +28,8 @@ import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -48,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Util;
@@ -68,6 +71,7 @@ import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.Daemon;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Sets;
/**
* Keeps information related to the blocks stored in the Hadoop cluster.
@@ -80,17 +84,27 @@ public class BlockManager {
/** Default load factor of map */
public static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
+ private static final String QUEUE_REASON_CORRUPT_STATE =
+ "it has the wrong state or generation stamp";
+
+ private static final String QUEUE_REASON_FUTURE_GENSTAMP =
+ "generation stamp is in the future";
+
private final Namesystem namesystem;
private final DatanodeManager datanodeManager;
private final HeartbeatManager heartbeatManager;
private final BlockTokenSecretManager blockTokenSecretManager;
+
+ private final PendingDataNodeMessages pendingDNMessages =
+ new PendingDataNodeMessages();
private volatile long pendingReplicationBlocksCount = 0L;
private volatile long corruptReplicaBlocksCount = 0L;
private volatile long underReplicatedBlocksCount = 0L;
private volatile long scheduledReplicationBlocksCount = 0L;
private volatile long excessBlocksCount = 0L;
+ private volatile long postponedMisreplicatedBlocksCount = 0L;
/** Used by metrics */
public long getPendingReplicationBlocksCount() {
@@ -116,6 +130,14 @@ public class BlockManager {
public long getExcessBlocksCount() {
return excessBlocksCount;
}
+ /** Used by metrics */
+ public long getPostponedMisreplicatedBlocksCount() {
+ return postponedMisreplicatedBlocksCount;
+ }
+ /** Used by metrics */
+ public int getPendingDataNodeMessageCount() {
+ return pendingDNMessages.count();
+ }
/**replicationRecheckInterval is how often namenode checks for new replication work*/
private final long replicationRecheckInterval;
@@ -134,6 +156,15 @@ public class BlockManager {
/** Blocks to be invalidated. */
private final InvalidateBlocks invalidateBlocks;
+
+ /**
+ * After a failover, over-replicated blocks may not be handled
+ * until all of the replicas have done a block report to the
+ * new active. This is to make sure that this NameNode has been
+ * notified of all block deletions that might have been pending
+ * when the failover happened.
+ */
+ private final Set postponedMisreplicatedBlocks = Sets.newHashSet();
//
// Keeps a TreeSet for every named node. Each treeset contains
@@ -316,49 +347,15 @@ public class BlockManager {
out.println("Metasave: Blocks waiting for replication: " +
neededReplications.size());
for (Block block : neededReplications) {
- List containingNodes =
- new ArrayList();
- List containingLiveReplicasNodes =
- new ArrayList();
-
- NumberReplicas numReplicas = new NumberReplicas();
- // source node returned is not used
- chooseSourceDatanode(block, containingNodes,
- containingLiveReplicasNodes, numReplicas);
- assert containingLiveReplicasNodes.size() == numReplicas.liveReplicas();
- int usableReplicas = numReplicas.liveReplicas() +
- numReplicas.decommissionedReplicas();
-
- if (block instanceof BlockInfo) {
- String fileName = ((BlockInfo)block).getINode().getFullPathName();
- out.print(fileName + ": ");
- }
- // l: == live:, d: == decommissioned c: == corrupt e: == excess
- out.print(block + ((usableReplicas > 0)? "" : " MISSING") +
- " (replicas:" +
- " l: " + numReplicas.liveReplicas() +
- " d: " + numReplicas.decommissionedReplicas() +
- " c: " + numReplicas.corruptReplicas() +
- " e: " + numReplicas.excessReplicas() + ") ");
-
- Collection corruptNodes =
- corruptReplicas.getNodes(block);
-
- for (Iterator jt = blocksMap.nodeIterator(block);
- jt.hasNext();) {
- DatanodeDescriptor node = jt.next();
- String state = "";
- if (corruptNodes != null && corruptNodes.contains(node)) {
- state = "(corrupt)";
- } else if (node.isDecommissioned() ||
- node.isDecommissionInProgress()) {
- state = "(decommissioned)";
- }
- out.print(" " + node + state + " : ");
- }
- out.println("");
+ dumpBlockMeta(block, out);
}
}
+
+ // Dump any postponed over-replicated blocks
+ out.println("Mis-replicated blocks that have been postponed:");
+ for (Block block : postponedMisreplicatedBlocks) {
+ dumpBlockMeta(block, out);
+ }
// Dump blocks from pendingReplication
pendingReplications.metaSave(out);
@@ -369,6 +366,58 @@ public class BlockManager {
// Dump all datanodes
getDatanodeManager().datanodeDump(out);
}
+
+ /**
+ * Dump the metadata for the given block in a human-readable
+ * form.
+ */
+ private void dumpBlockMeta(Block block, PrintWriter out) {
+ List containingNodes =
+ new ArrayList();
+ List containingLiveReplicasNodes =
+ new ArrayList();
+
+ NumberReplicas numReplicas = new NumberReplicas();
+ // source node returned is not used
+ chooseSourceDatanode(block, containingNodes,
+ containingLiveReplicasNodes, numReplicas);
+ assert containingLiveReplicasNodes.size() == numReplicas.liveReplicas();
+ int usableReplicas = numReplicas.liveReplicas() +
+ numReplicas.decommissionedReplicas();
+
+ if (block instanceof BlockInfo) {
+ String fileName = ((BlockInfo)block).getINode().getFullPathName();
+ out.print(fileName + ": ");
+ }
+ // l: == live:, d: == decommissioned c: == corrupt e: == excess
+ out.print(block + ((usableReplicas > 0)? "" : " MISSING") +
+ " (replicas:" +
+ " l: " + numReplicas.liveReplicas() +
+ " d: " + numReplicas.decommissionedReplicas() +
+ " c: " + numReplicas.corruptReplicas() +
+ " e: " + numReplicas.excessReplicas() + ") ");
+
+ Collection corruptNodes =
+ corruptReplicas.getNodes(block);
+
+ for (Iterator jt = blocksMap.nodeIterator(block);
+ jt.hasNext();) {
+ DatanodeDescriptor node = jt.next();
+ String state = "";
+ if (corruptNodes != null && corruptNodes.contains(node)) {
+ state = "(corrupt)";
+ } else if (node.isDecommissioned() ||
+ node.isDecommissionInProgress()) {
+ state = "(decommissioned)";
+ }
+
+ if (node.areBlockContentsStale()) {
+ state += " (block deletions maybe out of date)";
+ }
+ out.print(" " + node + state + " : ");
+ }
+ out.println("");
+ }
/** @return maxReplicationStreams */
public int getMaxReplicationStreams() {
@@ -425,7 +474,7 @@ public class BlockManager {
final boolean b = commitBlock((BlockInfoUnderConstruction)lastBlock, commitBlock);
if(countNodes(lastBlock).liveReplicas() >= minReplication)
- completeBlock(fileINode,fileINode.numBlocks()-1);
+ completeBlock(fileINode,fileINode.numBlocks()-1, false);
return b;
}
@@ -437,19 +486,15 @@ public class BlockManager {
* of replicas reported from data-nodes.
*/
private BlockInfo completeBlock(final INodeFile fileINode,
- final int blkIndex) throws IOException {
- return completeBlock(fileINode, blkIndex, false);
- }
-
- public BlockInfo completeBlock(final INodeFile fileINode,
- final int blkIndex, final boolean force) throws IOException {
+ final int blkIndex, boolean force) throws IOException {
if(blkIndex < 0)
return null;
BlockInfo curBlock = fileINode.getBlocks()[blkIndex];
if(curBlock.isComplete())
return curBlock;
BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
- if(!force && ucBlock.numNodes() < minReplication)
+ int numNodes = ucBlock.numNodes();
+ if (!force && numNodes < minReplication)
throw new IOException("Cannot complete block: " +
"block does not satisfy minimal replication requirement.");
if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
@@ -458,20 +503,43 @@ public class BlockManager {
BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
// replace penultimate block in file
fileINode.setBlock(blkIndex, completeBlock);
+
+ // Since safe-mode only counts complete blocks, and we now have
+ // one more complete block, we need to adjust the total up, and
+ // also count it as safe, if we have at least the minimum replica
+ // count. (We may not have the minimum replica count yet if this is
+ // a "forced" completion when a file is getting closed by an
+ // OP_CLOSE edit on the standby).
+ namesystem.adjustSafeModeBlockTotals(0, 1);
+ namesystem.incrementSafeBlockCount(
+ Math.min(numNodes, minReplication));
+
// replace block in the blocksMap
return blocksMap.replaceBlock(completeBlock);
}
private BlockInfo completeBlock(final INodeFile fileINode,
- final BlockInfo block) throws IOException {
+ final BlockInfo block, boolean force) throws IOException {
BlockInfo[] fileBlocks = fileINode.getBlocks();
for(int idx = 0; idx < fileBlocks.length; idx++)
if(fileBlocks[idx] == block) {
- return completeBlock(fileINode, idx);
+ return completeBlock(fileINode, idx, force);
}
return block;
}
+
+ /**
+ * Force the given block in the given file to be marked as complete,
+ * regardless of whether enough replicas are present. This is necessary
+ * when tailing edit logs as a Standby.
+ */
+ public BlockInfo forceCompleteBlock(final INodeFile fileINode,
+ final BlockInfoUnderConstruction block) throws IOException {
+ block.commitBlock(block);
+ return completeBlock(fileINode, block, true);
+ }
+
/**
* Convert the last block of the file to an under construction block.
* The block is converted only if the file has blocks and the last one
@@ -508,6 +576,14 @@ public class BlockManager {
String datanodeId = dd.getStorageID();
invalidateBlocks.remove(datanodeId, oldBlock);
}
+
+ // Adjust safe-mode totals, since under-construction blocks don't
+ // count in safe-mode.
+ namesystem.adjustSafeModeBlockTotals(
+ // decrement safe if we had enough
+ targets.length >= minReplication ? -1 : 0,
+ // always decrement total blocks
+ -1);
final long fileLength = fileINode.computeContentSummary().getLength();
final long pos = fileLength - ucBlock.getNumBytes();
@@ -598,8 +674,8 @@ public class BlockManager {
final boolean isCorrupt = numCorruptNodes == numNodes;
final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines];
+ int j = 0;
if (numMachines > 0) {
- int j = 0;
for(Iterator it = blocksMap.nodeIterator(blk);
it.hasNext();) {
final DatanodeDescriptor d = it.next();
@@ -608,6 +684,12 @@ public class BlockManager {
machines[j++] = d;
}
}
+ assert j == machines.length :
+ "isCorrupt: " + isCorrupt +
+ " numMachines: " + numMachines +
+ " numNodes: " + numNodes +
+ " numCorrupt: " + numCorruptNodes +
+ " numCorruptRepls: " + numCorruptReplicas;
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
return new LocatedBlock(eb, machines, pos, isCorrupt);
}
@@ -772,6 +854,14 @@ public class BlockManager {
node.resetBlocks();
invalidateBlocks.remove(node.getStorageID());
+
+ // If the DN hasn't block-reported since the most recent
+ // failover, then we may have been holding up on processing
+ // over-replicated blocks because of it. But we can now
+ // process those blocks.
+ if (node.areBlockContentsStale()) {
+ rescanPostponedMisreplicatedBlocks();
+ }
}
/**
@@ -809,22 +899,18 @@ public class BlockManager {
*/
public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
final DatanodeInfo dn, String reason) throws IOException {
- namesystem.writeLock();
- try {
- final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
- if (storedBlock == null) {
- // Check if the replica is in the blockMap, if not
- // ignore the request for now. This could happen when BlockScanner
- // thread of Datanode reports bad block before Block reports are sent
- // by the Datanode on startup
- NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
- + blk + " not found.");
- return;
- }
- markBlockAsCorrupt(storedBlock, dn, reason);
- } finally {
- namesystem.writeUnlock();
+ assert namesystem.hasWriteLock();
+ final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
+ if (storedBlock == null) {
+ // Check if the replica is in the blockMap, if not
+ // ignore the request for now. This could happen when BlockScanner
+ // thread of Datanode reports bad block before Block reports are sent
+ // by the Datanode on startup
+ NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
+ + blk + " not found.");
+ return;
}
+ markBlockAsCorrupt(storedBlock, dn, reason);
}
private void markBlockAsCorrupt(BlockInfo storedBlock,
@@ -876,10 +962,17 @@ public class BlockManager {
+ " because datanode " + dn.getName() + " does not exist.");
}
- // Check how many copies we have of the block. If we have at least one
- // copy on a live node, then we can delete it.
- int count = countNodes(blk).liveReplicas();
- if (count >= 1) {
+ // Check how many copies we have of the block
+ NumberReplicas nr = countNodes(blk);
+ if (nr.replicasOnStaleNodes() > 0) {
+ NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: postponing " +
+ "invalidation of block " + blk + " on " + dn + " because " +
+ nr.replicasOnStaleNodes() + " replica(s) are located on nodes " +
+ "with potentially out-of-date block reports.");
+ postponeBlock(blk);
+
+ } else if (nr.liveReplicas() >= 1) {
+ // If we have at least one copy on a live node, then we can delete it.
addToInvalidates(blk, dn);
removeStoredBlock(blk, node);
if(NameNode.stateChangeLog.isDebugEnabled()) {
@@ -892,6 +985,13 @@ public class BlockManager {
}
}
+ private void postponeBlock(Block blk) {
+ if (postponedMisreplicatedBlocks.add(blk)) {
+ postponedMisreplicatedBlocksCount++;
+ }
+ }
+
+
void updateState() {
pendingReplicationBlocksCount = pendingReplications.size();
underReplicatedBlocksCount = neededReplications.size();
@@ -930,7 +1030,7 @@ public class BlockManager {
*
* @return number of blocks scheduled for replication during this iteration.
*/
- private int computeReplicationWork(int blocksToProcess) throws IOException {
+ int computeReplicationWork(int blocksToProcess) throws IOException {
List> blocksToReplicate = null;
namesystem.writeLock();
try {
@@ -981,8 +1081,10 @@ public class BlockManager {
NumberReplicas numReplicas = new NumberReplicas();
srcNode = chooseSourceDatanode(
block, containingNodes, liveReplicaNodes, numReplicas);
- if(srcNode == null) // block can not be replicated from any node
+ if(srcNode == null) { // block can not be replicated from any node
+ LOG.debug("Block " + block + " cannot be repl from any node");
continue;
+ }
assert liveReplicaNodes.size() == numReplicas.liveReplicas();
// do not schedule more if enough replicas is already pending
@@ -1232,7 +1334,7 @@ public class BlockManager {
srcNode = node;
}
if(numReplicas != null)
- numReplicas.initialize(live, decommissioned, corrupt, excess);
+ numReplicas.initialize(live, decommissioned, corrupt, excess, 0);
return srcNode;
}
@@ -1314,7 +1416,7 @@ public class BlockManager {
// To minimize startup time, we discard any second (or later) block reports
// that we receive while still in startup phase.
- if (namesystem.isInStartupSafeMode() && node.numBlocks() > 0) {
+ if (namesystem.isInStartupSafeMode() && !node.isFirstBlockReport()) {
NameNode.stateChangeLog.info("BLOCK* processReport: "
+ "discarded non-initial block report from " + nodeID.getName()
+ " because namenode still in startup phase");
@@ -1328,6 +1430,19 @@ public class BlockManager {
} else {
processReport(node, newReport);
}
+
+ // Now that we have an up-to-date block report, we know that any
+ // deletions from a previous NN iteration have been accounted for.
+ boolean staleBefore = node.areBlockContentsStale();
+ node.receivedBlockReport();
+ if (staleBefore && !node.areBlockContentsStale()) {
+ LOG.info("BLOCK* processReport: " +
+ "Received first block report from " + node +
+ " after becoming active. Its block contents are no longer" +
+ " considered stale.");
+ rescanPostponedMisreplicatedBlocks();
+ }
+
} finally {
endTime = Util.now();
namesystem.writeUnlock();
@@ -1340,6 +1455,37 @@ public class BlockManager {
+ ", processing time: " + (endTime - startTime) + " msecs");
}
+ /**
+ * Rescan the list of blocks which were previously postponed.
+ */
+ private void rescanPostponedMisreplicatedBlocks() {
+ for (Iterator it = postponedMisreplicatedBlocks.iterator();
+ it.hasNext();) {
+ Block b = it.next();
+
+ BlockInfo bi = blocksMap.getStoredBlock(b);
+ if (bi == null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
+ "Postponed mis-replicated block " + b + " no longer found " +
+ "in block map.");
+ }
+ it.remove();
+ postponedMisreplicatedBlocksCount--;
+ continue;
+ }
+ MisReplicationResult res = processMisReplicatedBlock(bi);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
+ "Re-scanned block " + b + ", result is " + res);
+ }
+ if (res != MisReplicationResult.POSTPONE) {
+ it.remove();
+ postponedMisreplicatedBlocksCount--;
+ }
+ }
+ }
+
private void processReport(final DatanodeDescriptor node,
final BlockListAsLongs report) throws IOException {
// Normal case:
@@ -1392,9 +1538,19 @@ public class BlockManager {
assert (node.numBlocks() == 0);
BlockReportIterator itBR = report.getBlockReportIterator();
+ boolean isStandby = namesystem.isInStandbyState();
+
while(itBR.hasNext()) {
Block iblk = itBR.next();
ReplicaState reportedState = itBR.getCurrentReplicaState();
+
+ if (isStandby &&
+ namesystem.isGenStampInFuture(iblk.getGenerationStamp())) {
+ queueReportedBlock(node, iblk, reportedState,
+ QUEUE_REASON_FUTURE_GENSTAMP);
+ continue;
+ }
+
BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
// If block does not belong to any file, we are done.
if (storedBlock == null) continue;
@@ -1404,7 +1560,14 @@ public class BlockManager {
BlockToMarkCorrupt c = checkReplicaCorrupt(
iblk, reportedState, storedBlock, ucState, node);
if (c != null) {
- markBlockAsCorrupt(c.blockInfo, node, c.reason);
+ if (namesystem.isInStandbyState()) {
+ // In the Standby, we may receive a block report for a file that we
+ // just have an out-of-date gen-stamp or state for, for example.
+ queueReportedBlock(node, iblk, reportedState,
+ QUEUE_REASON_CORRUPT_STATE);
+ } else {
+ markBlockAsCorrupt(c.blockInfo, node, c.reason);
+ }
continue;
}
@@ -1487,7 +1650,8 @@ public class BlockManager {
* @param toCorrupt replicas with unexpected length or generation stamp;
* add to corrupt replicas
* @param toUC replicas of blocks currently under construction
- * @return
+ * @return the up-to-date stored block, if it should be kept.
+ * Otherwise, null.
*/
private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
final Block block, final ReplicaState reportedState,
@@ -1502,6 +1666,13 @@ public class BlockManager {
+ " replicaState = " + reportedState);
}
+ if (namesystem.isInStandbyState() &&
+ namesystem.isGenStampInFuture(block.getGenerationStamp())) {
+ queueReportedBlock(dn, block, reportedState,
+ QUEUE_REASON_FUTURE_GENSTAMP);
+ return null;
+ }
+
// find block by blockId
BlockInfo storedBlock = blocksMap.getStoredBlock(block);
if(storedBlock == null) {
@@ -1519,15 +1690,24 @@ public class BlockManager {
// Ignore replicas already scheduled to be removed from the DN
if(invalidateBlocks.contains(dn.getStorageID(), block)) {
- assert storedBlock.findDatanode(dn) < 0 : "Block " + block
- + " in invalidated blocks set should not appear in DN " + dn;
+/* TODO: following assertion is incorrect, see HDFS-2668
+assert storedBlock.findDatanode(dn) < 0 : "Block " + block
+ + " in recentInvalidatesSet should not appear in DN " + dn; */
return storedBlock;
}
BlockToMarkCorrupt c = checkReplicaCorrupt(
block, reportedState, storedBlock, ucState, dn);
if (c != null) {
- toCorrupt.add(c);
+ if (namesystem.isInStandbyState()) {
+ // If the block is an out-of-date generation stamp or state,
+ // but we're the standby, we shouldn't treat it as corrupt,
+ // but instead just queue it for later processing.
+ queueReportedBlock(dn, storedBlock, reportedState,
+ QUEUE_REASON_CORRUPT_STATE);
+ } else {
+ toCorrupt.add(c);
+ }
return storedBlock;
}
@@ -1545,6 +1725,68 @@ public class BlockManager {
return storedBlock;
}
+ /**
+ * Queue the given reported block for later processing in the
+ * standby node. {@see PendingDataNodeMessages}.
+ * @param reason a textual reason to report in the debug logs
+ */
+ private void queueReportedBlock(DatanodeDescriptor dn, Block block,
+ ReplicaState reportedState, String reason) {
+ assert namesystem.isInStandbyState();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Queueing reported block " + block +
+ " in state " + reportedState +
+ " from datanode " + dn + " for later processing " +
+ "because " + reason + ".");
+ }
+ pendingDNMessages.enqueueReportedBlock(dn, block, reportedState);
+ }
+
+ /**
+ * Try to process any messages that were previously queued for the given
+ * block. This is called from FSEditLogLoader whenever a block's state
+ * in the namespace has changed or a new block has been created.
+ */
+ public void processQueuedMessagesForBlock(Block b) throws IOException {
+ Queue queue = pendingDNMessages.takeBlockQueue(b);
+ if (queue == null) {
+ // Nothing to re-process
+ return;
+ }
+ processQueuedMessages(queue);
+ }
+
+ private void processQueuedMessages(Iterable rbis)
+ throws IOException {
+ for (ReportedBlockInfo rbi : rbis) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Processing previouly queued message " + rbi);
+ }
+ processAndHandleReportedBlock(
+ rbi.getNode(), rbi.getBlock(), rbi.getReportedState(), null);
+ }
+ }
+
+ /**
+ * Process any remaining queued datanode messages after entering
+ * active state. At this point they will not be re-queued since
+ * we are the definitive master node and thus should be up-to-date
+ * with the namespace information.
+ */
+ public void processAllPendingDNMessages() throws IOException {
+ assert !namesystem.isInStandbyState() :
+ "processAllPendingDNMessages() should be called after exiting " +
+ "standby state!";
+ int count = pendingDNMessages.count();
+ if (count > 0) {
+ LOG.info("Processing " + count + " messages from DataNodes " +
+ "that were previously queued during standby state.");
+ }
+ processQueuedMessages(pendingDNMessages.takeAll());
+ assert pendingDNMessages.count() == 0;
+ }
+
/*
* The next two methods test the various cases under which we must conclude
* the replica is corrupt, or under construction. These are laid out
@@ -1675,13 +1917,15 @@ public class BlockManager {
// Now check for completion of blocks and safe block count
int numCurrentReplica = countLiveNodes(storedBlock);
if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
- && numCurrentReplica >= minReplication)
- storedBlock = completeBlock(storedBlock.getINode(), storedBlock);
-
- // check whether safe replication is reached for the block
- // only complete blocks are counted towards that
- if(storedBlock.isComplete())
+ && numCurrentReplica >= minReplication) {
+ completeBlock(storedBlock.getINode(), storedBlock, false);
+ } else if (storedBlock.isComplete()) {
+ // check whether safe replication is reached for the block
+ // only complete blocks are counted towards that.
+ // In the case that the block just became complete above, completeBlock()
+ // handles the safe block count maintenance.
namesystem.incrementSafeBlockCount(numCurrentReplica);
+ }
}
/**
@@ -1738,15 +1982,17 @@ public class BlockManager {
+ pendingReplications.getNumReplicas(storedBlock);
if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
- numLiveReplicas >= minReplication)
- storedBlock = completeBlock(fileINode, storedBlock);
-
- // check whether safe replication is reached for the block
- // only complete blocks are counted towards that
- // Is no-op if not in safe mode.
- if(storedBlock.isComplete())
+ numLiveReplicas >= minReplication) {
+ storedBlock = completeBlock(fileINode, storedBlock, false);
+ } else if (storedBlock.isComplete()) {
+ // check whether safe replication is reached for the block
+ // only complete blocks are counted towards that
+ // Is no-op if not in safe mode.
+ // In the case that the block just became complete above, completeBlock()
+ // handles the safe block count maintenance.
namesystem.incrementSafeBlockCount(numCurrentReplica);
-
+ }
+
// if file is under construction, then done for now
if (fileINode.isUnderConstruction()) {
return storedBlock;
@@ -1839,49 +2085,93 @@ public class BlockManager {
public void processMisReplicatedBlocks() {
assert namesystem.hasWriteLock();
- long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0,
+ long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0, nrPostponed = 0,
nrUnderConstruction = 0;
neededReplications.clear();
for (BlockInfo block : blocksMap.getBlocks()) {
- INodeFile fileINode = block.getINode();
- if (fileINode == null) {
- // block does not belong to any file
- nrInvalid++;
- addToInvalidates(block);
- continue;
+ MisReplicationResult res = processMisReplicatedBlock(block);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("block " + block + ": " + res);
}
- if (!block.isComplete()) {
- // Incomplete blocks are never considered mis-replicated --
- // they'll be reached when they are completed or recovered.
- nrUnderConstruction++;
- continue;
- }
- // calculate current replication
- short expectedReplication = fileINode.getReplication();
- NumberReplicas num = countNodes(block);
- int numCurrentReplica = num.liveReplicas();
- // add to under-replicated queue if need to be
- if (isNeededReplication(block, expectedReplication, numCurrentReplica)) {
- if (neededReplications.add(block, numCurrentReplica, num
- .decommissionedReplicas(), expectedReplication)) {
- nrUnderReplicated++;
- }
- }
-
- if (numCurrentReplica > expectedReplication) {
- // over-replicated block
+ switch (res) {
+ case UNDER_REPLICATED:
+ nrUnderReplicated++;
+ break;
+ case OVER_REPLICATED:
nrOverReplicated++;
- processOverReplicatedBlock(block, expectedReplication, null, null);
+ break;
+ case INVALID:
+ nrInvalid++;
+ break;
+ case POSTPONE:
+ nrPostponed++;
+ postponeBlock(block);
+ break;
+ case UNDER_CONSTRUCTION:
+ nrUnderConstruction++;
+ break;
+ case OK:
+ break;
+ default:
+ throw new AssertionError("Invalid enum value: " + res);
}
}
-
+
LOG.info("Total number of blocks = " + blocksMap.size());
LOG.info("Number of invalid blocks = " + nrInvalid);
LOG.info("Number of under-replicated blocks = " + nrUnderReplicated);
- LOG.info("Number of over-replicated blocks = " + nrOverReplicated);
+ LOG.info("Number of over-replicated blocks = " + nrOverReplicated +
+ ((nrPostponed > 0) ? ( " (" + nrPostponed + " postponed)") : ""));
LOG.info("Number of blocks being written = " + nrUnderConstruction);
}
+ /**
+ * Process a single possibly misreplicated block. This adds it to the
+ * appropriate queues if necessary, and returns a result code indicating
+ * what happened with it.
+ */
+ private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
+ INodeFile fileINode = block.getINode();
+ if (fileINode == null) {
+ // block does not belong to any file
+ addToInvalidates(block);
+ return MisReplicationResult.INVALID;
+ }
+ if (!block.isComplete()) {
+ // Incomplete blocks are never considered mis-replicated --
+ // they'll be reached when they are completed or recovered.
+ return MisReplicationResult.UNDER_CONSTRUCTION;
+ }
+ // calculate current replication
+ short expectedReplication = fileINode.getReplication();
+ NumberReplicas num = countNodes(block);
+ int numCurrentReplica = num.liveReplicas();
+ // add to under-replicated queue if need to be
+ if (isNeededReplication(block, expectedReplication, numCurrentReplica)) {
+ if (neededReplications.add(block, numCurrentReplica, num
+ .decommissionedReplicas(), expectedReplication)) {
+ return MisReplicationResult.UNDER_REPLICATED;
+ }
+ }
+
+ if (numCurrentReplica > expectedReplication) {
+ if (num.replicasOnStaleNodes() > 0) {
+ // If any of the replicas of this block are on nodes that are
+ // considered "stale", then these replicas may in fact have
+ // already been deleted. So, we cannot safely act on the
+ // over-replication until a later point in time, when
+ // the "stale" nodes have block reported.
+ return MisReplicationResult.POSTPONE;
+ }
+
+ // over-replicated block
+ processOverReplicatedBlock(block, expectedReplication, null, null);
+ return MisReplicationResult.OVER_REPLICATED;
+ }
+
+ return MisReplicationResult.OK;
+ }
+
/** Set replication for the blocks. */
public void setReplication(final short oldRepl, final short newRepl,
final String src, final Block... blocks) throws IOException {
@@ -1925,6 +2215,14 @@ public class BlockManager {
for (Iterator it = blocksMap.nodeIterator(block);
it.hasNext();) {
DatanodeDescriptor cur = it.next();
+ if (cur.areBlockContentsStale()) {
+ LOG.info("BLOCK* processOverReplicatedBlock: " +
+ "Postponing processing of over-replicated block " +
+ block + " since datanode " + cur + " does not yet have up-to-date " +
+ "block information.");
+ postponeBlock(block);
+ return;
+ }
LightWeightLinkedSet excessBlocks = excessReplicateMap.get(cur
.getStorageID());
if (excessBlocks == null || !excessBlocks.contains(block)) {
@@ -2151,13 +2449,19 @@ public class BlockManager {
// Modify the blocks->datanode map and node's map.
//
pendingReplications.remove(block);
-
+ processAndHandleReportedBlock(node, block, ReplicaState.FINALIZED,
+ delHintNode);
+ }
+
+ private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
+ ReplicaState reportedState, DatanodeDescriptor delHintNode)
+ throws IOException {
// blockReceived reports a finalized block
Collection toAdd = new LinkedList();
Collection toInvalidate = new LinkedList();
Collection toCorrupt = new LinkedList();
Collection toUC = new LinkedList();
- processReportedBlock(node, block, ReplicaState.FINALIZED,
+ processReportedBlock(node, block, reportedState,
toAdd, toInvalidate, toCorrupt, toUC);
// the block is only in one of the to-do lists
// if it is in none then data-node already has it
@@ -2181,59 +2485,80 @@ public class BlockManager {
}
}
- /** The given node is reporting that it received/deleted certain blocks. */
- public void blockReceivedAndDeleted(final DatanodeID nodeID,
+ /**
+ * The given node is reporting incremental information about some blocks.
+ * This includes blocks that are starting to be received, completed being
+ * received, or deleted.
+ */
+ public void processIncrementalBlockReport(final DatanodeID nodeID,
final String poolId,
- final ReceivedDeletedBlockInfo receivedAndDeletedBlocks[]
+ final ReceivedDeletedBlockInfo blockInfos[]
) throws IOException {
namesystem.writeLock();
int received = 0;
int deleted = 0;
+ int receiving = 0;
try {
final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
if (node == null || !node.isAlive) {
NameNode.stateChangeLog
- .warn("BLOCK* blockReceivedDeleted"
+ .warn("BLOCK* processIncrementalBlockReport"
+ " is received from dead or unregistered node "
+ nodeID.getName());
throw new IOException(
- "Got blockReceivedDeleted message from unregistered or dead node");
+ "Got incremental block report from unregistered or dead node");
}
- for (int i = 0; i < receivedAndDeletedBlocks.length; i++) {
- if (receivedAndDeletedBlocks[i].isDeletedBlock()) {
- removeStoredBlock(
- receivedAndDeletedBlocks[i].getBlock(), node);
+ for (ReceivedDeletedBlockInfo rdbi : blockInfos) {
+ switch (rdbi.getStatus()) {
+ case DELETED_BLOCK:
+ removeStoredBlock(rdbi.getBlock(), node);
deleted++;
- } else {
- addBlock(node, receivedAndDeletedBlocks[i].getBlock(),
- receivedAndDeletedBlocks[i].getDelHints());
+ break;
+ case RECEIVED_BLOCK:
+ addBlock(node, rdbi.getBlock(), rdbi.getDelHints());
received++;
+ break;
+ case RECEIVING_BLOCK:
+ receiving++;
+ processAndHandleReportedBlock(node, rdbi.getBlock(),
+ ReplicaState.RBW, null);
+ break;
+ default:
+ String msg =
+ "Unknown block status code reported by " + nodeID.getName() +
+ ": " + rdbi;
+ NameNode.stateChangeLog.warn(msg);
+ assert false : msg; // if assertions are enabled, throw.
+ break;
}
if (NameNode.stateChangeLog.isDebugEnabled()) {
- NameNode.stateChangeLog.debug("BLOCK* block"
- + (receivedAndDeletedBlocks[i].isDeletedBlock() ? "Deleted"
- : "Received") + ": " + receivedAndDeletedBlocks[i].getBlock()
+ NameNode.stateChangeLog.debug("BLOCK* block "
+ + (rdbi.getStatus()) + ": " + rdbi.getBlock()
+ " is received from " + nodeID.getName());
}
}
} finally {
namesystem.writeUnlock();
NameNode.stateChangeLog
- .debug("*BLOCK* NameNode.blockReceivedAndDeleted: " + "from "
- + nodeID.getName() + " received: " + received + ", "
+ .debug("*BLOCK* NameNode.processIncrementalBlockReport: " + "from "
+ + nodeID.getName()
+ + " receiving: " + receiving + ", "
+ + " received: " + received + ", "
+ " deleted: " + deleted);
}
}
/**
- * Return the number of nodes that are live and decommissioned.
+ * Return the number of nodes hosting a given block, grouped
+ * by the state of those replicas.
*/
public NumberReplicas countNodes(Block b) {
- int count = 0;
+ int decommissioned = 0;
int live = 0;
int corrupt = 0;
int excess = 0;
+ int stale = 0;
Iterator nodeIter = blocksMap.nodeIterator(b);
Collection nodesCorrupt = corruptReplicas.getNodes(b);
while (nodeIter.hasNext()) {
@@ -2241,7 +2566,7 @@ public class BlockManager {
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
corrupt++;
} else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
- count++;
+ decommissioned++;
} else {
LightWeightLinkedSet blocksExcess = excessReplicateMap.get(node
.getStorageID());
@@ -2251,8 +2576,11 @@ public class BlockManager {
live++;
}
}
+ if (node.areBlockContentsStale()) {
+ stale++;
+ }
}
- return new NumberReplicas(live, count, corrupt, excess);
+ return new NumberReplicas(live, decommissioned, corrupt, excess, stale);
}
/**
@@ -2379,7 +2707,7 @@ public class BlockManager {
}
public int getActiveBlockCount() {
- return blocksMap.size() - (int)invalidateBlocks.numBlocks();
+ return blocksMap.size();
}
public DatanodeDescriptor[] getNodes(BlockInfo block) {
@@ -2397,10 +2725,17 @@ public class BlockManager {
}
public void removeBlock(Block block) {
+ assert namesystem.hasWriteLock();
+ // No need to ACK blocks that are being removed entirely
+ // from the namespace, since the removal of the associated
+ // file already removes them from the block map below.
block.setNumBytes(BlockCommand.NO_ACK);
addToInvalidates(block);
corruptReplicas.removeFromCorruptReplicasMap(block);
blocksMap.removeBlock(block);
+ if (postponedMisreplicatedBlocks.remove(block)) {
+ postponedMisreplicatedBlocksCount--;
+ }
}
public BlockInfo getStoredBlock(Block block) {
@@ -2412,6 +2747,9 @@ public class BlockManager {
final int curReplicasDelta, int expectedReplicasDelta) {
namesystem.writeLock();
try {
+ if (!namesystem.isPopulatingReplQueues()) {
+ return;
+ }
NumberReplicas repl = countNodes(block);
int curExpectedReplicas = getReplication(block);
if (isNeededReplication(block, curExpectedReplicas, repl.liveReplicas())) {
@@ -2461,8 +2799,10 @@ public class BlockManager {
namesystem.writeLock();
try {
// blocks should not be replicated or removed if safe mode is on
- if (namesystem.isInSafeMode())
+ if (namesystem.isInSafeMode()) {
+ LOG.debug("In safemode, not computing replication work");
return 0;
+ }
// get blocks to invalidate for the nodeId
assert nodeId != null;
return invalidateBlocks.invalidateWork(nodeId);
@@ -2645,6 +2985,19 @@ public class BlockManager {
return workFound;
}
+ /**
+ * Clear all queues that hold decisions previously made by
+ * this NameNode.
+ */
+ public void clearQueues() {
+ neededReplications.clear();
+ pendingReplications.clear();
+ excessReplicateMap.clear();
+ invalidateBlocks.clear();
+ datanodeManager.clearPendingQueues();
+ };
+
+
private static class ReplicationWork {
private Block block;
@@ -2675,4 +3028,24 @@ public class BlockManager {
this.targets = null;
}
}
+
+ /**
+ * A simple result enum for the result of
+ * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
+ */
+ enum MisReplicationResult {
+ /** The block should be invalidated since it belongs to a deleted file. */
+ INVALID,
+ /** The block is currently under-replicated. */
+ UNDER_REPLICATED,
+ /** The block is currently over-replicated. */
+ OVER_REPLICATED,
+ /** A decision can't currently be made about this block. */
+ POSTPONE,
+ /** The block is under construction, so should be ignored */
+ UNDER_CONSTRUCTION,
+ /** The block is properly replicated */
+ OK
+ }
+
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index ac1a7e68e0..058d2e37aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -63,7 +63,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
initialize(conf, stats, clusterMap);
}
- BlockPlacementPolicyDefault() {
+ protected BlockPlacementPolicyDefault() {
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index d927f05297..984456f142 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -94,6 +94,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
boolean contains(E e) {
return blockq.contains(e);
}
+
+ synchronized void clear() {
+ blockq.clear();
+ }
}
private volatile BlockInfo blockList = null;
@@ -103,6 +107,24 @@ public class DatanodeDescriptor extends DatanodeInfo {
public boolean isAlive = false;
public boolean needKeyUpdate = false;
+ /**
+ * Set to false on any NN failover, and reset to true
+ * whenever a block report is received.
+ */
+ private boolean heartbeatedSinceFailover = false;
+
+ /**
+ * At startup or at any failover, the DNs in the cluster may
+ * have pending block deletions from a previous incarnation
+ * of the NameNode. Thus, we consider their block contents
+ * stale until we have received a block report. When a DN
+ * is considered stale, any replicas on it are transitively
+ * considered stale. If any block has at least one stale replica,
+ * then no invalidations will be processed for this block.
+ * See HDFS-1972.
+ */
+ private boolean blockContentsStale = true;
+
// A system administrator can tune the balancer bandwidth parameter
// (dfs.balance.bandwidthPerSec) dynamically by calling
// "dfsadmin -setBalanacerBandwidth ", at which point the
@@ -129,6 +151,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
private long lastBlocksScheduledRollTime = 0;
private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
private int volumeFailures = 0;
+
+ /** Set to false after processing first block report */
+ private boolean firstBlockReport = true;
+
/**
* When set to true, the node is not in include list and is not allowed
* to communicate with the namenode
@@ -281,6 +307,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.invalidateBlocks.clear();
this.volumeFailures = 0;
}
+
+ public void clearBlockQueues() {
+ synchronized (invalidateBlocks) {
+ this.invalidateBlocks.clear();
+ this.recoverBlocks.clear();
+ this.replicateBlocks.clear();
+ }
+ }
public int numBlocks() {
return numBlocks;
@@ -298,6 +332,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.lastUpdate = System.currentTimeMillis();
this.xceiverCount = xceiverCount;
this.volumeFailures = volFailures;
+ this.heartbeatedSinceFailover = true;
rollBlocksScheduled(lastUpdate);
}
@@ -564,5 +599,41 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.bandwidth = bandwidth;
}
+ public boolean areBlockContentsStale() {
+ return blockContentsStale;
+ }
+ public void markStaleAfterFailover() {
+ heartbeatedSinceFailover = false;
+ blockContentsStale = true;
+ }
+
+ public void receivedBlockReport() {
+ if (heartbeatedSinceFailover) {
+ blockContentsStale = false;
+ }
+ firstBlockReport = false;
+ }
+
+ boolean isFirstBlockReport() {
+ return firstBlockReport;
+ }
+
+ @Override
+ public String dumpDatanode() {
+ StringBuilder sb = new StringBuilder(super.dumpDatanode());
+ int repl = replicateBlocks.size();
+ if (repl > 0) {
+ sb.append(" ").append(repl).append(" blocks to be replicated;");
+ }
+ int inval = invalidateBlocks.size();
+ if (inval > 0) {
+ sb.append(" ").append(inval).append(" blocks to be invalidated;");
+ }
+ int recover = recoverBlocks.size();
+ if (recover > 0) {
+ sb.append(" ").append(recover).append(" blocks to be recovered;");
+ }
+ return sb.toString();
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 5d795e7445..8c59ccba5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -923,7 +923,7 @@ public class DatanodeManager {
}
}
- return null;
+ return new DatanodeCommand[0];
}
/**
@@ -947,4 +947,27 @@ public class DatanodeManager {
}
}
}
+
+ public void markAllDatanodesStale() {
+ LOG.info("Marking all datandoes as stale");
+ synchronized (datanodeMap) {
+ for (DatanodeDescriptor dn : datanodeMap.values()) {
+ dn.markStaleAfterFailover();
+ }
+ }
+ }
+
+ /**
+ * Clear any actions that are queued up to be sent to the DNs
+ * on their next heartbeats. This includes block invalidations,
+ * recoveries, and replication requests.
+ */
+ public void clearPendingQueues() {
+ synchronized (datanodeMap) {
+ for (DatanodeDescriptor dn : datanodeMap.values()) {
+ dn.clearBlockQueues();
+ }
+ }
+ }
+
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
index 2c6b46f050..5c7e0bdca1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
@@ -160,4 +160,9 @@ class InvalidateBlocks {
numBlocks -= toInvalidate.size();
return toInvalidate;
}
+
+ synchronized void clear() {
+ node2blocks.clear();
+ numBlocks = 0;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
index 52f62587b1..9e5c8dfd5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
@@ -26,20 +26,22 @@ public class NumberReplicas {
private int decommissionedReplicas;
private int corruptReplicas;
private int excessReplicas;
+ private int replicasOnStaleNodes;
NumberReplicas() {
- initialize(0, 0, 0, 0);
+ initialize(0, 0, 0, 0, 0);
}
- NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
- initialize(live, decommissioned, corrupt, excess);
+ NumberReplicas(int live, int decommissioned, int corrupt, int excess, int stale) {
+ initialize(live, decommissioned, corrupt, excess, stale);
}
- void initialize(int live, int decommissioned, int corrupt, int excess) {
+ void initialize(int live, int decommissioned, int corrupt, int excess, int stale) {
liveReplicas = live;
decommissionedReplicas = decommissioned;
corruptReplicas = corrupt;
excessReplicas = excess;
+ replicasOnStaleNodes = stale;
}
public int liveReplicas() {
@@ -54,4 +56,13 @@ public class NumberReplicas {
public int excessReplicas() {
return excessReplicas;
}
+
+ /**
+ * @return the number of replicas which are on stale nodes.
+ * This is not mutually exclusive with the other counts -- ie a
+ * replica may count as both "live" and "stale".
+ */
+ public int replicasOnStaleNodes() {
+ return replicasOnStaleNodes;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java
new file mode 100644
index 0000000000..b7da116048
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * In the Standby Node, we can receive messages about blocks
+ * before they are actually available in the namespace, or while
+ * they have an outdated state in the namespace. In those cases,
+ * we queue those block-related messages in this structure.
+ * */
+class PendingDataNodeMessages {
+
+ Map> queueByBlockId =
+ Maps.newHashMap();
+ private int count = 0;
+
+
+ static class ReportedBlockInfo {
+ private final Block block;
+ private final DatanodeDescriptor dn;
+ private final ReplicaState reportedState;
+
+ ReportedBlockInfo(DatanodeDescriptor dn, Block block,
+ ReplicaState reportedState) {
+ this.dn = dn;
+ this.block = block;
+ this.reportedState = reportedState;
+ }
+
+ Block getBlock() {
+ return block;
+ }
+
+ DatanodeDescriptor getNode() {
+ return dn;
+ }
+
+ ReplicaState getReportedState() {
+ return reportedState;
+ }
+
+ @Override
+ public String toString() {
+ return "ReportedBlockInfo [block=" + block + ", dn=" + dn
+ + ", reportedState=" + reportedState + "]";
+ }
+ }
+
+ void enqueueReportedBlock(DatanodeDescriptor dn, Block block,
+ ReplicaState reportedState) {
+ block = new Block(block);
+ getBlockQueue(block).add(
+ new ReportedBlockInfo(dn, block, reportedState));
+ count++;
+ }
+
+ /**
+ * @return any messages that were previously queued for the given block,
+ * or null if no messages were queued.
+ */
+ Queue takeBlockQueue(Block block) {
+ Queue queue = queueByBlockId.remove(block);
+ if (queue != null) {
+ count -= queue.size();
+ }
+ return queue;
+ }
+
+
+ private Queue getBlockQueue(Block block) {
+ Queue queue = queueByBlockId.get(block);
+ if (queue == null) {
+ queue = Lists.newLinkedList();
+ queueByBlockId.put(block, queue);
+ }
+ return queue;
+ }
+
+ public int count() {
+ return count ;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ for (Map.Entry> entry :
+ queueByBlockId.entrySet()) {
+ sb.append("Block " + entry.getKey() + ":\n");
+ for (ReportedBlockInfo rbi : entry.getValue()) {
+ sb.append(" ").append(rbi).append("\n");
+ }
+ }
+ return sb.toString();
+ }
+
+ public Iterable takeAll() {
+ List rbis = Lists.newArrayListWithCapacity(
+ count);
+ for (Queue q : queueByBlockId.values()) {
+ rbis.addAll(q);
+ }
+ queueByBlockId.clear();
+ count = 0;
+ return rbis;
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
index e07cf9bb2a..e200ed0dea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
@@ -104,6 +104,14 @@ class PendingReplicationBlocks {
}
}
+
+ public void clear() {
+ synchronized (pendingReplications) {
+ pendingReplications.clear();
+ timedOutItems.clear();
+ }
+ }
+
/**
* The total number of blocks that are undergoing replication
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index 3de906701f..c76d24c6a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -568,7 +568,7 @@ public abstract class Storage extends StorageInfo {
*
Locking is not supported by all file systems.
* E.g., NFS does not consistently support exclusive locks.
*
- *
If locking is supported we guarantee exculsive access to the
+ *
If locking is supported we guarantee exclusive access to the
* storage directory. Otherwise, no guarantee is given.
*
* @throws IOException if locking fails
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java
index 642551e379..1f4e974166 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java
@@ -23,6 +23,7 @@ import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -97,9 +98,9 @@ public final class Util {
* @param names collection of strings to convert to URIs
* @return collection of URIs
*/
- public static Collection stringCollectionAsURIs(
+ public static List stringCollectionAsURIs(
Collection names) {
- Collection uris = new ArrayList(names.size());
+ List uris = new ArrayList(names.size());
for(String name : names) {
try {
uris.add(stringAsURI(name));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index 5b1ed7c5a5..27567b543f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -17,20 +17,16 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
-
import java.io.IOException;
import java.net.InetSocketAddress;
-import java.net.SocketTimeoutException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.LinkedList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -50,8 +46,11 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
+import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -61,23 +60,22 @@ import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
/**
- * A thread per namenode to perform:
- *
- *
Pre-registration handshake with namenode
- *
Registration with namenode
- *
Send periodic heartbeats to the namenode
- *
Handle commands received from the namenode
- *
+ * One instance per block-pool/namespace on the DN, which handles the
+ * heartbeats to the active and standby NNs for that namespace.
+ * This class manages an instance of {@link BPServiceActor} for each NN,
+ * and delegates calls to both NNs.
+ * It also maintains the state about which of the NNs is considered active.
*/
@InterfaceAudience.Private
-class BPOfferService implements Runnable {
+class BPOfferService {
static final Log LOG = DataNode.LOG;
- final InetSocketAddress nnAddr;
-
/**
* Information about the namespace that this service
* is registering with. This is assigned after
@@ -92,43 +90,80 @@ class BPOfferService implements Runnable {
*/
DatanodeRegistration bpRegistration;
- long lastBlockReport = 0;
- long lastDeletedReport = 0;
-
- boolean resetBlockReportTime = true;
-
- Thread bpThread;
- DatanodeProtocolClientSideTranslatorPB bpNamenode;
- private long lastHeartbeat = 0;
- private volatile boolean initialized = false;
- private final LinkedList receivedAndDeletedBlockList
- = new LinkedList();
- private volatile int pendingReceivedRequests = 0;
- private volatile boolean shouldServiceRun = true;
UpgradeManagerDatanode upgradeManager = null;
private final DataNode dn;
- private final DNConf dnConf;
- BPOfferService(InetSocketAddress nnAddr, DataNode dn) {
+ /**
+ * A reference to the BPServiceActor associated with the currently
+ * ACTIVE NN. In the case that all NameNodes are in STANDBY mode,
+ * this can be null. If non-null, this must always refer to a member
+ * of the {@link #bpServices} list.
+ */
+ private BPServiceActor bpServiceToActive = null;
+
+ /**
+ * The list of all actors for namenodes in this nameservice, regardless
+ * of their active or standby states.
+ */
+ private List bpServices =
+ new CopyOnWriteArrayList();
+
+ /**
+ * Each time we receive a heartbeat from a NN claiming to be ACTIVE,
+ * we record that NN's most recent transaction ID here, so long as it
+ * is more recent than the previous value. This allows us to detect
+ * split-brain scenarios in which a prior NN is still asserting its
+ * ACTIVE state but with a too-low transaction ID. See HDFS-2627
+ * for details.
+ */
+ private long lastActiveClaimTxId = -1;
+
+ BPOfferService(List nnAddrs, DataNode dn) {
+ Preconditions.checkArgument(!nnAddrs.isEmpty(),
+ "Must pass at least one NN.");
this.dn = dn;
- this.nnAddr = nnAddr;
- this.dnConf = dn.getDnConf();
+
+ for (InetSocketAddress addr : nnAddrs) {
+ this.bpServices.add(new BPServiceActor(addr, this));
+ }
+ }
+
+ void refreshNNList(ArrayList addrs) throws IOException {
+ Set oldAddrs = Sets.newHashSet();
+ for (BPServiceActor actor : bpServices) {
+ oldAddrs.add(actor.getNNSocketAddress());
+ }
+ Set newAddrs = Sets.newHashSet(addrs);
+
+ if (!Sets.symmetricDifference(oldAddrs, newAddrs).isEmpty()) {
+ // Keep things simple for now -- we can implement this at a later date.
+ throw new IOException(
+ "HA does not currently support adding a new standby to a running DN. " +
+ "Please do a rolling restart of DNs to reconfigure the list of NNs.");
+ }
}
/**
- * returns true if BP thread has completed initialization of storage
- * and has registered with the corresponding namenode
- * @return true if initialized
+ * @return true if the service has registered with at least one NameNode.
*/
- public boolean isInitialized() {
- return initialized;
+ boolean isInitialized() {
+ return bpRegistration != null;
}
- public boolean isAlive() {
- return shouldServiceRun && bpThread.isAlive();
+ /**
+ * @return true if there is at least one actor thread running which is
+ * talking to a NameNode.
+ */
+ boolean isAlive() {
+ for (BPServiceActor actor : bpServices) {
+ if (actor.isAlive()) {
+ return true;
+ }
+ }
+ return false;
}
- public String getBlockPoolId() {
+ String getBlockPoolId() {
if (bpNSInfo != null) {
return bpNSInfo.getBlockPoolID();
} else {
@@ -138,10 +173,11 @@ class BPOfferService implements Runnable {
}
}
- public NamespaceInfo getNamespaceInfo() {
+ synchronized NamespaceInfo getNamespaceInfo() {
return bpNSInfo;
}
+ @Override
public String toString() {
if (bpNSInfo == null) {
// If we haven't yet connected to our NN, we don't yet know our
@@ -153,522 +189,363 @@ class BPOfferService implements Runnable {
storageId = "unknown";
}
return "Block pool (storage id " + storageId +
- ") connecting to " + nnAddr;
+ ")";
} else {
return "Block pool " + getBlockPoolId() +
" (storage id " + dn.getStorageId() +
- ") registered with " + nnAddr;
+ ")";
}
}
- InetSocketAddress getNNSocketAddress() {
- return nnAddr;
- }
-
- /**
- * Used to inject a spy NN in the unit tests.
- */
- @VisibleForTesting
- void setNameNode(DatanodeProtocolClientSideTranslatorPB dnProtocol) {
- bpNamenode = dnProtocol;
- }
-
- /**
- * Perform the first part of the handshake with the NameNode.
- * This calls versionRequest to determine the NN's
- * namespace and version info. It automatically retries until
- * the NN responds or the DN is shutting down.
- *
- * @return the NamespaceInfo
- * @throws IncorrectVersionException if the remote NN does not match
- * this DN's version
- */
- NamespaceInfo retrieveNamespaceInfo() throws IncorrectVersionException {
- NamespaceInfo nsInfo = null;
- while (shouldRun()) {
- try {
- nsInfo = bpNamenode.versionRequest();
- LOG.debug(this + " received versionRequest response: " + nsInfo);
- break;
- } catch(SocketTimeoutException e) { // namenode is busy
- LOG.warn("Problem connecting to server: " + nnAddr);
- } catch(IOException e ) { // namenode is not available
- LOG.warn("Problem connecting to server: " + nnAddr);
- }
-
- // try again in a second
- sleepAndLogInterrupts(5000, "requesting version info from NN");
- }
-
- if (nsInfo != null) {
- checkNNVersion(nsInfo);
- }
- return nsInfo;
- }
-
- private void checkNNVersion(NamespaceInfo nsInfo)
- throws IncorrectVersionException {
- // build and layout versions should match
- String nsBuildVer = nsInfo.getBuildVersion();
- String stBuildVer = Storage.getBuildVersion();
- if (!nsBuildVer.equals(stBuildVer)) {
- LOG.warn("Data-node and name-node Build versions must be the same. " +
- "Namenode build version: " + nsBuildVer + "Datanode " +
- "build version: " + stBuildVer);
- throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
- }
-
- if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
- LOG.warn("Data-node and name-node layout versions must be the same." +
- " Expected: "+ HdfsConstants.LAYOUT_VERSION +
- " actual "+ bpNSInfo.getLayoutVersion());
- throw new IncorrectVersionException(
- bpNSInfo.getLayoutVersion(), "namenode");
- }
- }
-
- private void connectToNNAndHandshake() throws IOException {
- // get NN proxy
- bpNamenode = dn.connectToNN(nnAddr);
-
- // First phase of the handshake with NN - get the namespace
- // info.
- bpNSInfo = retrieveNamespaceInfo();
-
- // Now that we know the namespace ID, etc, we can pass this to the DN.
- // The DN can now initialize its local storage if we are the
- // first BP to handshake, etc.
- dn.initBlockPool(this);
-
- // Second phase of the handshake with the NN.
- register();
- }
-
- /**
- * This methods arranges for the data node to send the block report at
- * the next heartbeat.
- */
- void scheduleBlockReport(long delay) {
- if (delay > 0) { // send BR after random delay
- lastBlockReport = System.currentTimeMillis()
- - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
- } else { // send at next heartbeat
- lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
- }
- resetBlockReportTime = true; // reset future BRs for randomness
- }
-
void reportBadBlocks(ExtendedBlock block) {
- DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
- LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) };
-
- try {
- bpNamenode.reportBadBlocks(blocks);
- } catch (IOException e){
- /* One common reason is that NameNode could be in safe mode.
- * Should we keep on retrying in that case?
- */
- LOG.warn("Failed to report bad block " + block + " to namenode : "
- + " Exception", e);
+ checkBlock(block);
+ for (BPServiceActor actor : bpServices) {
+ actor.reportBadBlocks(block);
}
-
}
- /**
- * Report received blocks and delete hints to the Namenode
- *
- * @throws IOException
- */
- private void reportReceivedDeletedBlocks() throws IOException {
-
- // check if there are newly received blocks
- ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
- int currentReceivedRequestsCounter;
- synchronized (receivedAndDeletedBlockList) {
- currentReceivedRequestsCounter = pendingReceivedRequests;
- int numBlocks = receivedAndDeletedBlockList.size();
- if (numBlocks > 0) {
- //
- // Send newly-received and deleted blockids to namenode
- //
- receivedAndDeletedBlockArray = receivedAndDeletedBlockList
- .toArray(new ReceivedDeletedBlockInfo[numBlocks]);
- }
- }
- if (receivedAndDeletedBlockArray != null) {
- StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
- bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
- bpNamenode.blockReceivedAndDeleted(bpRegistration, getBlockPoolId(),
- report);
- synchronized (receivedAndDeletedBlockList) {
- for (int i = 0; i < receivedAndDeletedBlockArray.length; i++) {
- receivedAndDeletedBlockList.remove(receivedAndDeletedBlockArray[i]);
- }
- pendingReceivedRequests -= currentReceivedRequestsCounter;
- }
- }
- }
-
/*
* Informing the name node could take a long long time! Should we wait
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
- if (block == null || delHint == null) {
- throw new IllegalArgumentException(block == null ? "Block is null"
- : "delHint is null");
- }
+ checkBlock(block);
+ checkDelHint(delHint);
+ ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
+ block.getLocalBlock(),
+ ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK,
+ delHint);
- if (!block.getBlockPoolId().equals(getBlockPoolId())) {
- LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
- + getBlockPoolId());
- return;
+ for (BPServiceActor actor : bpServices) {
+ actor.notifyNamenodeBlockImmediately(bInfo);
}
+ }
- synchronized (receivedAndDeletedBlockList) {
- receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
- .getLocalBlock(), delHint));
- pendingReceivedRequests++;
- receivedAndDeletedBlockList.notifyAll();
- }
+ private void checkBlock(ExtendedBlock block) {
+ Preconditions.checkArgument(block != null,
+ "block is null");
+ Preconditions.checkArgument(block.getBlockPoolId().equals(getBlockPoolId()),
+ "block belongs to BP %s instead of BP %s",
+ block.getBlockPoolId(), getBlockPoolId());
+ }
+
+ private void checkDelHint(String delHint) {
+ Preconditions.checkArgument(delHint != null,
+ "delHint is null");
}
void notifyNamenodeDeletedBlock(ExtendedBlock block) {
- if (block == null) {
- throw new IllegalArgumentException("Block is null");
+ checkBlock(block);
+ ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
+ block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null);
+
+ for (BPServiceActor actor : bpServices) {
+ actor.notifyNamenodeDeletedBlock(bInfo);
}
-
- if (!block.getBlockPoolId().equals(getBlockPoolId())) {
- LOG.warn("BlockPool mismatch " + block.getBlockPoolId() + " vs. "
- + getBlockPoolId());
- return;
- }
-
- synchronized (receivedAndDeletedBlockList) {
- receivedAndDeletedBlockList.add(new ReceivedDeletedBlockInfo(block
- .getLocalBlock(), ReceivedDeletedBlockInfo.TODELETE_HINT));
+ }
+
+ void notifyNamenodeReceivingBlock(ExtendedBlock block) {
+ checkBlock(block);
+ ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
+ block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null);
+
+ for (BPServiceActor actor : bpServices) {
+ actor.notifyNamenodeBlockImmediately(bInfo);
}
}
-
- /**
- * Report the list blocks to the Namenode
- * @throws IOException
- */
- DatanodeCommand blockReport() throws IOException {
- // send block report if timer has expired.
- DatanodeCommand cmd = null;
- long startTime = now();
- if (startTime - lastBlockReport > dnConf.blockReportInterval) {
-
- // Create block report
- long brCreateStartTime = now();
- BlockListAsLongs bReport = dn.data.getBlockReport(getBlockPoolId());
-
- // Send block report
- long brSendStartTime = now();
- StorageBlockReport[] report = { new StorageBlockReport(
- bpRegistration.getStorageID(), bReport.getBlockListAsLongs()) };
- cmd = bpNamenode.blockReport(bpRegistration, getBlockPoolId(), report);
-
- // Log the block report processing stats from Datanode perspective
- long brSendCost = now() - brSendStartTime;
- long brCreateCost = brSendStartTime - brCreateStartTime;
- dn.metrics.addBlockReport(brSendCost);
- LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
- + " blocks took " + brCreateCost + " msec to generate and "
- + brSendCost + " msecs for RPC and NN processing");
-
- // If we have sent the first block report, then wait a random
- // time before we start the periodic block reports.
- if (resetBlockReportTime) {
- lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dnConf.blockReportInterval));
- resetBlockReportTime = false;
- } else {
- /* say the last block report was at 8:20:14. The current report
- * should have started around 9:20:14 (default 1 hour interval).
- * If current time is :
- * 1) normal like 9:20:18, next report should be at 10:20:14
- * 2) unexpected like 11:35:43, next report should be at 12:20:14
- */
- lastBlockReport += (now() - lastBlockReport) /
- dnConf.blockReportInterval * dnConf.blockReportInterval;
- }
- LOG.info("sent block report, processed command:" + cmd);
- }
- return cmd;
- }
-
-
- DatanodeCommand [] sendHeartBeat() throws IOException {
- // reports number of failed volumes
- StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
- false, dn.data.getCapacity(), dn.data.getDfsUsed(),
- dn.data.getRemaining(), dn.data.getBlockPoolUsed(getBlockPoolId())) };
- return bpNamenode.sendHeartbeat(bpRegistration, report,
- dn.xmitsInProgress.get(),
- dn.getXceiverCount(), dn.data.getNumFailedVolumes());
- }
-
//This must be called only by blockPoolManager
void start() {
- if ((bpThread != null) && (bpThread.isAlive())) {
- //Thread is started already
- return;
+ for (BPServiceActor actor : bpServices) {
+ actor.start();
}
- bpThread = new Thread(this, formatThreadName());
- bpThread.setDaemon(true); // needed for JUnit testing
- bpThread.start();
- }
-
- private String formatThreadName() {
- Collection dataDirs = DataNode.getStorageDirs(dn.getConf());
- return "DataNode: [" +
- StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
- " heartbeating to " + nnAddr;
}
//This must be called only by blockPoolManager.
void stop() {
- shouldServiceRun = false;
- if (bpThread != null) {
- bpThread.interrupt();
+ for (BPServiceActor actor : bpServices) {
+ actor.stop();
}
}
//This must be called only by blockPoolManager
void join() {
- try {
- if (bpThread != null) {
- bpThread.join();
- }
- } catch (InterruptedException ie) { }
+ for (BPServiceActor actor : bpServices) {
+ actor.join();
+ }
+ }
+
+ synchronized UpgradeManagerDatanode getUpgradeManager() {
+ if(upgradeManager == null)
+ upgradeManager =
+ new UpgradeManagerDatanode(dn, getBlockPoolId());
+
+ return upgradeManager;
}
- //Cleanup method to be called by current thread before exiting.
- private synchronized void cleanUp() {
-
- if(upgradeManager != null)
- upgradeManager.shutdownUpgrade();
- shouldServiceRun = false;
- IOUtils.cleanup(LOG, bpNamenode);
- dn.shutdownBlockPool(this);
+ void processDistributedUpgradeCommand(UpgradeCommand comm)
+ throws IOException {
+ UpgradeManagerDatanode upgradeManager = getUpgradeManager();
+ upgradeManager.processUpgradeCommand(comm);
}
/**
- * Main loop for each BP thread. Run until shutdown,
- * forever calling remote NameNode functions.
+ * Start distributed upgrade if it should be initiated by the data-node.
*/
- private void offerService() throws Exception {
- LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
- + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
- + dnConf.blockReportInterval + "msec" + " Initial delay: "
- + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
- + dnConf.heartBeatInterval);
-
- //
- // Now loop for a long time....
- //
- while (shouldRun()) {
- try {
- long startTime = now();
-
- //
- // Every so often, send heartbeat or block-report
- //
- if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
- //
- // All heartbeat messages include following info:
- // -- Datanode name
- // -- data transfer port
- // -- Total capacity
- // -- Bytes remaining
- //
- lastHeartbeat = startTime;
- if (!dn.areHeartbeatsDisabledForTests()) {
- DatanodeCommand[] cmds = sendHeartBeat();
- dn.metrics.addHeartbeat(now() - startTime);
-
- long startProcessCommands = now();
- if (!processCommand(cmds))
- continue;
- long endProcessCommands = now();
- if (endProcessCommands - startProcessCommands > 2000) {
- LOG.info("Took " + (endProcessCommands - startProcessCommands) +
- "ms to process " + cmds.length + " commands from NN");
- }
- }
- }
- if (pendingReceivedRequests > 0
- || (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
- reportReceivedDeletedBlocks();
- lastDeletedReport = startTime;
- }
-
- DatanodeCommand cmd = blockReport();
- processCommand(cmd);
-
- // Now safe to start scanning the block pool
- if (dn.blockScanner != null) {
- dn.blockScanner.addBlockPool(this.getBlockPoolId());
- }
-
- //
- // There is no work to do; sleep until hearbeat timer elapses,
- // or work arrives, and then iterate again.
- //
- long waitTime = dnConf.heartBeatInterval -
- (System.currentTimeMillis() - lastHeartbeat);
- synchronized(receivedAndDeletedBlockList) {
- if (waitTime > 0 && pendingReceivedRequests == 0) {
- try {
- receivedAndDeletedBlockList.wait(waitTime);
- } catch (InterruptedException ie) {
- LOG.warn("BPOfferService for " + this + " interrupted");
- }
- }
- } // synchronized
- } catch(RemoteException re) {
- String reClass = re.getClassName();
- if (UnregisteredNodeException.class.getName().equals(reClass) ||
- DisallowedDatanodeException.class.getName().equals(reClass) ||
- IncorrectVersionException.class.getName().equals(reClass)) {
- LOG.warn(this + " is shutting down", re);
- shouldServiceRun = false;
- return;
- }
- LOG.warn("RemoteException in offerService", re);
- try {
- long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
- Thread.sleep(sleepTime);
- } catch (InterruptedException ie) {
- Thread.currentThread().interrupt();
- }
- } catch (IOException e) {
- LOG.warn("IOException in offerService", e);
- }
- } // while (shouldRun())
- } // offerService
-
- /**
- * Register one bp with the corresponding NameNode
- *
- * The bpDatanode needs to register with the namenode on startup in order
- * 1) to report which storage it is serving now and
- * 2) to receive a registrationID
- *
- * issued by the namenode to recognize registered datanodes.
- *
- * @see FSNamesystem#registerDatanode(DatanodeRegistration)
- * @throws IOException
- */
- void register() throws IOException {
- Preconditions.checkState(bpNSInfo != null,
- "register() should be called after handshake()");
+ synchronized void startDistributedUpgradeIfNeeded() throws IOException {
+ UpgradeManagerDatanode um = getUpgradeManager();
- // The handshake() phase loaded the block pool storage
- // off disk - so update the bpRegistration object from that info
- bpRegistration = dn.createBPRegistration(bpNSInfo);
-
- LOG.info(this + " beginning handshake with NN");
-
- while (shouldRun()) {
- try {
- // Use returned registration from namenode with updated machine name.
- bpRegistration = bpNamenode.registerDatanode(bpRegistration,
- new DatanodeStorage[0]);
- break;
- } catch(SocketTimeoutException e) { // namenode is busy
- LOG.info("Problem connecting to server: " + nnAddr);
- sleepAndLogInterrupts(1000, "connecting to server");
- }
- }
-
- LOG.info("Block pool " + this + " successfully registered with NN");
- dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
-
- // random short delay - helps scatter the BR from all DNs
- scheduleBlockReport(dnConf.initialBlockReportDelay);
+ if(!um.getUpgradeState())
+ return;
+ um.setUpgradeState(false, um.getUpgradeVersion());
+ um.startUpgrade();
+ return;
}
-
-
- private void sleepAndLogInterrupts(int millis,
- String stateString) {
- try {
- Thread.sleep(millis);
- } catch (InterruptedException ie) {
- LOG.info("BPOfferService " + this +
- " interrupted while " + stateString);
- }
+
+ DataNode getDataNode() {
+ return dn;
}
/**
- * No matter what kind of exception we get, keep retrying to offerService().
- * That's the loop that connects to the NameNode and provides basic DataNode
- * functionality.
- *
- * Only stop when "shouldRun" or "shouldServiceRun" is turned off, which can
- * happen either at shutdown or due to refreshNamenodes.
+ * Called by the BPServiceActors when they handshake to a NN.
+ * If this is the first NN connection, this sets the namespace info
+ * for this BPOfferService. If it's a connection to a new NN, it
+ * verifies that this namespace matches (eg to prevent a misconfiguration
+ * where a StandbyNode from a different cluster is specified)
*/
- @Override
- public void run() {
- LOG.info(this + " starting to offer service");
-
- try {
- // init stuff
- try {
- // setup storage
- connectToNNAndHandshake();
- } catch (IOException ioe) {
- // Initial handshake, storage recovery or registration failed
- // End BPOfferService thread
- LOG.fatal("Initialization failed for block pool " + this, ioe);
- return;
- }
-
- initialized = true; // bp is initialized;
+ synchronized void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
+ if (this.bpNSInfo == null) {
+ this.bpNSInfo = nsInfo;
- while (shouldRun()) {
- try {
- startDistributedUpgradeIfNeeded();
- offerService();
- } catch (Exception ex) {
- LOG.error("Exception in BPOfferService for " + this, ex);
- sleepAndLogInterrupts(5000, "offering service");
- }
- }
- } catch (Throwable ex) {
- LOG.warn("Unexpected exception in block pool " + this, ex);
- } finally {
- LOG.warn("Ending block pool service for: " + this);
- cleanUp();
+ // Now that we know the namespace ID, etc, we can pass this to the DN.
+ // The DN can now initialize its local storage if we are the
+ // first BP to handshake, etc.
+ dn.initBlockPool(this);
+ return;
+ } else {
+ checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
+ "Blockpool ID");
+ checkNSEquality(bpNSInfo.getNamespaceID(), nsInfo.getNamespaceID(),
+ "Namespace ID");
+ checkNSEquality(bpNSInfo.getClusterID(), nsInfo.getClusterID(),
+ "Cluster ID");
}
}
- private boolean shouldRun() {
- return shouldServiceRun && dn.shouldRun();
- }
-
/**
- * Process an array of datanode commands
- *
- * @param cmds an array of datanode commands
- * @return true if further processing may be required or false otherwise.
+ * After one of the BPServiceActors registers successfully with the
+ * NN, it calls this function to verify that the NN it connected to
+ * is consistent with other NNs serving the block-pool.
*/
- private boolean processCommand(DatanodeCommand[] cmds) {
- if (cmds != null) {
- for (DatanodeCommand cmd : cmds) {
- try {
- if (processCommand(cmd) == false) {
- return false;
- }
- } catch (IOException ioe) {
- LOG.warn("Error processing datanode Command", ioe);
- }
+ void registrationSucceeded(BPServiceActor bpServiceActor,
+ DatanodeRegistration reg) throws IOException {
+ if (bpRegistration != null) {
+ checkNSEquality(bpRegistration.storageInfo.getNamespaceID(),
+ reg.storageInfo.getNamespaceID(), "namespace ID");
+ checkNSEquality(bpRegistration.storageInfo.getClusterID(),
+ reg.storageInfo.getClusterID(), "cluster ID");
+ } else {
+ bpRegistration = reg;
+ }
+
+ dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
+ }
+
+ /**
+ * Verify equality of two namespace-related fields, throwing
+ * an exception if they are unequal.
+ */
+ private static void checkNSEquality(
+ Object ourID, Object theirID,
+ String idHelpText) throws IOException {
+ if (!ourID.equals(theirID)) {
+ throw new IOException(idHelpText + " mismatch: " +
+ "previously connected to " + idHelpText + " " + ourID +
+ " but now connected to " + idHelpText + " " + theirID);
+ }
+ }
+
+ synchronized DatanodeRegistration createRegistration() {
+ Preconditions.checkState(bpNSInfo != null,
+ "getRegistration() can only be called after initial handshake");
+ return dn.createBPRegistration(bpNSInfo);
+ }
+
+ /**
+ * Called when an actor shuts down. If this is the last actor
+ * to shut down, shuts down the whole blockpool in the DN.
+ */
+ synchronized void shutdownActor(BPServiceActor actor) {
+ if (bpServiceToActive == actor) {
+ bpServiceToActive = null;
+ }
+
+ bpServices.remove(actor);
+
+ if (bpServices.isEmpty()) {
+ dn.shutdownBlockPool(this);
+
+ if(upgradeManager != null)
+ upgradeManager.shutdownUpgrade();
+ }
+ }
+
+ /**
+ * Called by the DN to report an error to the NNs.
+ */
+ void trySendErrorReport(int errCode, String errMsg) {
+ for (BPServiceActor actor : bpServices) {
+ actor.trySendErrorReport(errCode, errMsg);
+ }
+ }
+
+ /**
+ * Ask each of the actors to schedule a block report after
+ * the specified delay.
+ */
+ void scheduleBlockReport(long delay) {
+ for (BPServiceActor actor : bpServices) {
+ actor.scheduleBlockReport(delay);
+ }
+ }
+
+ /**
+ * Ask each of the actors to report a bad block hosted on another DN.
+ */
+ void reportRemoteBadBlock(DatanodeInfo dnInfo, ExtendedBlock block) {
+ for (BPServiceActor actor : bpServices) {
+ try {
+ actor.reportRemoteBadBlock(dnInfo, block);
+ } catch (IOException e) {
+ LOG.warn("Couldn't report bad block " + block + " to " + actor,
+ e);
}
}
- return true;
+ }
+
+ /**
+ * @return a proxy to the active NN, or null if the BPOS has not
+ * acknowledged any NN as active yet.
+ */
+ synchronized DatanodeProtocolClientSideTranslatorPB getActiveNN() {
+ if (bpServiceToActive != null) {
+ return bpServiceToActive.bpNamenode;
+ } else {
+ return null;
+ }
+ }
+
+ @VisibleForTesting
+ synchronized List getBPServiceActors() {
+ return Lists.newArrayList(bpServices);
+ }
+
+ /**
+ * Update the BPOS's view of which NN is active, based on a heartbeat
+ * response from one of the actors.
+ *
+ * @param actor the actor which received the heartbeat
+ * @param nnHaState the HA-related heartbeat contents
+ */
+ synchronized void updateActorStatesFromHeartbeat(
+ BPServiceActor actor,
+ NNHAStatusHeartbeat nnHaState) {
+ final long txid = nnHaState.getTxId();
+
+ final boolean nnClaimsActive =
+ nnHaState.getState() == NNHAStatusHeartbeat.State.ACTIVE;
+ final boolean bposThinksActive = bpServiceToActive == actor;
+ final boolean isMoreRecentClaim = txid > lastActiveClaimTxId;
+
+ if (nnClaimsActive && !bposThinksActive) {
+ LOG.info("Namenode " + actor + " trying to claim ACTIVE state with " +
+ "txid=" + txid);
+ if (!isMoreRecentClaim) {
+ // Split-brain scenario - an NN is trying to claim active
+ // state when a different NN has already claimed it with a higher
+ // txid.
+ LOG.warn("NN " + actor + " tried to claim ACTIVE state at txid=" +
+ txid + " but there was already a more recent claim at txid=" +
+ lastActiveClaimTxId);
+ return;
+ } else {
+ if (bpServiceToActive == null) {
+ LOG.info("Acknowledging ACTIVE Namenode " + actor);
+ } else {
+ LOG.info("Namenode " + actor + " taking over ACTIVE state from " +
+ bpServiceToActive + " at higher txid=" + txid);
+ }
+ bpServiceToActive = actor;
+ }
+ } else if (!nnClaimsActive && bposThinksActive) {
+ LOG.info("Namenode " + actor + " relinquishing ACTIVE state with " +
+ "txid=" + nnHaState.getTxId());
+ bpServiceToActive = null;
+ }
+
+ if (bpServiceToActive == actor) {
+ assert txid >= lastActiveClaimTxId;
+ lastActiveClaimTxId = txid;
+ }
+ }
+
+ /**
+ * @return true if the given NN address is one of the NNs for this
+ * block pool
+ */
+ boolean containsNN(InetSocketAddress addr) {
+ for (BPServiceActor actor : bpServices) {
+ if (actor.getNNSocketAddress().equals(addr)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @VisibleForTesting
+ int countNameNodes() {
+ return bpServices.size();
+ }
+
+ /**
+ * Run an immediate block report on this thread. Used by tests.
+ */
+ @VisibleForTesting
+ void triggerBlockReportForTests() throws IOException {
+ for (BPServiceActor actor : bpServices) {
+ actor.triggerBlockReportForTests();
+ }
+ }
+
+ /**
+ * Run an immediate deletion report on this thread. Used by tests.
+ */
+ @VisibleForTesting
+ void triggerDeletionReportForTests() throws IOException {
+ for (BPServiceActor actor : bpServices) {
+ actor.triggerDeletionReportForTests();
+ }
+ }
+
+ /**
+ * Run an immediate heartbeat from all actors. Used by tests.
+ */
+ @VisibleForTesting
+ void triggerHeartbeatForTests() throws IOException {
+ for (BPServiceActor actor : bpServices) {
+ actor.triggerHeartbeatForTests();
+ }
+ }
+
+ synchronized boolean processCommandFromActor(DatanodeCommand cmd,
+ BPServiceActor actor) throws IOException {
+ assert bpServices.contains(actor);
+ if (actor == bpServiceToActive) {
+ return processCommandFromActive(cmd, actor);
+ } else {
+ return processCommandFromStandby(cmd, actor);
+ }
}
/**
@@ -677,7 +554,8 @@ class BPOfferService implements Runnable {
* @return true if further processing may be required or false otherwise.
* @throws IOException
*/
- private boolean processCommand(DatanodeCommand cmd) throws IOException {
+ private boolean processCommandFromActive(DatanodeCommand cmd,
+ BPServiceActor actor) throws IOException {
if (cmd == null)
return true;
final BlockCommand bcmd =
@@ -708,19 +586,13 @@ class BPOfferService implements Runnable {
dn.metrics.incrBlocksRemoved(toDelete.length);
break;
case DatanodeProtocol.DNA_SHUTDOWN:
- // shut down the data node
- shouldServiceRun = false;
- return false;
+ // TODO: DNA_SHUTDOWN appears to be unused - the NN never sends this command
+ // See HDFS-2987.
+ throw new UnsupportedOperationException("Received unimplemented DNA_SHUTDOWN");
case DatanodeProtocol.DNA_REGISTER:
// namenode requested a registration - at start or if NN lost contact
LOG.info("DatanodeCommand action: DNA_REGISTER");
- if (shouldRun()) {
- // re-retrieve namespace info to make sure that, if the NN
- // was restarted, we still match its version (HDFS-2120)
- retrieveNamespaceInfo();
- // and re-register
- register();
- }
+ actor.reRegister();
break;
case DatanodeProtocol.DNA_FINALIZE:
String bp = ((FinalizeCommand) cmd).getBlockPoolId();
@@ -740,7 +612,8 @@ class BPOfferService implements Runnable {
case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
if (dn.isBlockTokenEnabled) {
- dn.blockPoolTokenSecretManager.setKeys(getBlockPoolId(),
+ dn.blockPoolTokenSecretManager.setKeys(
+ getBlockPoolId(),
((KeyUpdateCommand) cmd).getExportedKeys());
}
break;
@@ -759,41 +632,29 @@ class BPOfferService implements Runnable {
}
return true;
}
-
- private void processDistributedUpgradeCommand(UpgradeCommand comm)
- throws IOException {
- UpgradeManagerDatanode upgradeManager = getUpgradeManager();
- upgradeManager.processUpgradeCommand(comm);
+
+ private boolean processCommandFromStandby(DatanodeCommand cmd,
+ BPServiceActor actor) throws IOException {
+ if (cmd == null)
+ return true;
+ switch(cmd.getAction()) {
+ case DatanodeProtocol.DNA_REGISTER:
+ // namenode requested a registration - at start or if NN lost contact
+ LOG.info("DatanodeCommand action: DNA_REGISTER");
+ actor.reRegister();
+ return true;
+ case DatanodeProtocol.DNA_TRANSFER:
+ case DatanodeProtocol.DNA_INVALIDATE:
+ case DatanodeProtocol.DNA_SHUTDOWN:
+ case DatanodeProtocol.DNA_RECOVERBLOCK:
+ case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
+ case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
+ LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
+ return true;
+ default:
+ LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
+ }
+ return true;
}
- synchronized UpgradeManagerDatanode getUpgradeManager() {
- if(upgradeManager == null)
- upgradeManager =
- new UpgradeManagerDatanode(dn, getBlockPoolId());
-
- return upgradeManager;
- }
-
- /**
- * Start distributed upgrade if it should be initiated by the data-node.
- */
- private void startDistributedUpgradeIfNeeded() throws IOException {
- UpgradeManagerDatanode um = getUpgradeManager();
-
- if(!um.getUpgradeState())
- return;
- um.setUpgradeState(false, um.getUpgradeVersion());
- um.startUpgrade();
- return;
- }
-
- @VisibleForTesting
- DatanodeProtocolClientSideTranslatorPB getBpNamenode() {
- return bpNamenode;
- }
-
- @VisibleForTesting
- void setBpNamenode(DatanodeProtocolClientSideTranslatorPB bpNamenode) {
- this.bpNamenode = bpNamenode;
- }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
new file mode 100644
index 0000000000..75f32cbc04
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -0,0 +1,730 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
+import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+
+/**
+ * A thread per active or standby namenode to perform:
+ *
+ *
Pre-registration handshake with namenode
+ *
Registration with namenode
+ *
Send periodic heartbeats to the namenode
+ *
Handle commands received from the namenode
+ *
+ */
+@InterfaceAudience.Private
+class BPServiceActor implements Runnable {
+
+ static final Log LOG = DataNode.LOG;
+ final InetSocketAddress nnAddr;
+
+ BPOfferService bpos;
+
+ long lastBlockReport = 0;
+ long lastDeletedReport = 0;
+
+ boolean resetBlockReportTime = true;
+
+ Thread bpThread;
+ DatanodeProtocolClientSideTranslatorPB bpNamenode;
+ private long lastHeartbeat = 0;
+ private volatile boolean initialized = false;
+
+ /**
+ * Between block reports (which happen on the order of once an hour) the
+ * DN reports smaller incremental changes to its block list. This map,
+ * keyed by block ID, contains the pending changes which have yet to be
+ * reported to the NN. Access should be synchronized on this object.
+ */
+ private final Map pendingIncrementalBR
+ = Maps.newHashMap();
+
+ private volatile int pendingReceivedRequests = 0;
+ private volatile boolean shouldServiceRun = true;
+ private final DataNode dn;
+ private final DNConf dnConf;
+
+ private DatanodeRegistration bpRegistration;
+
+ BPServiceActor(InetSocketAddress nnAddr, BPOfferService bpos) {
+ this.bpos = bpos;
+ this.dn = bpos.getDataNode();
+ this.nnAddr = nnAddr;
+ this.dnConf = dn.getDnConf();
+ }
+
+ /**
+ * returns true if BP thread has completed initialization of storage
+ * and has registered with the corresponding namenode
+ * @return true if initialized
+ */
+ boolean isInitialized() {
+ return initialized;
+ }
+
+ boolean isAlive() {
+ return shouldServiceRun && bpThread.isAlive();
+ }
+
+ @Override
+ public String toString() {
+ return bpos.toString() + " service to " + nnAddr;
+ }
+
+ InetSocketAddress getNNSocketAddress() {
+ return nnAddr;
+ }
+
+ /**
+ * Used to inject a spy NN in the unit tests.
+ */
+ @VisibleForTesting
+ void setNameNode(DatanodeProtocolClientSideTranslatorPB dnProtocol) {
+ bpNamenode = dnProtocol;
+ }
+
+ @VisibleForTesting
+ DatanodeProtocolClientSideTranslatorPB getNameNodeProxy() {
+ return bpNamenode;
+ }
+
+ /**
+ * Perform the first part of the handshake with the NameNode.
+ * This calls versionRequest to determine the NN's
+ * namespace and version info. It automatically retries until
+ * the NN responds or the DN is shutting down.
+ *
+ * @return the NamespaceInfo
+ */
+ @VisibleForTesting
+ NamespaceInfo retrieveNamespaceInfo() throws IOException {
+ NamespaceInfo nsInfo = null;
+ while (shouldRun()) {
+ try {
+ nsInfo = bpNamenode.versionRequest();
+ LOG.debug(this + " received versionRequest response: " + nsInfo);
+ break;
+ } catch(SocketTimeoutException e) { // namenode is busy
+ LOG.warn("Problem connecting to server: " + nnAddr);
+ } catch(IOException e ) { // namenode is not available
+ LOG.warn("Problem connecting to server: " + nnAddr);
+ }
+
+ // try again in a second
+ sleepAndLogInterrupts(5000, "requesting version info from NN");
+ }
+
+ if (nsInfo != null) {
+ checkNNVersion(nsInfo);
+ } else {
+ throw new IOException("DN shut down before block pool connected");
+ }
+ return nsInfo;
+ }
+
+ private void checkNNVersion(NamespaceInfo nsInfo)
+ throws IncorrectVersionException {
+ // build and layout versions should match
+ String nsBuildVer = nsInfo.getBuildVersion();
+ String stBuildVer = Storage.getBuildVersion();
+ if (!nsBuildVer.equals(stBuildVer)) {
+ LOG.warn("Data-node and name-node Build versions must be the same. " +
+ "Namenode build version: " + nsBuildVer + "Datanode " +
+ "build version: " + stBuildVer);
+ throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
+ }
+
+ if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
+ LOG.warn("Data-node and name-node layout versions must be the same." +
+ " Expected: "+ HdfsConstants.LAYOUT_VERSION +
+ " actual "+ nsInfo.getLayoutVersion());
+ throw new IncorrectVersionException(
+ nsInfo.getLayoutVersion(), "namenode");
+ }
+ }
+
+ private void connectToNNAndHandshake() throws IOException {
+ // get NN proxy
+ bpNamenode = dn.connectToNN(nnAddr);
+
+ // First phase of the handshake with NN - get the namespace
+ // info.
+ NamespaceInfo nsInfo = retrieveNamespaceInfo();
+
+ // Verify that this matches the other NN in this HA pair.
+ // This also initializes our block pool in the DN if we are
+ // the first NN connection for this BP.
+ bpos.verifyAndSetNamespaceInfo(nsInfo);
+
+ // Second phase of the handshake with the NN.
+ register();
+ }
+
+ /**
+ * This methods arranges for the data node to send the block report at
+ * the next heartbeat.
+ */
+ void scheduleBlockReport(long delay) {
+ if (delay > 0) { // send BR after random delay
+ lastBlockReport = System.currentTimeMillis()
+ - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
+ } else { // send at next heartbeat
+ lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
+ }
+ resetBlockReportTime = true; // reset future BRs for randomness
+ }
+
+ void reportBadBlocks(ExtendedBlock block) {
+ DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
+ LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) };
+
+ try {
+ bpNamenode.reportBadBlocks(blocks);
+ } catch (IOException e){
+ /* One common reason is that NameNode could be in safe mode.
+ * Should we keep on retrying in that case?
+ */
+ LOG.warn("Failed to report bad block " + block + " to namenode : "
+ + " Exception", e);
+ }
+ }
+
+ /**
+ * Report received blocks and delete hints to the Namenode
+ *
+ * @throws IOException
+ */
+ private void reportReceivedDeletedBlocks() throws IOException {
+
+ // check if there are newly received blocks
+ ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
+ synchronized (pendingIncrementalBR) {
+ int numBlocks = pendingIncrementalBR.size();
+ if (numBlocks > 0) {
+ //
+ // Send newly-received and deleted blockids to namenode
+ //
+ receivedAndDeletedBlockArray = pendingIncrementalBR
+ .values().toArray(new ReceivedDeletedBlockInfo[numBlocks]);
+ }
+ pendingIncrementalBR.clear();
+ }
+ if (receivedAndDeletedBlockArray != null) {
+ StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
+ bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
+ boolean success = false;
+ try {
+ bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
+ report);
+ success = true;
+ } finally {
+ synchronized (pendingIncrementalBR) {
+ if (!success) {
+ // If we didn't succeed in sending the report, put all of the
+ // blocks back onto our queue, but only in the case where we didn't
+ // put something newer in the meantime.
+ for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) {
+ if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
+ pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
+ }
+ }
+ }
+ pendingReceivedRequests = pendingIncrementalBR.size();
+ }
+ }
+ }
+ }
+
+ /*
+ * Informing the name node could take a long long time! Should we wait
+ * till namenode is informed before responding with success to the
+ * client? For now we don't.
+ */
+ void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
+ synchronized (pendingIncrementalBR) {
+ pendingIncrementalBR.put(
+ bInfo.getBlock().getBlockId(), bInfo);
+ pendingReceivedRequests++;
+ pendingIncrementalBR.notifyAll();
+ }
+ }
+
+ void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
+ synchronized (pendingIncrementalBR) {
+ pendingIncrementalBR.put(
+ bInfo.getBlock().getBlockId(), bInfo);
+ }
+ }
+
+ /**
+ * Run an immediate block report on this thread. Used by tests.
+ */
+ @VisibleForTesting
+ void triggerBlockReportForTests() throws IOException {
+ synchronized (pendingIncrementalBR) {
+ lastBlockReport = 0;
+ lastHeartbeat = 0;
+ pendingIncrementalBR.notifyAll();
+ while (lastBlockReport == 0) {
+ try {
+ pendingIncrementalBR.wait(100);
+ } catch (InterruptedException e) {
+ return;
+ }
+ }
+ }
+ }
+
+ @VisibleForTesting
+ void triggerHeartbeatForTests() throws IOException {
+ synchronized (pendingIncrementalBR) {
+ lastHeartbeat = 0;
+ pendingIncrementalBR.notifyAll();
+ while (lastHeartbeat == 0) {
+ try {
+ pendingIncrementalBR.wait(100);
+ } catch (InterruptedException e) {
+ return;
+ }
+ }
+ }
+ }
+
+ @VisibleForTesting
+ void triggerDeletionReportForTests() throws IOException {
+ synchronized (pendingIncrementalBR) {
+ lastDeletedReport = 0;
+ pendingIncrementalBR.notifyAll();
+
+ while (lastDeletedReport == 0) {
+ try {
+ pendingIncrementalBR.wait(100);
+ } catch (InterruptedException e) {
+ return;
+ }
+ }
+ }
+ }
+
+ /**
+ * Report the list blocks to the Namenode
+ * @throws IOException
+ */
+ DatanodeCommand blockReport() throws IOException {
+ // send block report if timer has expired.
+ DatanodeCommand cmd = null;
+ long startTime = now();
+ if (startTime - lastBlockReport > dnConf.blockReportInterval) {
+
+ // Flush any block information that precedes the block report. Otherwise
+ // we have a chance that we will miss the delHint information
+ // or we will report an RBW replica after the BlockReport already reports
+ // a FINALIZED one.
+ reportReceivedDeletedBlocks();
+
+ // Create block report
+ long brCreateStartTime = now();
+ BlockListAsLongs bReport = dn.getFSDataset().getBlockReport(
+ bpos.getBlockPoolId());
+
+ // Send block report
+ long brSendStartTime = now();
+ StorageBlockReport[] report = { new StorageBlockReport(
+ bpRegistration.getStorageID(), bReport.getBlockListAsLongs()) };
+ cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report);
+
+ // Log the block report processing stats from Datanode perspective
+ long brSendCost = now() - brSendStartTime;
+ long brCreateCost = brSendStartTime - brCreateStartTime;
+ dn.getMetrics().addBlockReport(brSendCost);
+ LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
+ + " blocks took " + brCreateCost + " msec to generate and "
+ + brSendCost + " msecs for RPC and NN processing");
+
+ // If we have sent the first block report, then wait a random
+ // time before we start the periodic block reports.
+ if (resetBlockReportTime) {
+ lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dnConf.blockReportInterval));
+ resetBlockReportTime = false;
+ } else {
+ /* say the last block report was at 8:20:14. The current report
+ * should have started around 9:20:14 (default 1 hour interval).
+ * If current time is :
+ * 1) normal like 9:20:18, next report should be at 10:20:14
+ * 2) unexpected like 11:35:43, next report should be at 12:20:14
+ */
+ lastBlockReport += (now() - lastBlockReport) /
+ dnConf.blockReportInterval * dnConf.blockReportInterval;
+ }
+ LOG.info("sent block report, processed command:" + cmd);
+ }
+ return cmd;
+ }
+
+
+ HeartbeatResponse sendHeartBeat() throws IOException {
+ LOG.info("heartbeat: " + this);
+ // reports number of failed volumes
+ StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
+ false,
+ dn.getFSDataset().getCapacity(),
+ dn.getFSDataset().getDfsUsed(),
+ dn.getFSDataset().getRemaining(),
+ dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
+ return bpNamenode.sendHeartbeat(bpRegistration, report,
+ dn.getXmitsInProgress(),
+ dn.getXceiverCount(),
+ dn.getFSDataset().getNumFailedVolumes());
+ }
+
+ //This must be called only by BPOfferService
+ void start() {
+ if ((bpThread != null) && (bpThread.isAlive())) {
+ //Thread is started already
+ return;
+ }
+ bpThread = new Thread(this, formatThreadName());
+ bpThread.setDaemon(true); // needed for JUnit testing
+ bpThread.start();
+ }
+
+ private String formatThreadName() {
+ Collection dataDirs = DataNode.getStorageDirs(dn.getConf());
+ return "DataNode: [" +
+ StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
+ " heartbeating to " + nnAddr;
+ }
+
+ //This must be called only by blockPoolManager.
+ void stop() {
+ shouldServiceRun = false;
+ if (bpThread != null) {
+ bpThread.interrupt();
+ }
+ }
+
+ //This must be called only by blockPoolManager
+ void join() {
+ try {
+ if (bpThread != null) {
+ bpThread.join();
+ }
+ } catch (InterruptedException ie) { }
+ }
+
+ //Cleanup method to be called by current thread before exiting.
+ private synchronized void cleanUp() {
+
+ shouldServiceRun = false;
+ IOUtils.cleanup(LOG, bpNamenode);
+ bpos.shutdownActor(this);
+ }
+
+ /**
+ * Main loop for each BP thread. Run until shutdown,
+ * forever calling remote NameNode functions.
+ */
+ private void offerService() throws Exception {
+ LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
+ + dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
+ + dnConf.blockReportInterval + "msec" + " Initial delay: "
+ + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
+ + dnConf.heartBeatInterval);
+
+ //
+ // Now loop for a long time....
+ //
+ while (shouldRun()) {
+ try {
+ long startTime = now();
+
+ //
+ // Every so often, send heartbeat or block-report
+ //
+ if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
+ //
+ // All heartbeat messages include following info:
+ // -- Datanode name
+ // -- data transfer port
+ // -- Total capacity
+ // -- Bytes remaining
+ //
+ lastHeartbeat = startTime;
+ if (!dn.areHeartbeatsDisabledForTests()) {
+ HeartbeatResponse resp = sendHeartBeat();
+ assert resp != null;
+ dn.getMetrics().addHeartbeat(now() - startTime);
+
+ // If the state of this NN has changed (eg STANDBY->ACTIVE)
+ // then let the BPOfferService update itself.
+ //
+ // Important that this happens before processCommand below,
+ // since the first heartbeat to a new active might have commands
+ // that we should actually process.
+ bpos.updateActorStatesFromHeartbeat(
+ this, resp.getNameNodeHaState());
+
+ long startProcessCommands = now();
+ if (!processCommand(resp.getCommands()))
+ continue;
+ long endProcessCommands = now();
+ if (endProcessCommands - startProcessCommands > 2000) {
+ LOG.info("Took " + (endProcessCommands - startProcessCommands)
+ + "ms to process " + resp.getCommands().length
+ + " commands from NN");
+ }
+ }
+ }
+ if (pendingReceivedRequests > 0
+ || (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
+ reportReceivedDeletedBlocks();
+ lastDeletedReport = startTime;
+ }
+
+ DatanodeCommand cmd = blockReport();
+ processCommand(new DatanodeCommand[]{ cmd });
+
+ // Now safe to start scanning the block pool.
+ // If it has already been started, this is a no-op.
+ if (dn.blockScanner != null) {
+ dn.blockScanner.addBlockPool(bpos.getBlockPoolId());
+ }
+
+ //
+ // There is no work to do; sleep until hearbeat timer elapses,
+ // or work arrives, and then iterate again.
+ //
+ long waitTime = dnConf.heartBeatInterval -
+ (System.currentTimeMillis() - lastHeartbeat);
+ synchronized(pendingIncrementalBR) {
+ if (waitTime > 0 && pendingReceivedRequests == 0) {
+ try {
+ pendingIncrementalBR.wait(waitTime);
+ } catch (InterruptedException ie) {
+ LOG.warn("BPOfferService for " + this + " interrupted");
+ }
+ }
+ } // synchronized
+ } catch(RemoteException re) {
+ String reClass = re.getClassName();
+ if (UnregisteredNodeException.class.getName().equals(reClass) ||
+ DisallowedDatanodeException.class.getName().equals(reClass) ||
+ IncorrectVersionException.class.getName().equals(reClass)) {
+ LOG.warn(this + " is shutting down", re);
+ shouldServiceRun = false;
+ return;
+ }
+ LOG.warn("RemoteException in offerService", re);
+ try {
+ long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
+ Thread.sleep(sleepTime);
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ }
+ } catch (IOException e) {
+ LOG.warn("IOException in offerService", e);
+ }
+ } // while (shouldRun())
+ } // offerService
+
+ /**
+ * Register one bp with the corresponding NameNode
+ *
+ * The bpDatanode needs to register with the namenode on startup in order
+ * 1) to report which storage it is serving now and
+ * 2) to receive a registrationID
+ *
+ * issued by the namenode to recognize registered datanodes.
+ *
+ * @see FSNamesystem#registerDatanode(DatanodeRegistration)
+ * @throws IOException
+ */
+ void register() throws IOException {
+ // The handshake() phase loaded the block pool storage
+ // off disk - so update the bpRegistration object from that info
+ bpRegistration = bpos.createRegistration();
+
+ LOG.info(this + " beginning handshake with NN");
+
+ while (shouldRun()) {
+ try {
+ // Use returned registration from namenode with updated machine name.
+ bpRegistration = bpNamenode.registerDatanode(bpRegistration,
+ new DatanodeStorage[0]);
+ break;
+ } catch(SocketTimeoutException e) { // namenode is busy
+ LOG.info("Problem connecting to server: " + nnAddr);
+ sleepAndLogInterrupts(1000, "connecting to server");
+ }
+ }
+
+ LOG.info("Block pool " + this + " successfully registered with NN");
+ bpos.registrationSucceeded(this, bpRegistration);
+
+ // random short delay - helps scatter the BR from all DNs
+ scheduleBlockReport(dnConf.initialBlockReportDelay);
+ }
+
+
+ private void sleepAndLogInterrupts(int millis,
+ String stateString) {
+ try {
+ Thread.sleep(millis);
+ } catch (InterruptedException ie) {
+ LOG.info("BPOfferService " + this +
+ " interrupted while " + stateString);
+ }
+ }
+
+ /**
+ * No matter what kind of exception we get, keep retrying to offerService().
+ * That's the loop that connects to the NameNode and provides basic DataNode
+ * functionality.
+ *
+ * Only stop when "shouldRun" or "shouldServiceRun" is turned off, which can
+ * happen either at shutdown or due to refreshNamenodes.
+ */
+ @Override
+ public void run() {
+ LOG.info(this + " starting to offer service");
+
+ try {
+ // init stuff
+ try {
+ // setup storage
+ connectToNNAndHandshake();
+ } catch (IOException ioe) {
+ // Initial handshake, storage recovery or registration failed
+ // End BPOfferService thread
+ LOG.fatal("Initialization failed for block pool " + this, ioe);
+ return;
+ }
+
+ initialized = true; // bp is initialized;
+
+ while (shouldRun()) {
+ try {
+ bpos.startDistributedUpgradeIfNeeded();
+ offerService();
+ } catch (Exception ex) {
+ LOG.error("Exception in BPOfferService for " + this, ex);
+ sleepAndLogInterrupts(5000, "offering service");
+ }
+ }
+ } catch (Throwable ex) {
+ LOG.warn("Unexpected exception in block pool " + this, ex);
+ } finally {
+ LOG.warn("Ending block pool service for: " + this);
+ cleanUp();
+ }
+ }
+
+ private boolean shouldRun() {
+ return shouldServiceRun && dn.shouldRun();
+ }
+
+ /**
+ * Process an array of datanode commands
+ *
+ * @param cmds an array of datanode commands
+ * @return true if further processing may be required or false otherwise.
+ */
+ boolean processCommand(DatanodeCommand[] cmds) {
+ if (cmds != null) {
+ for (DatanodeCommand cmd : cmds) {
+ try {
+ if (bpos.processCommandFromActor(cmd, this) == false) {
+ return false;
+ }
+ } catch (IOException ioe) {
+ LOG.warn("Error processing datanode Command", ioe);
+ }
+ }
+ }
+ return true;
+ }
+
+ void trySendErrorReport(int errCode, String errMsg) {
+ try {
+ bpNamenode.errorReport(bpRegistration, errCode, errMsg);
+ } catch(IOException e) {
+ LOG.warn("Error reporting an error to NameNode " + nnAddr,
+ e);
+ }
+ }
+
+ /**
+ * Report a bad block from another DN in this cluster.
+ */
+ void reportRemoteBadBlock(DatanodeInfo dnInfo, ExtendedBlock block)
+ throws IOException {
+ LocatedBlock lb = new LocatedBlock(block,
+ new DatanodeInfo[] {dnInfo});
+ bpNamenode.reportBadBlocks(new LocatedBlock[] {lb});
+ }
+
+ void reRegister() throws IOException {
+ if (shouldRun()) {
+ // re-retrieve namespace info to make sure that, if the NN
+ // was restarted, we still match its version (HDFS-2120)
+ retrieveNamespaceInfo();
+ // and re-register
+ register();
+ }
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
new file mode 100644
index 0000000000..3355ee269a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
@@ -0,0 +1,241 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+/**
+ * Manages the BPOfferService objects for the data node.
+ * Creation, removal, starting, stopping, shutdown on BPOfferService
+ * objects must be done via APIs in this class.
+ */
+@InterfaceAudience.Private
+class BlockPoolManager {
+ private static final Log LOG = DataNode.LOG;
+
+ private final Map bpByNameserviceId =
+ Maps.newHashMap();
+ private final Map bpByBlockPoolId =
+ Maps.newHashMap();
+ private final List offerServices =
+ Lists.newArrayList();
+
+ private final DataNode dn;
+
+ //This lock is used only to ensure exclusion of refreshNamenodes
+ private final Object refreshNamenodesLock = new Object();
+
+ BlockPoolManager(DataNode dn) {
+ this.dn = dn;
+ }
+
+ synchronized void addBlockPool(BPOfferService bpos) {
+ Preconditions.checkArgument(offerServices.contains(bpos),
+ "Unknown BPOS: %s", bpos);
+ if (bpos.getBlockPoolId() == null) {
+ throw new IllegalArgumentException("Null blockpool id");
+ }
+ bpByBlockPoolId.put(bpos.getBlockPoolId(), bpos);
+ }
+
+ /**
+ * Returns the array of BPOfferService objects.
+ * Caution: The BPOfferService returned could be shutdown any time.
+ */
+ synchronized BPOfferService[] getAllNamenodeThreads() {
+ BPOfferService[] bposArray = new BPOfferService[offerServices.size()];
+ return offerServices.toArray(bposArray);
+ }
+
+ synchronized BPOfferService get(String bpid) {
+ return bpByBlockPoolId.get(bpid);
+ }
+
+ synchronized void remove(BPOfferService t) {
+ offerServices.remove(t);
+ bpByBlockPoolId.remove(t.getBlockPoolId());
+
+ boolean removed = false;
+ for (Iterator it = bpByNameserviceId.values().iterator();
+ it.hasNext() && !removed;) {
+ BPOfferService bpos = it.next();
+ if (bpos == t) {
+ it.remove();
+ LOG.info("Removed " + bpos);
+ removed = true;
+ }
+ }
+
+ if (!removed) {
+ LOG.warn("Couldn't remove BPOS " + t + " from bpByNameserviceId map");
+ }
+ }
+
+ void shutDownAll() throws InterruptedException {
+ BPOfferService[] bposArray = this.getAllNamenodeThreads();
+
+ for (BPOfferService bpos : bposArray) {
+ bpos.stop(); //interrupts the threads
+ }
+ //now join
+ for (BPOfferService bpos : bposArray) {
+ bpos.join();
+ }
+ }
+
+ synchronized void startAll() throws IOException {
+ try {
+ UserGroupInformation.getLoginUser().doAs(
+ new PrivilegedExceptionAction