Merge trunk into HA branch.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1296485 13f79535-47bb-0310-9956-ffa450edef68
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 0000000..748ff93
--- /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 3624c99..855b028 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 @@
       <!-- protobuf generated code -->
       <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.ProtocolInfoProtos.*"/>
     </Match>
-		<Match>
+    <Match>
       <!-- protobuf generated code -->
       <Class name="~org\.apache\.hadoop\.ipc\.protobuf\.IpcConnectionContextProtos.*"/>
     </Match>
+    <Match>
+      <!-- protobuf generated code -->
+      <Class name="~org\.apache\.hadoop\.ha\.proto\.HAServiceProtocolProtos.*"/>
+    </Match>
  </FindBugsFilter>
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 12d98c6..fd18b607 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -263,6 +263,38 @@
       <artifactId>json-simple</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>com.jcraft</groupId>
+      <artifactId>jsch</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <version>3.4.2</version>
+      <exclusions>
+        <exclusion>
+          <!-- otherwise seems to drag in junit 3.8.1 via jline -->
+          <groupId>junit</groupId>
+          <artifactId>junit</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jdmk</groupId>
+          <artifactId>jmxtools</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jmx</groupId>
+          <artifactId>jmxri</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <version>3.4.2</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
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 b8f5f51..771ac05 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.
             </td>
           </tr>
+          <tr>
+            <td><code>security.ha.service.protocol.acl</code></td>
+            <td>ACL for HAService protocol used by HAAdmin to manage the
+            active and stand-by states of namenode.
+            </td>
+          </tr>
         </table>
       </section>
       
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 f0ca72b..c2a6479 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 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 401d07ab..7953411 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 @@
   /** Default value for IPC_CLIENT_CONNECT_MAX_RETRIES_KEY */
   public static final int     IPC_CLIENT_CONNECT_MAX_RETRIES_DEFAULT = 10;
   /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
+  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 <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
   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 0000000..7da2d3e
--- /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. <br/>
+ * 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. <br/>
+ * 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. <br/>
+   * 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). <br/>
+   * 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. <br/>
+     * 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<ACL> zkAcl;
+  private byte[] appData;
+  private final String zkLockFilePath;
+  private final String znodeWorkingDir;
+
+  /**
+   * Create a new ActiveStandbyElector object <br/>
+   * 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. <br/>
+   * The parent znode name must be the same for all service instances and
+   * different across services. <br/>
+   * 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> 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. <br/>
+   * After this the elector will automatically monitor the leader status and
+   * perform re-election if necessary<br/>
+   * 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. 
+   * <br/>
+   * This will lose any leader status, if held, and stop monitoring of the lock
+   * node. <br/>
+   * If the instance wants to participate in election again, then it needs to
+   * call joinElection(). <br/>
+   * 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 0000000..3d3b1ba
--- /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 0000000..0960fb7
--- /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 0000000..09982b4
--- /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 0000000..d8bda14
--- /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.
+ * <p>
+ * Since these methods are often vendor- or device-specific, operators
+ * may implement this interface in order to achieve fencing.
+ * <p>
+ * 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.
+ * <p>
+ * If an implementation also implements {@link Configurable} then its
+ * <code>setConf</code> 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 0000000..3350692
--- /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<String, UsageInfo> USAGE =
+    ImmutableMap.<String, UsageInfo>builder()
+    .put("-transitionToActive",
+        new UsageInfo("<serviceId>", "Transitions the service into Active state"))
+    .put("-transitionToStandby",
+        new UsageInfo("<serviceId>", "Transitions the service into Standby state"))
+    .put("-failover",
+        new UsageInfo("[--"+FORCEFENCE+"] [--"+FORCEACTIVE+"] <serviceId> <serviceId>",
+            "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("<serviceId>", "Returns the state of the service"))
+    .put("-checkHealth",
+        new UsageInfo("<serviceId>",
+            "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("<command>", "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<String, UsageInfo> 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 <host:ipcport>.
+   */
+  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 0000000..18b10f9
--- /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 0000000..b8ee717
--- /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 0000000..e636adf
--- /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 0000000..34a2c8b
--- /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.<p>
+ * 
+ * The fencing methods are configured as a carriage-return separated list.
+ * Each line in the list is of the form:<p>
+ * <code>com.example.foo.MyMethod(arg string)</code>
+ * or
+ * <code>com.example.foo.MyMethod</code>
+ * The class provided must implement the {@link FenceMethod} interface.
+ * The fencing methods that ship with Hadoop may also be referred to
+ * by shortened names:<p>
+ * <ul>
+ * <li><code>shell(/path/to/some/script.sh args...)</code></li>
+ * <li><code>sshfence(...)</code> (see {@link SshFenceByTcpPort})
+ * </ul>
+ */
+@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<String, Class<? extends FenceMethod>> STANDARD_METHODS =
+    ImmutableMap.<String, Class<? extends FenceMethod>>of(
+        "shell", ShellCommandFencer.class,
+        "sshfence", SshFenceByTcpPort.class);
+  
+  private final List<FenceMethodWithArg> 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<FenceMethodWithArg> parseMethods(Configuration conf)
+      throws BadFencingConfigurationException {
+    String confStr = conf.get(CONF_METHODS_KEY);
+    String[] lines = confStr.split("\\s*\n\\s*");
+    
+    List<FenceMethodWithArg> 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 0000000..6f3e444
--- /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 0000000..ca81f23
--- /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:<br>
+ * <code>
+ *   shell(/path/to/my/script.sh arg1 arg2 ...)
+ * </code><br>
+ * The string between '(' and ')' is passed directly to a bash shell and
+ * may not include any closing parentheses.<p>
+ * 
+ * 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.<p>
+ * 
+ * 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.<p>
+ * 
+ * <em>Note:</em> 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<String> cmdList = Arrays.asList(cmd.split("\\s+"));
+
+    // Create arg list with service as the first argument
+    List<String> argList = new ArrayList<String>();
+    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<String, String> env) {
+    for (Map.Entry<String, String> 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 0000000..cec731c
--- /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 
+ * <code>fuser</code> 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.<p>
+ * It returns a successful status code if:
+ * <ul>
+ * <li><code>fuser</code> indicates it successfully killed a process, <em>or</em>
+ * <li><code>nc -z</code> indicates that nothing is listening on the target port
+ * </ul>
+ * <p>
+ * This fencing mechanism is configured as following in the fencing method
+ * list:
+ * <code>sshfence([[username][:ssh-port]])</code>
+ * where the optional argument specifies the username and port to use
+ * with ssh.
+ * <p>
+ * In order to achieve passwordless SSH, the operator must also configure
+ * <code>dfs.ha.fencing.ssh.private-key-files<code> 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<String> 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 0000000..8bc16af
--- /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 0000000..3bf4f6f
--- /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 0000000..57eefce
--- /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 0000000..3655a4e
--- /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 812a46e..ae37d0b 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 @@
  * event of failover, and always returns the same proxy object. 
  */
 @InterfaceStability.Evolving
-public class DefaultFailoverProxyProvider implements FailoverProxyProvider {
+public class DefaultFailoverProxyProvider<T> implements FailoverProxyProvider<T> {
   
-  private Object proxy;
-  private Class<?> iface;
+  private T proxy;
+  private Class<T> iface;
   
-  public DefaultFailoverProxyProvider(Class<?> iface, Object proxy) {
+  public DefaultFailoverProxyProvider(Class<T> iface, T proxy) {
     this.proxy = proxy;
     this.iface = iface;
   }
 
   @Override
-  public Class<?> getInterface() {
+  public Class<T> 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 707a40d..ba7d29f 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 @@
  * {@link RetryPolicy}.
  */
 @InterfaceStability.Evolving
-public interface FailoverProxyProvider extends Closeable {
+public interface FailoverProxyProvider<T> extends Closeable {
 
   /**
    * Get the proxy object which should be used until the next failover event
@@ -37,7 +37,7 @@
    * 
    * @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 @@
    * @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 @@
    * @return the interface implemented by the proxy objects returned by
    *         {@link FailoverProxyProvider#getProxy()}
    */
-  public Class<?> getInterface();
+  public Class<T> 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 0dad53b..323542c 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 @@
 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 @@
    * 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<String,RetryPolicy> methodNameToPolicyMap;
@@ -78,47 +80,82 @@
         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 @@
 
   @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 3634e18..2be8b75 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.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * <p>
  * A collection of useful implementations of {@link RetryPolicy}.
@@ -42,6 +44,8 @@
   
   public static final Log LOG = LogFactory.getLog(RetryPolicies.class);
   
+  private static final Random RAND = new Random();
+  
   /**
    * <p>
    * Try once, and fail by re-throwing the exception.
@@ -52,14 +56,6 @@
   
   /**
    * <p>
-   * Try once, and fail silently for <code>void</code> methods, or by
-   * re-throwing the exception for non-<code>void</code> methods.
-   * </p>
-   */
-  public static final RetryPolicy TRY_ONCE_DONT_FAIL = new TryOnceDontFail();
-  
-  /**
-   * <p>
    * Keep trying forever.
    * </p>
    */
@@ -137,18 +133,19 @@
   
   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 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 @@
   }
   
   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 @@
     
     @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 @@
     
     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 @@
     }
     
   }
+
+  /**
+   * Return a value which is <code>time</code> increasing exponentially as a
+   * function of <code>retries</code>, +/- 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 4c4534f..ed673e9 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 @@
 
 import org.apache.hadoop.classification.InterfaceStability;
 
-
 /**
  * <p>
  * Specifies a policy for retrying method failures.
@@ -33,10 +32,39 @@
    * 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 f09600d..e5a2d7f 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 @@
     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 @@
       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 @@
           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 @@
     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 @@
     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 @@
       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 @@
       return maxRetries;
     }
     
+    /** max connection retries on socket time outs */
+    public int getMaxRetriesOnSocketTimeouts() {
+      return maxRetriesOnSocketTimeouts;
+    }
+    
     boolean getTcpNoDelay() {
       return tcpNoDelay;
     }
@@ -1369,6 +1377,9 @@
               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 0000000..5bf9dba
--- /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 4f85e90..eee364c 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 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 @@
    * 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 @@
    * @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 2b35598..5f642c4 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 @@
               // 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 49f4fad..7a16861 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 @@
  * 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 2685887..43132d2 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.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;
@@ -449,6 +450,27 @@
   }
   
   /**
+   * 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> T doAsLoginUserOrFatal(PrivilegedAction<T> 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 @@
     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 3c2e666..11df981 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.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 @@
   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 @@
 
   /** 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 @@
     }
   }
 
-  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 @@
             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 0000000..6e4dfaf
--- /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 b3e12d1..2fd9f8d 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.</description>
   </property>
+  
+  <property>
+    <name>security.ha.service.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HAService protocol used by HAAdmin to manage the
+      active and stand-by states of namenode.</description>
+  </property>
 
    <property>
       <name>security.mrhs.client.protocol.acl</name>
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 0000000..a3fd86c
--- /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 8fc45c5..a968400 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
@@ -488,6 +488,14 @@
 </property>
 
 <property>
+  <name>ipc.client.connect.max.retries.on.timeouts</name>
+  <value>45</value>
+  <description>Indicates the number of retries a client will make on socket timeout
+               to establish a server connection.
+  </description>
+</property>
+
+<property>
   <name>ipc.server.listen.queue.size</name>
   <value>128</value>
   <description>Indicates the length of the listen queue for servers accepting
@@ -849,4 +857,30 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.ha.fencing.methods</name>
+  <value></value>
+  <description>
+    List of fencing methods to use for service fencing. May contain
+    builtin methods (eg shell and sshfence) or user-defined method.
+  </description>
+</property>
+
+<property>
+  <name>dfs.ha.fencing.ssh.connect-timeout</name>
+  <value>30000</value>
+  <description>
+    SSH connection timeout, in milliseconds, to use with the builtin
+    sshfence fencer.
+  </description>
+</property>
+
+<property>
+  <name>dfs.ha.fencing.ssh.private-key-files</name>
+  <value></value>
+  <description>
+    The SSH private key files to use with the builtin sshfence fencer.
+  </description>
+</property>
+
 </configuration>
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 0000000..fec350d
--- /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> 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.<AsyncCallback.StatCallback> anyObject(),
+        Mockito.<Object> anyObject());
+
+    // disconnection should enter safe mode
+    Mockito.when(mockEvent.getState()).thenReturn(
+        Event.KeeperState.Disconnected);
+    elector.process(mockEvent);
+    Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
+
+    // re-connection should monitor master status
+    Mockito.when(mockEvent.getState()).thenReturn(
+        Event.KeeperState.SyncConnected);
+    elector.process(mockEvent);
+    Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
+        elector, null);
+
+    // session expired should enter safe mode and initiate re-election
+    // re-election checked via checking re-creation of new zookeeper and
+    // call to create lock znode
+    Mockito.when(mockEvent.getState()).thenReturn(Event.KeeperState.Expired);
+    elector.process(mockEvent);
+    // already in safe mode above. should not enter safe mode again
+    Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
+    // called getNewZooKeeper to create new session. first call was in
+    // constructor
+    Assert.assertEquals(2, TestActiveStandbyElector.count);
+    // once in initial joinElection and one now
+    Mockito.verify(mockZK, Mockito.times(2)).create(zkLockPathName, data,
+        Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+
+    // create znode success. become master and monitor
+    elector.processResult(Code.OK.intValue(), zkLockPathName, null,
+        zkLockPathName);
+    Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
+    Mockito.verify(mockZK, Mockito.times(2)).exists(zkLockPathName, true,
+        elector, null);
+
+    // error event results in fatal error
+    Mockito.when(mockEvent.getState()).thenReturn(Event.KeeperState.AuthFailed);
+    elector.process(mockEvent);
+    Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
+        "Unexpected Zookeeper watch event state: AuthFailed");
+    // only 1 state change callback is called at a time
+    Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
+  }
+
+  /**
+   * verify behavior of watcher.process with node event
+   */
+  @Test
+  public void testProcessCallbackEventNode() {
+    elector.joinElection(data);
+
+    // make the object go into the monitoring state
+    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);
+
+    WatchedEvent mockEvent = Mockito.mock(WatchedEvent.class);
+    Mockito.when(mockEvent.getPath()).thenReturn(zkLockPathName);
+
+    // monitoring should be setup again after event is received
+    Mockito.when(mockEvent.getType()).thenReturn(
+        Event.EventType.NodeDataChanged);
+    elector.process(mockEvent);
+    Mockito.verify(mockZK, Mockito.times(2)).exists(zkLockPathName, true,
+        elector, null);
+
+    // monitoring should be setup again after event is received
+    Mockito.when(mockEvent.getType()).thenReturn(
+        Event.EventType.NodeChildrenChanged);
+    elector.process(mockEvent);
+    Mockito.verify(mockZK, Mockito.times(3)).exists(zkLockPathName, true,
+        elector, null);
+
+    // lock node deletion when in standby mode should create znode again
+    // successful znode creation enters active state and sets monitor
+    Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.NodeDeleted);
+    elector.process(mockEvent);
+    // enterNeutralMode not called when app is standby and leader is lost
+    Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
+    // once in initial joinElection() and one now
+    Mockito.verify(mockZK, Mockito.times(2)).create(zkLockPathName, data,
+        Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+    elector.processResult(Code.OK.intValue(), zkLockPathName, null,
+        zkLockPathName);
+    Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
+    Mockito.verify(mockZK, Mockito.times(4)).exists(zkLockPathName, true,
+        elector, null);
+
+    // lock node deletion in active mode should enter neutral mode and create
+    // znode again successful znode creation enters active state and sets
+    // monitor
+    Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.NodeDeleted);
+    elector.process(mockEvent);
+    Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
+    // another joinElection called
+    Mockito.verify(mockZK, Mockito.times(3)).create(zkLockPathName, data,
+        Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+    elector.processResult(Code.OK.intValue(), zkLockPathName, null,
+        zkLockPathName);
+    Mockito.verify(mockApp, Mockito.times(2)).becomeActive();
+    Mockito.verify(mockZK, Mockito.times(5)).exists(zkLockPathName, true,
+        elector, null);
+
+    // bad path name results in fatal error
+    Mockito.when(mockEvent.getPath()).thenReturn(null);
+    elector.process(mockEvent);
+    Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
+        "Unexpected watch error from Zookeeper");
+    // fatal error means no new connection other than one from constructor
+    Assert.assertEquals(1, TestActiveStandbyElector.count);
+    // no new watches after fatal error
+    Mockito.verify(mockZK, Mockito.times(5)).exists(zkLockPathName, true,
+        elector, null);
+
+  }
+
+  /**
+   * verify becomeStandby is not called if already in standby
+   */
+  @Test
+  public void testSuccessiveStandbyCalls() {
+    elector.joinElection(data);
+
+    // make the object go into the monitoring standby state
+    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);
+
+    WatchedEvent mockEvent = Mockito.mock(WatchedEvent.class);
+    Mockito.when(mockEvent.getPath()).thenReturn(zkLockPathName);
+
+    // notify node deletion
+    // monitoring should be setup again after event is received
+    Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.NodeDeleted);
+    elector.process(mockEvent);
+    // is standby. no need to notify anything now
+    Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
+    // another joinElection called.
+    Mockito.verify(mockZK, Mockito.times(2)).create(zkLockPathName, data,
+        Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
+    // lost election
+    elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
+        zkLockPathName);
+    // still standby. so no need to notify again
+    Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
+    // monitor is set again
+    Mockito.verify(mockZK, Mockito.times(2)).exists(zkLockPathName, true,
+        elector, null);
+  }
+
+  /**
+   * verify quit election terminates connection and there are no new watches.
+   * next call to joinElection creates new connection and performs election
+   */
+  @Test
+  public void testQuitElection() throws InterruptedException {
+    elector.quitElection();
+    Mockito.verify(mockZK, Mockito.times(1)).close();
+    // no watches added
+    Mockito.verify(mockZK, Mockito.times(0)).exists(zkLockPathName, true,
+        elector, null);
+
+    byte[] data = new byte[8];
+    elector.joinElection(data);
+    // getNewZooKeeper called 2 times. once in constructor and once now
+    Assert.assertEquals(2, TestActiveStandbyElector.count);
+    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 receiveActiveData gives data when active exists, tells that
+   * active does not exist and reports error in getting active information
+   * 
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws KeeperException
+   * @throws ActiveNotFoundException
+   */
+  @Test
+  public void testGetActiveData() throws ActiveNotFoundException,
+      KeeperException, InterruptedException, IOException {
+    // get valid active data
+    byte[] data = new byte[8];
+    Mockito.when(
+        mockZK.getData(Mockito.eq(zkLockPathName), Mockito.eq(false),
+            Mockito.<Stat> anyObject())).thenReturn(data);
+    Assert.assertEquals(data, elector.getActiveData());
+    Mockito.verify(mockZK, Mockito.times(1)).getData(
+        Mockito.eq(zkLockPathName), Mockito.eq(false),
+        Mockito.<Stat> anyObject());
+
+    // active does not exist
+    Mockito.when(
+        mockZK.getData(Mockito.eq(zkLockPathName), Mockito.eq(false),
+            Mockito.<Stat> anyObject())).thenThrow(
+        new KeeperException.NoNodeException());
+    try {
+      elector.getActiveData();
+      Assert.fail("ActiveNotFoundException expected");
+    } catch(ActiveNotFoundException e) {
+      Mockito.verify(mockZK, Mockito.times(2)).getData(
+          Mockito.eq(zkLockPathName), Mockito.eq(false),
+          Mockito.<Stat> anyObject());
+    }
+
+    // error getting active data rethrows keeperexception
+    try {
+      Mockito.when(
+          mockZK.getData(Mockito.eq(zkLockPathName), Mockito.eq(false),
+              Mockito.<Stat> anyObject())).thenThrow(
+          new KeeperException.AuthFailedException());
+      elector.getActiveData();
+      Assert.fail("KeeperException.AuthFailedException expected");
+    } catch(KeeperException.AuthFailedException ke) {
+      Mockito.verify(mockZK, Mockito.times(3)).getData(
+          Mockito.eq(zkLockPathName), Mockito.eq(false),
+          Mockito.<Stat> anyObject());
+    }
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java
new file mode 100644
index 0000000..672e8d3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestActiveStandbyElectorRealZK.java
@@ -0,0 +1,231 @@
+/**
+ * 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.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.test.ClientBase;
+
+/**
+ * Test for {@link ActiveStandbyElector} using real zookeeper.
+ */
+public class TestActiveStandbyElectorRealZK extends ClientBase {
+  static final int NUM_ELECTORS = 2;
+  static ZooKeeper[] zkClient = new ZooKeeper[NUM_ELECTORS];
+  static int currentClientIndex = 0;
+  
+  @Override
+  public void setUp() throws Exception {
+    // build.test.dir is used by zookeeper
+    new File(System.getProperty("build.test.dir", "build")).mkdirs();
+    super.setUp();
+  }
+
+  class ActiveStandbyElectorTesterRealZK extends ActiveStandbyElector {
+    ActiveStandbyElectorTesterRealZK(String hostPort, int timeout,
+        String parent, List<ACL> acl, ActiveStandbyElectorCallback app)
+        throws IOException {
+      super(hostPort, timeout, parent, acl, app);
+    }
+
+    @Override
+    public ZooKeeper getNewZooKeeper() {
+      return TestActiveStandbyElectorRealZK.zkClient[
+                             TestActiveStandbyElectorRealZK.currentClientIndex];
+    }
+  }
+
+  /**
+   * The class object runs on a thread and waits for a signal to start from the 
+   * test object. On getting the signal it joins the election and thus by doing 
+   * this on multiple threads we can test simultaneous attempts at leader lock 
+   * creation. after joining the election, the object waits on a signal to exit.
+   * this signal comes when the object's elector has become a leader or there is 
+   * an unexpected fatal error. this lets another thread object to become a 
+   * leader.
+   */
+  class ThreadRunner implements Runnable, ActiveStandbyElectorCallback {
+    int index;
+    TestActiveStandbyElectorRealZK test;
+    boolean wait = true;
+
+    ThreadRunner(int i, TestActiveStandbyElectorRealZK s) {
+      index = i;
+      test = s;
+    }
+
+    @Override
+    public void run() {
+      LOG.info("starting " + index);
+      while(true) {
+        synchronized (test) {
+          // wait for test start signal to come
+          if (!test.start) {
+            try {
+              test.wait();
+            } catch(InterruptedException e) {
+              Assert.fail(e.getMessage());
+            }
+          } else {
+            break;
+          }
+        }
+      }
+      // join election
+      byte[] data = new byte[8];
+      ActiveStandbyElector elector = test.elector[index];
+      LOG.info("joining " + index);
+      elector.joinElection(data);
+      try {
+        while(true) {
+          synchronized (this) {
+            // wait for elector to become active/fatal error
+            if (wait) {
+              // wait to become active
+              // wait capped at 30s to prevent hung test
+              wait(30000);
+            } else {
+              break;
+            }
+          }
+        }
+        Thread.sleep(1000);
+        // quit election to allow other elector to become active
+        elector.quitElection();
+      } catch(InterruptedException e) {
+        Assert.fail(e.getMessage());
+      }
+      LOG.info("ending " + index);
+    }
+
+    @Override
+    public synchronized void becomeActive() {
+      test.reportActive(index);
+      LOG.info("active " + index);
+      wait = false;
+      notifyAll();
+    }
+
+    @Override
+    public synchronized void becomeStandby() {
+      test.reportStandby(index);
+      LOG.info("standby " + index);
+    }
+
+    @Override
+    public synchronized void enterNeutralMode() {
+      LOG.info("neutral " + index);
+    }
+
+    @Override
+    public synchronized void notifyFatalError(String errorMessage) {
+      LOG.info("fatal " + index + " .Error message:" + errorMessage);
+      wait = false;
+      notifyAll();
+    }
+  }
+
+  boolean start = false;
+  int activeIndex = -1;
+  int standbyIndex = -1;
+  String parentDir = "/" + java.util.UUID.randomUUID().toString();
+
+  ActiveStandbyElector[] elector = new ActiveStandbyElector[NUM_ELECTORS];
+  ThreadRunner[] threadRunner = new ThreadRunner[NUM_ELECTORS];
+  Thread[] thread = new Thread[NUM_ELECTORS];
+
+  synchronized void reportActive(int index) {
+    if (activeIndex == -1) {
+      activeIndex = index;
+    } else {
+      // standby should become active
+      Assert.assertEquals(standbyIndex, index);
+      // old active should not become active
+      Assert.assertFalse(activeIndex == index);
+    }
+    activeIndex = index;
+  }
+
+  synchronized void reportStandby(int index) {
+    // only 1 standby should be reported and it should not be the same as active
+    Assert.assertEquals(-1, standbyIndex);
+    standbyIndex = index;
+    Assert.assertFalse(activeIndex == standbyIndex);
+  }
+
+  /**
+   * the test creates 2 electors which try to become active using a real
+   * zookeeper server. It verifies that 1 becomes active and 1 becomes standby.
+   * Upon becoming active the leader quits election and the test verifies that
+   * the standby now becomes active. these electors run on different threads and 
+   * callback to the test class to report active and standby where the outcome 
+   * is verified
+   * 
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws KeeperException
+   */
+  @Test
+  public void testActiveStandbyTransition() throws IOException,
+      InterruptedException, KeeperException {
+    LOG.info("starting test with parentDir:" + parentDir);
+    start = false;
+    byte[] data = new byte[8];
+    // create random working directory
+    createClient().create(parentDir, data, Ids.OPEN_ACL_UNSAFE,
+        CreateMode.PERSISTENT);
+
+    for(currentClientIndex = 0; 
+        currentClientIndex < NUM_ELECTORS; 
+        ++currentClientIndex) {
+      LOG.info("creating " + currentClientIndex);
+      zkClient[currentClientIndex] = createClient();
+      threadRunner[currentClientIndex] = new ThreadRunner(currentClientIndex,
+          this);
+      elector[currentClientIndex] = new ActiveStandbyElectorTesterRealZK(
+          "hostPort", 1000, parentDir, Ids.OPEN_ACL_UNSAFE,
+          threadRunner[currentClientIndex]);
+      zkClient[currentClientIndex].register(elector[currentClientIndex]);
+      thread[currentClientIndex] = new Thread(threadRunner[currentClientIndex]);
+      thread[currentClientIndex].start();
+    }
+
+    synchronized (this) {
+      // signal threads to start
+      LOG.info("signaling threads");
+      start = true;
+      notifyAll();
+    }
+
+    for(int i = 0; i < thread.length; i++) {
+      thread[i].join();
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestFailoverController.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestFailoverController.java
new file mode 100644
index 0000000..9e2cc75
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestFailoverController.java
@@ -0,0 +1,441 @@
+/**
+ * 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 static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB;
+import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
+import org.apache.hadoop.ha.TestNodeFencer.AlwaysFailFencer;
+import static org.apache.hadoop.ha.TestNodeFencer.setupFencer;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestFailoverController {
+
+  private InetSocketAddress svc1Addr = new InetSocketAddress("svc1", 1234); 
+  private InetSocketAddress svc2Addr = new InetSocketAddress("svc2", 5678); 
+
+  private class DummyService implements HAServiceProtocol {
+    HAServiceState state;
+
+    DummyService(HAServiceState state) {
+      this.state = state;
+    }
+
+    @Override
+    public void monitorHealth() throws HealthCheckFailedException, IOException {
+      // Do nothing
+    }
+
+    @Override
+    public void transitionToActive() throws ServiceFailedException, IOException {
+      state = HAServiceState.ACTIVE;
+    }
+
+    @Override
+    public void transitionToStandby() throws ServiceFailedException, IOException {
+      state = HAServiceState.STANDBY;
+    }
+
+    @Override
+    public HAServiceState getServiceState() throws IOException {
+      return state;
+    }
+
+    @Override
+    public boolean readyToBecomeActive() throws ServiceFailedException, IOException {
+      return true;
+    }
+  }
+  
+  @Test
+  public void testFailoverAndFailback() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    AlwaysSucceedFencer.fenceCalled = 0;
+    FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+    assertEquals(0, TestNodeFencer.AlwaysSucceedFencer.fenceCalled);
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
+
+    AlwaysSucceedFencer.fenceCalled = 0;
+    FailoverController.failover(svc2, svc2Addr, svc1, svc1Addr, fencer, false, false);
+    assertEquals(0, TestNodeFencer.AlwaysSucceedFencer.fenceCalled);
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverFromStandbyToStandby() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.STANDBY);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverFromActiveToActive() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.ACTIVE);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Can't failover to an already active service");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverWithoutPermission() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
+      @Override
+      public HAServiceState getServiceState() throws IOException {
+        throw new AccessControlException("Access denied");
+      }
+    };
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public HAServiceState getServiceState() throws IOException {
+        throw new AccessControlException("Access denied");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Can't failover when access is denied");
+    } catch (FailoverFailedException ffe) {
+      assertTrue(ffe.getCause().getMessage().contains("Access denied"));
+    }
+  }
+
+
+  @Test
+  public void testFailoverToUnreadyService() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public boolean readyToBecomeActive() throws ServiceFailedException, IOException {
+        return false;
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Can't failover to a service that's not ready");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+
+    // Forcing it means we ignore readyToBecomeActive
+    FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, true);
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverToUnhealthyServiceFailsAndFailsback() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void monitorHealth() throws HealthCheckFailedException {
+        throw new HealthCheckFailedException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Failover to unhealthy service");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverFromFaultyServiceSucceeds() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
+      @Override
+      public void transitionToStandby() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    AlwaysSucceedFencer.fenceCalled = 0;
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+    } catch (FailoverFailedException ffe) {
+      fail("Faulty active prevented failover");
+    }
+
+    // svc1 still thinks it's active, that's OK, it was fenced
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("svc1:1234", AlwaysSucceedFencer.fencedSvc);
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverFromFaultyServiceFencingFailure() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
+      @Override
+      public void transitionToStandby() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
+
+    AlwaysFailFencer.fenceCalled = 0;
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Failed over even though fencing failed");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("svc1:1234", AlwaysFailFencer.fencedSvc);
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFencingFailureDuringFailover() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
+
+    AlwaysFailFencer.fenceCalled = 0;
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, true, false);
+      fail("Failed over even though fencing requested and failed");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // If fencing was requested and it failed we don't try to make
+    // svc2 active anyway, and we don't failback to svc1.
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("svc1:1234", AlwaysFailFencer.fencedSvc);
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+  
+  private HAServiceProtocol getProtocol(String target)
+      throws IOException {
+    InetSocketAddress addr = NetUtils.createSocketAddr(target);
+    Configuration conf = new Configuration();
+    // Lower the timeout so we quickly fail to connect
+    conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
+    return new HAServiceProtocolClientSideTranslatorPB(addr, conf);
+  }
+
+  @Test
+  public void testFailoverFromNonExistantServiceWithFencer() throws Exception {
+    HAServiceProtocol svc1 = getProtocol("localhost:1234");
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY);
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+    } catch (FailoverFailedException ffe) {
+      fail("Non-existant active prevented failover");
+    }
+
+    // Don't check svc1 because we can't reach it, but that's OK, it's been fenced.
+    assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
+  }
+
+  @Test
+  public void testFailoverToNonExistantServiceFails() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    HAServiceProtocol svc2 = getProtocol("localhost:1234");
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Failed over to a non-existant standby");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+  }
+
+  @Test
+  public void testFailoverToFaultyServiceFailsbackOK() throws Exception {
+    DummyService svc1 = spy(new DummyService(HAServiceState.ACTIVE));
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Failover to already active service");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // svc1 went standby then back to active
+    verify(svc1).transitionToStandby();
+    verify(svc1).transitionToActive();
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testWeDontFailbackIfActiveWasFenced() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, true, false);
+      fail("Failed over to service that won't transition to active");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // We failed to failover and did not failback because we fenced
+    // svc1 (we forced it), therefore svc1 and svc2 should be standby.
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+
+  @Test
+  public void testWeFenceOnFailbackIfTransitionToActiveFails() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException, IOException {
+        throw new IOException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+    AlwaysSucceedFencer.fenceCalled = 0;
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Failed over to service that won't transition to active");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // We failed to failover. We did not fence svc1 because it cooperated
+    // and we didn't force it, so we failed back to svc1 and fenced svc2.
+    // Note svc2 still thinks it's active, that's OK, we fenced it.
+    assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("svc2:5678", AlwaysSucceedFencer.fencedSvc);
+  }
+
+  @Test
+  public void testFailureToFenceOnFailbackFailsTheFailback() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException, IOException {
+        throw new IOException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
+    AlwaysFailFencer.fenceCalled = 0;
+
+    try {
+      FailoverController.failover(svc1,  svc1Addr,  svc2,  svc2Addr, fencer, false, false);
+      fail("Failed over to service that won't transition to active");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    // We did not fence svc1 because it cooperated and we didn't force it, 
+    // we failed to failover so we fenced svc2, we failed to fence svc2
+    // so we did not failback to svc1, ie it's still standby.
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("svc2:5678", AlwaysFailFencer.fencedSvc);
+  }
+
+  @Test
+  public void testFailbackToFaultyServiceFails() throws Exception {
+    DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
+      @Override
+      public void transitionToActive() throws ServiceFailedException {
+        throw new ServiceFailedException("Failed!");
+      }
+    };
+    NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
+
+    try {
+      FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
+      fail("Failover to already active service");
+    } catch (FailoverFailedException ffe) {
+      // Expected
+    }
+
+    assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
+    assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java
new file mode 100644
index 0000000..f22056a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHAAdmin.java
@@ -0,0 +1,106 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.when;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+
+public class TestHAAdmin {
+  private static final Log LOG = LogFactory.getLog(TestHAAdmin.class);
+  
+  private HAAdmin tool;
+  private ByteArrayOutputStream errOutBytes = new ByteArrayOutputStream();
+  private String errOutput;
+  private HAServiceProtocol mockProtocol;
+  
+  @Before
+  public void setup() throws IOException {
+    mockProtocol = Mockito.mock(HAServiceProtocol.class);
+    when(mockProtocol.readyToBecomeActive()).thenReturn(true);
+    tool = new HAAdmin() {
+      @Override
+      protected HAServiceProtocol getProtocol(String target) throws IOException {
+        return mockProtocol;
+      }
+    };
+    tool.setConf(new Configuration());
+    tool.errOut = new PrintStream(errOutBytes);
+  }
+  
+  private void assertOutputContains(String string) {
+    if (!errOutput.contains(string)) {
+      fail("Expected output to contain '" + string + "' but was:\n" +
+          errOutput);
+    }
+  }
+  
+  @Test
+  public void testAdminUsage() throws Exception {
+    assertEquals(-1, runTool());
+    assertOutputContains("Usage:");
+    assertOutputContains("-transitionToActive");
+    
+    assertEquals(-1, runTool("badCommand"));
+    assertOutputContains("Bad command 'badCommand'");
+    
+    assertEquals(-1, runTool("-badCommand"));
+    assertOutputContains("badCommand: Unknown");    
+
+    // valid command but not enough arguments
+    assertEquals(-1, runTool("-transitionToActive"));
+    assertOutputContains("transitionToActive: incorrect number of arguments");
+    assertEquals(-1, runTool("-transitionToActive", "x", "y"));
+    assertOutputContains("transitionToActive: incorrect number of arguments");
+    assertEquals(-1, runTool("-failover"));
+    assertOutputContains("failover: incorrect arguments");
+    assertOutputContains("failover: incorrect arguments");    
+    assertEquals(-1, runTool("-failover", "foo:1234"));
+    assertOutputContains("failover: incorrect arguments");
+  }
+
+  @Test
+  public void testHelp() throws Exception {
+    assertEquals(-1, runTool("-help"));
+    assertEquals(0, runTool("-help", "transitionToActive"));
+    assertOutputContains("Transitions the service into Active");
+  }
+
+  private Object runTool(String ... args) throws Exception {
+    errOutBytes.reset();
+    LOG.info("Running: HAAdmin " + Joiner.on(" ").join(args));
+    int ret = tool.run(args);
+    errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8);
+    LOG.info("Output:\n" + errOutput);
+    return ret;
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
new file mode 100644
index 0000000..5508547
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
@@ -0,0 +1,173 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestNodeFencer {
+
+  @Before
+  public void clearMockState() {
+    AlwaysSucceedFencer.fenceCalled = 0;
+    AlwaysSucceedFencer.callArgs.clear();
+    AlwaysFailFencer.fenceCalled = 0;
+    AlwaysFailFencer.callArgs.clear();
+  }
+
+  @Test
+  public void testSingleFencer() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer(
+        AlwaysSucceedFencer.class.getName() + "(foo)");
+    assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("host:1234", AlwaysSucceedFencer.fencedSvc);
+    assertEquals("foo", AlwaysSucceedFencer.callArgs.get(0));
+  }
+  
+  @Test
+  public void testMultipleFencers() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer(
+        AlwaysSucceedFencer.class.getName() + "(foo)\n" +
+        AlwaysSucceedFencer.class.getName() + "(bar)\n");
+    assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
+    // Only one call, since the first fencer succeeds
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("foo", AlwaysSucceedFencer.callArgs.get(0));
+  }
+  
+  @Test
+  public void testWhitespaceAndCommentsInConfig()
+      throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer(
+        "\n" +
+        " # the next one will always fail\n" +
+        " " + AlwaysFailFencer.class.getName() + "(foo) # <- fails\n" +
+        AlwaysSucceedFencer.class.getName() + "(bar) \n");
+    assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
+    // One call to each, since top fencer fails
+    assertEquals(1, AlwaysFailFencer.fenceCalled);
+    assertEquals("host:1234", AlwaysFailFencer.fencedSvc);
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("host:1234", AlwaysSucceedFencer.fencedSvc);
+    assertEquals("foo", AlwaysFailFencer.callArgs.get(0));
+    assertEquals("bar", AlwaysSucceedFencer.callArgs.get(0));
+  }
+ 
+  @Test
+  public void testArglessFencer() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer(
+        AlwaysSucceedFencer.class.getName());
+    assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
+    // One call to each, since top fencer fails
+    assertEquals(1, AlwaysSucceedFencer.fenceCalled);
+    assertEquals("host:1234", AlwaysSucceedFencer.fencedSvc);
+    assertEquals(null, AlwaysSucceedFencer.callArgs.get(0));
+  }
+
+  @Test
+  public void testShortNameShell() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer("shell(true)");
+    assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
+  }
+
+  @Test
+  public void testShortNameSsh() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer("sshfence");
+    assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
+  }
+
+  @Test
+  public void testShortNameSshWithUser() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer("sshfence(user)");
+    assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
+  }
+
+  @Test
+  public void testShortNameSshWithPort() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer("sshfence(:123)");
+    assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
+  }
+
+  @Test
+  public void testShortNameSshWithUserPort() throws BadFencingConfigurationException {
+    NodeFencer fencer = setupFencer("sshfence(user:123)");
+    assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
+  }
+
+  public static NodeFencer setupFencer(String confStr)
+      throws BadFencingConfigurationException {
+    System.err.println("Testing configuration:\n" + confStr);
+    Configuration conf = new Configuration();
+    conf.set(NodeFencer.CONF_METHODS_KEY, confStr);
+    return new NodeFencer(conf);
+  }
+  
+  /**
+   * Mock fencing method that always returns true
+   */
+  public static class AlwaysSucceedFencer extends Configured
+      implements FenceMethod {
+    static int fenceCalled = 0;
+    static String fencedSvc;
+    static List<String> callArgs = Lists.newArrayList();
+
+    @Override
+    public boolean tryFence(InetSocketAddress serviceAddr, String args) {
+      fencedSvc = serviceAddr.getHostName() + ":" + serviceAddr.getPort();
+      callArgs.add(args);
+      fenceCalled++;
+      return true;
+    }
+
+    @Override
+    public void checkArgs(String args) {
+    }
+  }
+  
+  /**
+   * Identical mock to above, except always returns false
+   */
+  public static class AlwaysFailFencer extends Configured
+      implements FenceMethod {
+    static int fenceCalled = 0;
+    static String fencedSvc;
+    static List<String> callArgs = Lists.newArrayList();
+
+    @Override
+    public boolean tryFence(InetSocketAddress serviceAddr, String args) {
+      fencedSvc = serviceAddr.getHostName() + ":" + serviceAddr.getPort();
+      callArgs.add(args);
+      fenceCalled++;
+      return false;
+    }
+
+    @Override
+    public void checkArgs(String args) {
+    }
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java
new file mode 100644
index 0000000..49bae03
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestShellCommandFencer.java
@@ -0,0 +1,154 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.mockito.Mockito.spy;
+
+public class TestShellCommandFencer {
+  private ShellCommandFencer fencer = createFencer();
+  
+  @BeforeClass
+  public static void setupLogSpy() {
+    ShellCommandFencer.LOG = spy(ShellCommandFencer.LOG);
+  }
+  
+  @Before
+  public void resetLogSpy() {
+    Mockito.reset(ShellCommandFencer.LOG);
+  }
+  
+  private static ShellCommandFencer createFencer() {
+    Configuration conf = new Configuration();
+    conf.set("in.fencing.tests", "yessir");
+    ShellCommandFencer fencer = new ShellCommandFencer();
+    fencer.setConf(conf);
+    return fencer;
+  }
+  
+  /**
+   * Test that the exit code of the script determines
+   * whether the fencer succeeded or failed
+   */
+  @Test
+  public void testBasicSuccessFailure() {
+    InetSocketAddress addr = new InetSocketAddress("host", 1234);
+    assertTrue(fencer.tryFence(addr, "echo"));
+    assertFalse(fencer.tryFence(addr, "exit 1"));
+    // bad path should also fail
+    assertFalse(fencer.tryFence(addr, "xxxxxxxxxxxx"));
+  }
+  
+  @Test
+  public void testCheckNoArgs() {
+    try {
+      Configuration conf = new Configuration();
+      conf.set(NodeFencer.CONF_METHODS_KEY, "shell");
+      new NodeFencer(conf);
+      fail("Didn't throw when passing no args to shell");
+    } catch (BadFencingConfigurationException confe) {
+      assertTrue(
+        "Unexpected exception:" + StringUtils.stringifyException(confe),
+        confe.getMessage().contains("No argument passed"));    
+    }
+  }
+
+  @Test
+  public void testCheckParensNoArgs() {
+    try {
+      Configuration conf = new Configuration();
+      conf.set(NodeFencer.CONF_METHODS_KEY, "shell()");
+      new NodeFencer(conf);
+      fail("Didn't throw when passing no args to shell");
+    } catch (BadFencingConfigurationException confe) {
+      assertTrue(
+        "Unexpected exception:" + StringUtils.stringifyException(confe),
+        confe.getMessage().contains("Unable to parse line: 'shell()'"));
+    }
+  }
+
+  /**
+   * Test that lines on stdout get passed as INFO
+   * level messages
+   */
+  @Test
+  public void testStdoutLogging() {
+    InetSocketAddress addr = new InetSocketAddress("host", 1234);
+    assertTrue(fencer.tryFence(addr, "echo hello"));
+    Mockito.verify(ShellCommandFencer.LOG).info(
+        Mockito.endsWith("echo hello: host:1234 hello"));
+  }
+   
+  /**
+   * Test that lines on stderr get passed as
+   * WARN level log messages
+   */
+  @Test
+  public void testStderrLogging() {
+    InetSocketAddress addr = new InetSocketAddress("host", 1234);
+    assertTrue(fencer.tryFence(addr, "echo hello >&2"));
+    Mockito.verify(ShellCommandFencer.LOG).warn(
+        Mockito.endsWith("echo hello >&2: host:1234 hello"));
+  }
+
+  /**
+   * Verify that the Configuration gets passed as
+   * environment variables to the fencer.
+   */
+  @Test
+  public void testConfAsEnvironment() {
+    InetSocketAddress addr = new InetSocketAddress("host", 1234);
+    fencer.tryFence(addr, "echo $in_fencing_tests");
+    Mockito.verify(ShellCommandFencer.LOG).info(
+        Mockito.endsWith("echo $in...ing_tests: host:1234 yessir"));
+  }
+
+  /**
+   * Test that we properly close off our input to the subprocess
+   * such that it knows there's no tty connected. This is important
+   * so that, if we use 'ssh', it won't try to prompt for a password
+   * and block forever, for example.
+   */
+  @Test(timeout=10000)
+  public void testSubprocessInputIsClosed() {
+    InetSocketAddress addr = new InetSocketAddress("host", 1234);
+    assertFalse(fencer.tryFence(addr, "read"));
+  }
+  
+  @Test
+  public void testCommandAbbreviation() {
+    assertEquals("a...f", ShellCommandFencer.abbreviate("abcdef", 5));
+    assertEquals("abcdef", ShellCommandFencer.abbreviate("abcdef", 6));
+    assertEquals("abcdef", ShellCommandFencer.abbreviate("abcdef", 7));
+
+    assertEquals("a...g", ShellCommandFencer.abbreviate("abcdefg", 5));
+    assertEquals("a...h", ShellCommandFencer.abbreviate("abcdefgh", 5));
+    assertEquals("a...gh", ShellCommandFencer.abbreviate("abcdefgh", 6));
+    assertEquals("ab...gh", ShellCommandFencer.abbreviate("abcdefgh", 7));
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java
new file mode 100644
index 0000000..f89df6a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestSshFenceByTcpPort.java
@@ -0,0 +1,130 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.SshFenceByTcpPort.Args;
+import org.apache.log4j.Level;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestSshFenceByTcpPort {
+
+  static {
+    ((Log4JLogger)SshFenceByTcpPort.LOG).getLogger().setLevel(Level.ALL);
+  }
+  
+  private String TEST_FENCING_HOST = System.getProperty(
+      "test.TestSshFenceByTcpPort.host", "localhost");
+  private String TEST_FENCING_PORT = System.getProperty(
+      "test.TestSshFenceByTcpPort.port", "8020");
+  private final String TEST_KEYFILE = System.getProperty(
+      "test.TestSshFenceByTcpPort.key");
+
+  @Test(timeout=20000)
+  public void testFence() throws BadFencingConfigurationException {
+    Assume.assumeTrue(isConfigured());
+    Configuration conf = new Configuration();
+    conf.set(SshFenceByTcpPort.CONF_IDENTITIES_KEY, TEST_KEYFILE);
+    SshFenceByTcpPort fence = new SshFenceByTcpPort();
+    fence.setConf(conf);
+    assertTrue(fence.tryFence(
+        new InetSocketAddress(TEST_FENCING_HOST,
+                              Integer.valueOf(TEST_FENCING_PORT)),
+        null));
+  }
+
+  /**
+   * Test connecting to a host which definitely won't respond.
+   * Make sure that it times out and returns false, but doesn't throw
+   * any exception
+   */
+  @Test(timeout=20000)
+  public void testConnectTimeout() throws BadFencingConfigurationException {
+    Configuration conf = new Configuration();
+    conf.setInt(SshFenceByTcpPort.CONF_CONNECT_TIMEOUT_KEY, 3000);
+    SshFenceByTcpPort fence = new SshFenceByTcpPort();
+    fence.setConf(conf);
+    // Connect to Google's DNS server - not running ssh!
+    assertFalse(fence.tryFence(new InetSocketAddress("8.8.8.8", 1234), ""));
+  }
+  
+  @Test
+  public void testArgsParsing() throws BadFencingConfigurationException {
+    InetSocketAddress addr = new InetSocketAddress("bar.com", 1234);
+
+    Args args = new SshFenceByTcpPort.Args(addr, null);
+    assertEquals("bar.com", args.host);
+    assertEquals(1234, args.targetPort);
+    assertEquals(System.getProperty("user.name"), args.user);
+    assertEquals(22, args.sshPort);
+    
+    args = new SshFenceByTcpPort.Args(addr, "");
+    assertEquals("bar.com", args.host);
+    assertEquals(1234, args.targetPort);    
+    assertEquals(System.getProperty("user.name"), args.user);
+    assertEquals(22, args.sshPort);
+
+    args = new SshFenceByTcpPort.Args(addr, "12345");
+    assertEquals("bar.com", args.host);
+    assertEquals(1234, args.targetPort);
+    assertEquals("12345", args.user);
+    assertEquals(22, args.sshPort);
+
+    args = new SshFenceByTcpPort.Args(addr, ":12345");
+    assertEquals("bar.com", args.host);
+    assertEquals(1234, args.targetPort);
+    assertEquals(System.getProperty("user.name"), args.user);
+    assertEquals(12345, args.sshPort);
+
+    args = new SshFenceByTcpPort.Args(addr, "foo:8020");
+    assertEquals("bar.com", args.host);
+    assertEquals(1234, args.targetPort);
+    assertEquals("foo", args.user);
+    assertEquals(8020, args.sshPort);
+  }
+  
+  @Test
+  public void testBadArgsParsing() throws BadFencingConfigurationException {
+    assertBadArgs(":");          // No port specified
+    assertBadArgs("bar.com:");   // "
+    assertBadArgs(":xx");        // Port does not parse
+    assertBadArgs("bar.com:xx"); // "
+  }
+  
+  private void assertBadArgs(String argStr) {
+    InetSocketAddress addr = new InetSocketAddress("bar.com", 1234);
+    try {
+      new Args(addr, argStr);
+      fail("Did not fail on bad args: " + argStr);
+    } catch (BadFencingConfigurationException e) {
+      // Expected
+    }
+  }
+
+  private boolean isConfigured() {
+    return (TEST_FENCING_HOST != null && !TEST_FENCING_HOST.isEmpty()) &&
+           (TEST_FENCING_PORT != null && !TEST_FENCING_PORT.isEmpty()) &&
+           (TEST_KEYFILE != null && !TEST_KEYFILE.isEmpty());
+  }
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
index eec4797..4949ef3 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
@@ -25,21 +25,23 @@
 import org.apache.hadoop.io.retry.UnreliableImplementation.TypeOfExceptionToFailWith;
 import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
 import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.util.ThreadUtil;
 import org.junit.Test;
 
+@SuppressWarnings("unchecked")
 public class TestFailoverProxy {
 
-  public static class FlipFlopProxyProvider implements FailoverProxyProvider {
+  public static class FlipFlopProxyProvider<T> implements FailoverProxyProvider<T> {
     
-    private Class<?> iface;
-    private Object currentlyActive;
-    private Object impl1;
-    private Object impl2;
+    private Class<T> iface;
+    private T currentlyActive;
+    private T impl1;
+    private T impl2;
     
     private int failoversOccurred = 0;
     
-    public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
-        Object standbyImpl) {
+    public FlipFlopProxyProvider(Class<T> iface, T activeImpl,
+        T standbyImpl) {
       this.iface = iface;
       this.impl1 = activeImpl;
       this.impl2 = standbyImpl;
@@ -47,7 +49,7 @@
     }
     
     @Override
-    public Object getProxy() {
+    public T getProxy() {
       return currentlyActive;
     }
 
@@ -58,7 +60,7 @@
     }
 
     @Override
-    public Class<?> getInterface() {
+    public Class<T> getInterface() {
       return iface;
     }
 
@@ -126,7 +128,7 @@
         new FlipFlopProxyProvider(UnreliableInterface.class,
           new UnreliableImplementation("impl1"),
           new UnreliableImplementation("impl2")),
-        RetryPolicies.TRY_ONCE_DONT_FAIL);
+        RetryPolicies.TRY_ONCE_THEN_FAIL);
 
     unreliable.succeedsOnceThenFailsReturningString();
     try {
@@ -180,7 +182,7 @@
     
     assertEquals("impl1", unreliable.succeedsOnceThenFailsReturningString());
     try {
-      assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningString());
+      unreliable.succeedsOnceThenFailsReturningString();
       fail("should not have succeeded twice");
     } catch (IOException e) {
       // Make sure we *don't* fail over since the first implementation threw an
@@ -194,6 +196,27 @@
     assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
   }
   
+  /**
+   * Test that if a non-idempotent void function is called, and there is an exception,
+   * the exception is properly propagated
+   */
+  @Test
+  public void testExceptionPropagatedForNonIdempotentVoid() throws Exception {
+    UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
+    .create(UnreliableInterface.class,
+        new FlipFlopProxyProvider(UnreliableInterface.class,
+          new UnreliableImplementation("impl1", TypeOfExceptionToFailWith.IO_EXCEPTION),
+          new UnreliableImplementation("impl2", TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION)),
+        RetryPolicies.failoverOnNetworkException(1));
+
+    try {
+      unreliable.nonIdempotentVoidFailsIfIdentifierDoesntMatch("impl2");
+      fail("did not throw an exception");
+    } catch (Exception e) {
+    }
+
+  }
+  
   private static class SynchronizedUnreliableImplementation extends UnreliableImplementation {
     
     private CountDownLatch methodLatch;
@@ -267,4 +290,62 @@
     assertEquals("impl2", t2.result);
     assertEquals(1, proxyProvider.getFailoversOccurred());
   }
+
+  /**
+   * Ensure that when all configured services are throwing StandbyException
+   * that we fail over back and forth between them until one is no longer
+   * throwing StandbyException.
+   */
+  @Test
+  public void testFailoverBetweenMultipleStandbys()
+      throws UnreliableException, StandbyException, IOException {
+    
+    final long millisToSleep = 10000;
+    
+    final UnreliableImplementation impl1 = new UnreliableImplementation("impl1",
+        TypeOfExceptionToFailWith.STANDBY_EXCEPTION);
+    FlipFlopProxyProvider proxyProvider = new FlipFlopProxyProvider(
+        UnreliableInterface.class,
+        impl1,
+        new UnreliableImplementation("impl2",
+            TypeOfExceptionToFailWith.STANDBY_EXCEPTION));
+    
+    final UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
+      .create(UnreliableInterface.class, proxyProvider,
+          RetryPolicies.failoverOnNetworkException(
+              RetryPolicies.TRY_ONCE_THEN_FAIL, 10, 1000, 10000));
+    
+    new Thread() {
+      @Override
+      public void run() {
+        ThreadUtil.sleepAtLeastIgnoreInterrupts(millisToSleep);
+        impl1.setIdentifier("renamed-impl1");
+      }
+    }.start();
+    
+    String result = unreliable.failsIfIdentifierDoesntMatch("renamed-impl1");
+    assertEquals("renamed-impl1", result);
+  }
+  
+  /**
+   * Ensure that normal IO exceptions don't result in a failover.
+   */
+  @Test
+  public void testExpectedIOException() {
+    UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
+    .create(UnreliableInterface.class,
+        new FlipFlopProxyProvider(UnreliableInterface.class,
+          new UnreliableImplementation("impl1", TypeOfExceptionToFailWith.REMOTE_EXCEPTION),
+          new UnreliableImplementation("impl2", TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION)),
+          RetryPolicies.failoverOnNetworkException(
+              RetryPolicies.TRY_ONCE_THEN_FAIL, 10, 1000, 10000));
+    
+    try {
+      unreliable.failsIfIdentifierDoesntMatch("no-such-identifier");
+      fail("Should have thrown *some* exception");
+    } catch (Exception e) {
+      assertTrue("Expected IOE but got " + e.getClass(),
+          e instanceof IOException);
+    }
+  }
 }
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
index c48e87b..696f40d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.io.retry;
 
 import static org.apache.hadoop.io.retry.RetryPolicies.RETRY_FOREVER;
-import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_DONT_FAIL;
 import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_THEN_FAIL;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryByException;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryByRemoteException;
@@ -59,19 +58,6 @@
     }
   }
   
-  public void testTryOnceDontFail() throws UnreliableException {
-    UnreliableInterface unreliable = (UnreliableInterface)
-      RetryProxy.create(UnreliableInterface.class, unreliableImpl, TRY_ONCE_DONT_FAIL);
-    unreliable.alwaysSucceeds();
-    unreliable.failsOnceThenSucceeds();
-    try {
-      unreliable.failsOnceThenSucceedsWithReturnValue();
-      fail("Should fail");
-    } catch (UnreliableException e) {
-      // expected
-    }
-  }
-  
   public void testRetryForever() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl, RETRY_FOREVER);
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableImplementation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableImplementation.java
index 7fa88b3..54fe677 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableImplementation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableImplementation.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 
+import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 
@@ -37,7 +38,8 @@
   public static enum TypeOfExceptionToFailWith {
     UNRELIABLE_EXCEPTION,
     STANDBY_EXCEPTION,
-    IO_EXCEPTION
+    IO_EXCEPTION,
+    REMOTE_EXCEPTION
   }
   
   public UnreliableImplementation() {
@@ -48,6 +50,10 @@
     this(identifier, TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION);
   }
   
+  public void setIdentifier(String identifier) {
+    this.identifier = identifier;
+  }
+  
   public UnreliableImplementation(String identifier,
       TypeOfExceptionToFailWith exceptionToFailWith) {
     this.identifier = identifier;
@@ -91,14 +97,7 @@
     if (succeedsOnceThenFailsCount++ < 1) {
       return identifier;
     } else {
-      switch (exceptionToFailWith) {
-      case STANDBY_EXCEPTION:
-        throw new StandbyException(identifier);
-      case UNRELIABLE_EXCEPTION:
-        throw new UnreliableException(identifier);
-      case IO_EXCEPTION:
-        throw new IOException(identifier);
-      }
+      throwAppropriateException(exceptionToFailWith, identifier);
       return null;
     }
   }
@@ -109,16 +108,8 @@
     if (succeedsTenTimesThenFailsCount++ < 10) {
       return identifier;
     } else {
-      switch (exceptionToFailWith) {
-      case STANDBY_EXCEPTION:
-        throw new StandbyException(identifier);
-      case UNRELIABLE_EXCEPTION:
-        throw new UnreliableException(identifier);
-      case IO_EXCEPTION:
-        throw new IOException(identifier);
-      default:
-        throw new RuntimeException(identifier);
-      }
+      throwAppropriateException(exceptionToFailWith, identifier);
+      return null;
     }
   }
 
@@ -128,16 +119,8 @@
     if (succeedsOnceThenFailsIdempotentCount++ < 1) {
       return identifier;
     } else {
-      switch (exceptionToFailWith) {
-      case STANDBY_EXCEPTION:
-        throw new StandbyException(identifier);
-      case UNRELIABLE_EXCEPTION:
-        throw new UnreliableException(identifier);
-      case IO_EXCEPTION:
-        throw new IOException(identifier);
-      default:
-        throw new RuntimeException(identifier);
-      }
+      throwAppropriateException(exceptionToFailWith, identifier);
+      return null;
     }
   }
 
@@ -147,17 +130,38 @@
     if (this.identifier.equals(identifier)) {
       return identifier;
     } else {
-      switch (exceptionToFailWith) {
-      case STANDBY_EXCEPTION:
-        throw new StandbyException(identifier);
-      case UNRELIABLE_EXCEPTION:
-        throw new UnreliableException(identifier);
-      case IO_EXCEPTION:
-        throw new IOException(identifier);
-      default:
-        throw new RuntimeException(identifier);
-      }
+      String message = "expected '" + this.identifier + "' but received '" +
+          identifier + "'";
+      throwAppropriateException(exceptionToFailWith, message);
+      return null;
+    }
+  }
+  
+  @Override
+  public void nonIdempotentVoidFailsIfIdentifierDoesntMatch(String identifier)
+      throws UnreliableException, StandbyException, IOException {
+    if (this.identifier.equals(identifier)) {
+      return;
+    } else {
+      String message = "expected '" + this.identifier + "' but received '" +
+          identifier + "'";
+      throwAppropriateException(exceptionToFailWith, message);
     }
   }
 
+  private static void throwAppropriateException(TypeOfExceptionToFailWith eType,
+      String message) throws UnreliableException, StandbyException, IOException {
+    switch (eType) {
+    case STANDBY_EXCEPTION:
+      throw new StandbyException(message);
+    case UNRELIABLE_EXCEPTION:
+      throw new UnreliableException(message);
+    case IO_EXCEPTION:
+      throw new IOException(message);
+    case REMOTE_EXCEPTION:
+      throw new RemoteException(IOException.class.getName(), message);
+    default:
+      throw new RuntimeException(message);
+    }
+  }
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableInterface.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableInterface.java
index e794c16..66a8b85 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableInterface.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableInterface.java
@@ -67,4 +67,7 @@
   @Idempotent
   public String failsIfIdentifierDoesntMatch(String identifier)
       throws UnreliableException, StandbyException, IOException;
+
+  void nonIdempotentVoidFailsIfIdentifierDoesntMatch(String identifier)
+      throws UnreliableException, StandbyException, IOException;
 }
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index 1f3e67a..efb2dc1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -20,7 +20,9 @@
 
 import org.apache.commons.logging.*;
 
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
@@ -590,6 +592,38 @@
         Server.RECEIVED_HTTP_REQ_RESPONSE.getBytes());
   }
   
+  @Test
+  public void testConnectionRetriesOnSocketTimeoutExceptions() throws Exception {
+    Configuration conf = new Configuration();
+    // set max retries to 0
+    conf.setInt(
+      CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+      0);
+    assertRetriesOnSocketTimeouts(conf, 1);
+
+    // set max retries to 3
+    conf.setInt(
+      CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+      3);
+    assertRetriesOnSocketTimeouts(conf, 4);
+  }
+
+  private void assertRetriesOnSocketTimeouts(Configuration conf,
+      int maxTimeoutRetries) throws IOException, InterruptedException {
+    SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
+    doThrow(new SocketTimeoutException()).when(mockFactory).createSocket();
+    Client client = new Client(IntWritable.class, conf, mockFactory);
+    InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
+    try {
+      client.call(new IntWritable(RANDOM.nextInt()), address, null, null, 0,
+          conf);
+      fail("Not throwing the SocketTimeoutException");
+    } catch (SocketTimeoutException e) {
+      Mockito.verify(mockFactory, Mockito.times(maxTimeoutRetries))
+          .createSocket();
+    }
+  }
+  
   private void doIpcVersionTest(
       byte[] requestData,
       byte[] expectedResponse) throws Exception {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
new file mode 100644
index 0000000..3e59df7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
@@ -0,0 +1,252 @@
+Changes for HDFS-1623 branch.
+
+This change list will be merged into the trunk CHANGES.txt when the HDFS-1623
+branch is merged.
+------------------------------
+
+HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
+
+HDFS-1974. Introduce active and standy states to the namenode. (suresh)
+
+HDFS-2407. getServerDefaults and getStats don't check operation category (atm)
+
+HDFS-1973. HA: HDFS clients must handle namenode failover and switch over to the new active namenode. (atm)
+
+HDFS-2301. Start/stop appropriate namenode services when transition to active and standby states. (suresh)
+
+HDFS-2231. Configuration changes for HA namenode. (suresh)
+
+HDFS-2418. Change ConfiguredFailoverProxyProvider to take advantage of HDFS-2231. (atm)
+
+HDFS-2393. Mark appropriate methods of ClientProtocol with the idempotent annotation. (atm)
+
+HDFS-2523. Small NN fixes to include HAServiceProtocol and prevent NPE on shutdown. (todd)
+
+HDFS-2577. NN fails to start since it tries to start secret manager in safemode. (todd)
+
+HDFS-2582. Scope dfs.ha.namenodes config by nameservice (todd)
+
+HDFS-2591. MiniDFSCluster support to mix and match federation with HA (todd)
+
+HDFS-1975. Support for sharing the namenode state from active to standby. (jitendra, atm, todd)
+
+HDFS-1971. Send block report from datanode to both active and standby namenodes. (sanjay, todd via suresh)
+
+HDFS-2616. Change DatanodeProtocol#sendHeartbeat() to return HeartbeatResponse. (suresh)
+
+HDFS-2622. Fix TestDFSUpgrade in HA branch. (todd)
+
+HDFS-2612. Handle refreshNameNodes in federated HA clusters (todd)
+
+HDFS-2623. Add test case for hot standby capability (todd)
+
+HDFS-2626. BPOfferService.verifyAndSetNamespaceInfo needs to be synchronized (todd)
+
+HDFS-2624. ConfiguredFailoverProxyProvider doesn't correctly stop ProtocolTranslators (todd)
+
+HDFS-2625. TestDfsOverAvroRpc failing after introduction of HeartbeatResponse type (todd)
+
+HDFS-2627. Determine DN's view of which NN is active based on heartbeat responses (todd)
+
+HDFS-2634. Standby needs to ingest latest edit logs before transitioning to active (todd)
+
+HDFS-2671. NN should throw StandbyException in response to RPCs in STANDBY state (todd)
+
+HDFS-2680. DFSClient should construct failover proxy with exponential backoff (todd)
+
+HDFS-2683. Authority-based lookup of proxy provider fails if path becomes canonicalized (todd)
+
+HDFS-2689. HA: BookKeeperEditLogInputStream doesn't implement isInProgress() (atm)
+
+HDFS-2602. NN should log newly-allocated blocks without losing BlockInfo (atm)
+
+HDFS-2667. Fix transition from active to standby (todd)
+
+HDFS-2684. Fix up some failing unit tests on HA branch (todd)
+
+HDFS-2679. Add interface to query current state to HAServiceProtocol (eli via todd)
+
+HDFS-2677. Web UI should indicate the NN state. (eli via todd)
+
+HDFS-2678. When a FailoverProxyProvider is used, DFSClient should not retry connection ten times before failing over (atm via todd)
+
+HDFS-2682. When a FailoverProxyProvider is used, Client should not retry for 45 times if it is timing out to connect to server. (Uma Maheswara Rao G via todd)
+
+HDFS-2693. Fix synchronization issues around state transition (todd)
+
+HDFS-1972. Fencing mechanism for block invalidations and replications (todd)
+
+HDFS-2714. Fix test cases which use standalone FSNamesystems (todd)
+
+HDFS-2692. Fix bugs related to failover from/into safe mode. (todd)
+
+HDFS-2716. Configuration needs to allow different dfs.http.addresses for each HA NN (todd)
+
+HDFS-2720. Fix MiniDFSCluster HA support to work properly on Windows. (Uma Maheswara Rao G via todd)
+
+HDFS-2291. Allow the StandbyNode to make checkpoints in an HA setup. (todd)
+
+HDFS-2709. Appropriately handle error conditions in EditLogTailer (atm via todd)
+
+HDFS-2730. Refactor shared HA-related test code into HATestUtil class (todd)
+
+HDFS-2762. Fix TestCheckpoint timing out on HA branch. (Uma Maheswara Rao G via todd)
+
+HDFS-2724. NN web UI can throw NPE after startup, before standby state is entered. (todd)
+
+HDFS-2753. Fix standby getting stuck in safemode when blocks are written while SBN is down. (Hari Mankude and todd via todd)
+
+HDFS-2773. Reading edit logs from an earlier version should not leave blocks in under-construction state. (todd)
+
+HDFS-2775. Fix TestStandbyCheckpoints.testBothNodesInStandbyState failing intermittently. (todd)
+
+HDFS-2766. Test for case where standby partially reads log and then performs checkpoint. (atm)
+
+HDFS-2738. FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested. (atm)
+
+HDFS-2789. TestHAAdmin.testFailover is failing (eli)
+
+HDFS-2747. Entering safe mode after starting SBN can NPE. (Uma Maheswara Rao G via todd)
+
+HDFS-2772. On transition to active, standby should not swallow ELIE. (atm)
+
+HDFS-2767. ConfiguredFailoverProxyProvider should support NameNodeProtocol. (Uma Maheswara Rao G via todd)
+
+HDFS-2795. Standby NN takes a long time to recover from a dead DN starting up. (todd)
+
+HDFS-2592. Balancer support for HA namenodes. (Uma Maheswara Rao G via todd)
+
+HDFS-2367. Enable the configuration of multiple HA cluster addresses. (atm)
+
+HDFS-2812. When becoming active, the NN should treat all leases as freshly renewed. (todd)
+
+HDFS-2737. Automatically trigger log rolls periodically on the active NN. (todd and atm)
+
+HDFS-2820. Add a simple sanity check for HA config (todd)
+
+HDFS-2688. Add tests for quota tracking in an HA cluster. (todd)
+
+HDFS-2804. Should not mark blocks under-replicated when exiting safemode (todd)
+
+HDFS-2807. Service level authorizartion for HAServiceProtocol. (jitendra)
+
+HDFS-2809. Add test to verify that delegation tokens are honored after failover. (jitendra and atm)
+
+HDFS-2838. NPE in FSNamesystem when in safe mode. (Gregory Chanan via eli)
+
+HDFS-2805. Add a test for a federated cluster with HA NNs. (Brandon Li via jitendra)
+
+HDFS-2841. HAAdmin does not work if security is enabled. (atm)
+
+HDFS-2691. Fixes for pipeline recovery in an HA cluster: report RBW replicas immediately upon pipeline creation. (todd)
+
+HDFS-2824. Fix failover when prior NN died just after creating an edit log segment. (atm via todd)
+
+HDFS-2853. HA: NN fails to start if the shared edits dir is marked required (atm via eli)
+
+HDFS-2845. SBN should not allow browsing of the file system via web UI. (Bikas Saha via atm)
+
+HDFS-2742. HA: observed dataloss in replication stress test. (todd via eli)
+
+HDFS-2870. Fix log level for block debug info in processMisReplicatedBlocks (todd)
+
+HDFS-2859. LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs when the host is incorrect (Bikas Saha via todd)
+
+HDFS-2861. checkpointing should verify that the dfs.http.address has been configured to a non-loopback for peer NN (todd)
+
+HDFS-2860. TestDFSRollback#testRollback is failing. (atm)
+
+HDFS-2769. HA: When HA is enabled with a shared edits dir, that dir should be
+marked required. (atm via eli)
+
+HDFS-2863. Failures observed if dfs.edits.dir and shared.edits.dir have same directories. (Bikas Saha via atm)
+
+HDFS-2874. Edit log should log to shared dirs before local dirs. (todd)
+
+HDFS-2890. DFSUtil#getSuffixIDs should skip unset configurations. (atm)
+
+HDFS-2792. Make fsck work. (atm)
+
+HDFS-2808. HA: haadmin should use namenode ids. (eli)
+
+HDFS-2819. Document new HA-related configs in hdfs-default.xml. (eli)
+
+HDFS-2752. HA: exit if multiple shared dirs are configured. (eli)
+
+HDFS-2894. HA: automatically determine the nameservice Id if only one nameservice is configured. (eli)
+
+HDFS-2733. Document HA configuration and CLI. (atm)
+
+HDFS-2794. Active NN may purge edit log files before standby NN has a chance to read them (todd)
+
+HDFS-2901. Improvements for SBN web UI - not show under-replicated/missing blocks. (Brandon Li via jitendra)
+
+HDFS-2905. HA: Standby NN NPE when shared edits dir is deleted. (Bikas Saha via jitendra)
+
+HDFS-2579. Starting delegation token manager during safemode fails. (todd)
+
+HDFS-2510. Add HA-related metrics. (atm)
+
+HDFS-2924. Standby checkpointing fails to authenticate in secure cluster. (todd)
+
+HDFS-2915. HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition. (Bikas Saha via jitendra)
+
+HDFS-2912. Namenode not shutting down when shared edits dir is inaccessible. (Bikas Saha via atm)
+
+HDFS-2917. HA: haadmin should not work if run by regular user (eli)
+
+HDFS-2939. TestHAStateTransitions fails on Windows. (Uma Maheswara Rao G via atm)
+
+HDFS-2947. On startup NN throws an NPE in the metrics system. (atm)
+
+HDFS-2942. TestActiveStandbyElectorRealZK fails if build dir does not exist. (atm)
+
+HDFS-2948. NN throws NPE during shutdown if it fails to startup (todd)
+
+HDFS-2909. HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error. (Bikas Saha via jitendra)
+
+HDFS-2934. Allow configs to be scoped to all NNs in the nameservice. (todd)
+
+HDFS-2935. Shared edits dir property should be suffixed with nameservice and namenodeID (todd)
+
+HDFS-2928. ConfiguredFailoverProxyProvider should not create a NameNode proxy with an underlying retry proxy. (Uma Maheswara Rao G via atm)
+
+HDFS-2955. IllegalStateException during standby startup in getCurSegmentTxId. (Hari Mankude via atm)
+
+HDFS-2937. TestDFSHAAdmin needs tests with MiniDFSCluster. (Brandon Li via suresh)
+
+HDFS-2586. Add protobuf service and implementation for HAServiceProtocol. (suresh via atm)
+
+HDFS-2952. NN should not start with upgrade option or with a pending an unfinalized upgrade. (atm)
+
+HDFS-2974. MiniDFSCluster does not delete standby NN name dirs during format. (atm)
+
+HDFS-2929. Stress test and fixes for block synchronization (todd)
+
+HDFS-2972. Small optimization building incremental block report (todd)
+
+HDFS-2973. Re-enable NO_ACK optimization for block deletion. (todd)
+
+HDFS-2922. HA: close out operation categories (eli)
+
+HDFS-2993. HA: BackupNode#checkOperation should permit CHECKPOINT operations (eli)
+
+HDFS-2904. Client support for getting delegation tokens. (todd)
+
+HDFS-3013. HA: NameNode format doesn't pick up dfs.namenode.name.dir.NameServiceId configuration (Mingjie Lai via todd)
+
+HDFS-3019. Fix silent failure of TestEditLogJournalFailures (todd)
+
+HDFS-2958. Sweep for remaining proxy construction which doesn't go through failover path. (atm)
+
+HDFS-2920. fix remaining TODO items. (atm and todd)
+
+HDFS-3027. Implement a simple NN health check. (atm)
+
+HDFS-3023. Optimize entries in edits log for persistBlocks call. (todd)
+
+HDFS-2979. Balancer should use logical uri for creating failover proxy with HA enabled. (atm)
+
+HDFS-3035. Fix failure of TestFileAppendRestart due to OP_UPDATE_BLOCKS (todd)
+
+HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 709e52f..301d302 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -256,4 +256,12 @@
        <Field name="metrics" />
        <Bug pattern="IS2_INCONSISTENT_SYNC" />
      </Match>
+     <!--
+      This method isn't performance-critical and is much clearer to write as it's written.
+      -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.datanode.BlockPoolManager" />
+       <Method name="doRefreshNamenodes" />
+       <Bug category="PERFORMANCE" />
+     </Match>
  </FindBugsFilter>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 0b4da80..3f85de0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -387,6 +387,7 @@
         <configuration>
           <excludes>
             <exclude>CHANGES.txt</exclude>
+            <exclude>CHANGES.HDFS-1623.txt</exclude>
             <exclude>.idea/**</exclude>
             <exclude>src/main/conf/*</exclude>
             <exclude>src/main/docs/**</exclude>
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 707182e..636471a 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 @@
     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 7fa9026..047efd5 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 @@
     }
   }
 
+  // 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 @@
     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 b949bc2..5937fa8 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 @@
     out.close();
     bkjm.finalizeLogSegment(1, 100);
 
-    long numTrans = bkjm.getNumberOfTransactions(1);
+    long numTrans = bkjm.getNumberOfTransactions(1, true);
     assertEquals(100, numTrans);
   }
 
@@ -218,17 +218,17 @@
     }
     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 @@
     out.abort();
     out.close();
     
-    long numTrans = bkjm.getNumberOfTransactions(1);
+    long numTrans = bkjm.getNumberOfTransactions(1, true);
     assertEquals((txid-1), numTrans);
   }
 
@@ -357,7 +357,7 @@
     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 @@
     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 4c56bb3f..a01c939 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 @@
   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" = "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 5a45f51..82d0c36 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 @@
       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 359fd47..88b36b7 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.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.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.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.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 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 @@
    * 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 @@
     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 @@
    */
   private final Map<String, DFSOutputStream> filesBeingWritten
       = new HashMap<String, DFSOutputStream>();
+
   private boolean shortCircuitLocalReads;
   
   /**
@@ -247,59 +267,69 @@
   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<ClientProtocol> 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 @@
   
   /**
    * 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 Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
-    Token<DelegationTokenIdentifier> result =
+    assert dtService != null;
+    Token<DelegationTokenIdentifier> 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 @@
     @Override
     public long renew(Token<?> token, Configuration conf) throws IOException {
       Token<DelegationTokenIdentifier> delToken = 
-          (Token<DelegationTokenIdentifier>) token;
-      LOG.info("Renewing " + 
-               DelegationTokenIdentifier.stringifyToken(delToken));
-      ClientProtocol nn = 
-        DFSUtil.createNamenode
-           (SecurityUtil.getTokenServiceAddr(delToken),
-            conf, UserGroupInformation.getCurrentUser());
+        (Token<DelegationTokenIdentifier>) token;
+      ClientProtocol nn = getNNProxy(delToken, conf);
       try {
         return nn.renewDelegationToken(delToken);
       } catch (RemoteException re) {
@@ -647,9 +662,7 @@
           (Token<DelegationTokenIdentifier>) 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 @@
             AccessControlException.class);
       }
     }
+    
+    private static ClientProtocol getNNProxy(
+        Token<DelegationTokenIdentifier> 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<ClientProtocol> 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 fef6d8b9..4187f1c 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 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 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 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 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 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 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 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 7064616..cbc0f0e 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.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.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> RANDOM = new ThreadLocal<Random>() {
     @Override
     protected Random initialValue() {
@@ -101,13 +106,20 @@
           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,13 +316,39 @@
   /**
    * 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<String> getNameServiceIds(Configuration conf) {
-    return conf.getStringCollection(DFS_FEDERATION_NAMESERVICES);
+    return conf.getTrimmedStringCollection(DFS_FEDERATION_NAMESERVICES);
   }
 
   /**
+   * @return <code>coll</code> if it is non-null and non-empty. Otherwise,
+   * returns a list with a single null value.
+   */
+  private static Collection<String> emptyAsSingletonNull(Collection<String> 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<String> 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.
    * @param defaultValue default value to return, when key was not found
@@ -323,9 +361,7 @@
       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 @@
     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<InetSocketAddress> 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<String, Map<String, InetSocketAddress>>
+    getAddresses(Configuration conf,
       String defaultAddress, String... keys) {
     Collection<String> nameserviceIds = getNameServiceIds(conf);
-    List<InetSocketAddress> isas = new ArrayList<InetSocketAddress>();
-
-    // 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 <key>[.<nameserviceId>][.<namenodeId>]
+    // across all of the configured nameservices and namenodes.
+    Map<String, Map<String, InetSocketAddress>> ret = Maps.newHashMap();
+    for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
+      Map<String, InetSocketAddress> isas =
+        getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
+      if (!isas.isEmpty()) {
+        ret.put(nsId, isas);
       }
     }
-    return isas;
+    return ret;
+  }
+
+  private static Map<String, InetSocketAddress> getAddressesForNameserviceId(
+      Configuration conf, String nsId, String defaultValue,
+      String[] keys) {
+    Collection<String> nnIds = getNameNodeIds(conf, nsId);
+    Map<String, InetSocketAddress> 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<String, Map<String, InetSocketAddress>> getHaNnRpcAddresses(
+      Configuration conf) {
+    return getAddresses(conf, null, DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
   }
   
   /**
@@ -377,11 +461,11 @@
    * @return list of InetSocketAddresses
    * @throws IOException on error
    */
-  public static List<InetSocketAddress> getBackupNodeAddresses(
+  public static Map<String, Map<String, InetSocketAddress>> getBackupNodeAddresses(
       Configuration conf) throws IOException {
-    List<InetSocketAddress> addressList = getAddresses(conf,
+    Map<String, Map<String, InetSocketAddress>> 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 @@
    * @return list of InetSocketAddresses
    * @throws IOException on error
    */
-  public static List<InetSocketAddress> getSecondaryNameNodeAddresses(
+  public static Map<String, Map<String, InetSocketAddress>> getSecondaryNameNodeAddresses(
       Configuration conf) throws IOException {
-    List<InetSocketAddress> addressList = getAddresses(conf, null,
+    Map<String, Map<String, InetSocketAddress>> 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 @@
    * @return list of InetSocketAddress
    * @throws IOException on error
    */
-  public static List<InetSocketAddress> getNNServiceRpcAddresses(
+  public static Map<String, Map<String, InetSocketAddress>> getNNServiceRpcAddresses(
       Configuration conf) throws IOException {
     // Use default address as fall back
     String defaultAddress;
@@ -430,9 +514,10 @@
       defaultAddress = null;
     }
     
-    List<InetSocketAddress> addressList = getAddresses(conf, defaultAddress,
+    Map<String, Map<String, InetSocketAddress>> 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 @@
   }
   
   /**
-   * 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<ConfiguredNNAddress> flattenAddressMap(
+      Map<String, Map<String, InetSocketAddress>> map) {
+    List<ConfiguredNNAddress> ret = Lists.newArrayList();
+    
+    for (Map.Entry<String, Map<String, InetSocketAddress>> entry :
+      map.entrySet()) {
+      String nsId = entry.getKey();
+      Map<String, InetSocketAddress> nnMap = entry.getValue();
+      for (Map.Entry<String, InetSocketAddress> 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<String, Map<String, InetSocketAddress>> map) {
+    StringBuilder b = new StringBuilder();
+    for (Map.Entry<String, Map<String, InetSocketAddress>> entry :
+         map.entrySet()) {
+      String nsId = entry.getKey();
+      Map<String, InetSocketAddress> nnMap = entry.getValue();
+      b.append("Nameservice <").append(nsId).append(">:").append("\n");
+      for (Map.Entry<String, InetSocketAddress> 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<String, Map<String, InetSocketAddress>> 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<URI> 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<URI> getNameServiceUris(Configuration conf,
+      String... keys) {
+    Set<URI> ret = new HashSet<URI>();
+    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 @@
    *     not the NameServiceId-suffixed keys.
    * @return nameserviceId, or null if no match found
    */
-  public static String getNameServiceIdFromAddress(Configuration conf, 
-      InetSocketAddress address, String... keys) {
-    Collection<String> 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 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 @@
       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 @@
   }
   
   /**
-   * 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 @@
   }
   
   /**
-   * 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 @@
    * 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 @@
     if (nameserviceId != null) {
       return nameserviceId;
     }
-    
-    Collection<String> ids = getNameServiceIds(conf);
-    if (ids == null || ids.size() == 0) {
-      // Not federation configuration, hence no nameservice Id
-      return null;
+    Collection<String> 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}.<nameservice Id>.<namenode Id>
+   * 
+   * @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<String> nsIds = getNameServiceIds(conf);
+    for (String nsId : emptyAsSingletonNull(nsIds)) {
+      if (knownNsId != null && !knownNsId.equals(nsId)) {
+        continue;
+      }
+      
+      Collection<String> 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 @@
     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<String> 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 119bca9..8dfced3 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 @@
       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 0000000..34e9d2e
--- /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<String, Map<String, InetSocketAddress>> addresses =
+      DFSUtil.getHaNnRpcAddresses(conf);
+    if (addresses == null) return false;
+    Map<String, InetSocketAddress> 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<String> 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<String> 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<DelegationTokenIdentifier> 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<DelegationTokenIdentifier> 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<DelegationTokenIdentifier> haToken =
+        DelegationTokenSelector.selectHdfsDelegationToken(haService, ugi);
+    if (haToken == null) {
+      // no token
+      return;
+    }
+    Token<DelegationTokenIdentifier> specificToken =
+        new Token<DelegationTokenIdentifier>(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 27702b5..6e21245 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 @@
 
 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 @@
     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 0000000..650c313
--- /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<PROXYTYPE> {
+    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 <T> ProxyAndInfo<T> createProxy(Configuration conf,
+      URI nameNodeUri, Class<T> xface) throws IOException {
+    Class<FailoverProxyProvider<T>> 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<T> 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<T>(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 <T> ProxyAndInfo<T> createNonHAProxy(
+      Configuration conf, InetSocketAddress nnAddr, Class<T> 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<T>(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<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap 
+                     = new HashMap<Class<? extends Exception>, RetryPolicy>();
+      RetryPolicy methodPolicy = RetryPolicies.retryByException(timeoutPolicy,
+          exceptionToPolicyMap);
+      Map<String, RetryPolicy> methodNameToPolicyMap 
+                     = new HashMap<String, RetryPolicy>();
+      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<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 
+                 = new HashMap<Class<? extends Exception>, RetryPolicy>();
+      remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
+          createPolicy);
+    
+      Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap
+                 = new HashMap<Class<? extends Exception>, RetryPolicy>();
+      exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
+          .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+              remoteExceptionToPolicyMap));
+      RetryPolicy methodPolicy = RetryPolicies.retryByException(
+          RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+      Map<String, RetryPolicy> methodNameToPolicyMap 
+                 = new HashMap<String, RetryPolicy>();
+    
+      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 <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
+      Configuration conf, URI nameNodeUri, Class<T> 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<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) 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 <T> FailoverProxyProvider<T> createFailoverProxyProvider(
+      Configuration conf, Class<FailoverProxyProvider<T>> failoverProxyProviderClass,
+      Class<T> xface, URI nameNodeUri) throws IOException {
+    Preconditions.checkArgument(
+        xface.isAssignableFrom(NamenodeProtocols.class),
+        "Interface %s is not a NameNode protocol", xface);
+    try {
+      Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass
+          .getConstructor(Configuration.class, URI.class, Class.class);
+      FailoverProxyProvider<?> provider = ctor.newInstance(conf, nameNodeUri,
+          xface);
+      return (FailoverProxyProvider<T>) 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 e1006a6..58af5fd 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 @@
  * - 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 @@
     blockList[idx+1] = -1;
     blockList[idx+2] = -1;
   }
+
+  public long getMaxGsInBlockList() {
+    long maxGs = -1;
+    Iterator<Block> 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 ab6babc..099fd28 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.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 @@
    * @throws UnresolvedLinkException If <code>src</code> 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 @@
    * @return a set of server default configuration values
    * @throws IOException
    */
+  @Idempotent
   public FsServerDefaults getServerDefaults() throws IOException;
 
   /**
@@ -228,6 +231,7 @@
    * @throws UnresolvedLinkException if <code>src</code> 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 @@
    * @throws UnresolvedLinkException If <code>src</code> 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 @@
    * @throws UnresolvedLinkException If <code>src</code> 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 @@
    * @throws UnresolvedLinkException If <code>src</code> 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 @@
    * locations on datanodes).
    * @param blocks Array of located blocks to report
    */
+  @Idempotent
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
 
   ///////////////////////////////////////
@@ -482,6 +490,7 @@
    * RunTimeExceptions:
    * @throws InvalidPathException If <code>src</code> is invalid
    */
+  @Idempotent
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws AccessControlException, FileAlreadyExistsException,
       FileNotFoundException, NSQuotaExceededException,
@@ -502,6 +511,7 @@
    * @throws UnresolvedLinkException If <code>src</code> 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 @@
    * @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 @@
    * @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 @@
   
   /**
    * Get a set of statistics about the filesystem.
-   * Right now, only three values are returned.
+   * Right now, only seven values are returned.
    * <ul>
    * <li> [0] contains the total storage capacity of the system, in bytes.</li>
    * <li> [1] contains the total used space of the system, in bytes.</li>
@@ -567,6 +579,7 @@
    * 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 @@
    * 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 @@
    * @throws IOException
    * @throws UnresolvedLinkException if the path contains a symlink. 
    */
+  @Idempotent
   public long getPreferredBlockSize(String filename) 
       throws IOException, UnresolvedLinkException;
 
@@ -700,9 +715,9 @@
    * 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 @@
    * @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 @@
    * @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 @@
    * @throws UnresolvedLinkException if <code>src</code> 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 @@
    * @throws UnresolvedLinkException if <code>path</code> 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 @@
    * @throws UnresolvedLinkException if the <code>path</code> 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 @@
    * @throws UnresolvedLinkException if <code>src</code> 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 @@
    * @throws UnresolvedLinkException if <code>src</code> 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 @@
    * @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 @@
    * @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 @@
    * @return Token<DelegationTokenIdentifier>
    * @throws IOException
    */
+  @Idempotent
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
       throws IOException;
 
@@ -906,6 +931,7 @@
    * @return the new expiration time
    * @throws IOException
    */
+  @Idempotent
   public long renewDelegationToken(Token<DelegationTokenIdentifier> 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 6b4835f..da64b9e 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
@@ -100,6 +100,14 @@
   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.
    */
   public static final int LAYOUT_VERSION = LayoutVersion
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 729748f..3680ee5 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 @@
     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 f6a63fb..7382543 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.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 @@
 @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 @@
         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 3691584..46a3c82 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 @@
 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.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 @@
 @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<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap 
-        = new HashMap<Class<? extends Exception>, RetryPolicy>();
-    remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
-        createPolicy);
-
-    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap = 
-        new HashMap<Class<? extends Exception>, RetryPolicy>();
-    exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
-        .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
-            remoteExceptionToPolicyMap));
-    RetryPolicy methodPolicy = RetryPolicies.retryByException(
-        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
-    Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
-
-    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 @@
         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 d47eac2..2a661c0 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.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.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 @@
   }
 
   @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 @@
       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 413bd3a..c653daa 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.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 @@
   @Override
   public HeartbeatResponseProto sendHeartbeat(RpcController controller,
       HeartbeatRequestProto request) throws ServiceException {
-    DatanodeCommand[] cmds = null;
+    HeartbeatResponse response;
     try {
       List<StorageReportProto> list = request.getReportsList();
       StorageReport[] report = new StorageReport[list.size()];
@@ -113,7 +114,7 @@
             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 @@
     }
     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 @@
         }
       }
     }
+    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 c29595e..01bd88e 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 @@
 
 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 @@
   /** 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 0735cfd..76ca46f 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 @@
 
 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 @@
   /** 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 f2ec7ba..7de2c0e4 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 @@
 
 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.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 @@
       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 @@
   }
 
   @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 fab9f1f..b1e7be0 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.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.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 @@
     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 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 @@
         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 0fcf424..96ba2cf 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 @@
 
 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 @@
   /** 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 eb8e059..6f07617 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 @@
 
 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 @@
   /** 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 b5f24d1..ba62a2c 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 @@
 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 @@
   @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 1822b27..4f73b85 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 @@
         new InetSocketAddress(nnAddr.getHostName(), nnRpcPort));
     return INSTANCE.selectToken(serviceName, ugi.getTokens());
   }
+  
+  public static Token<DelegationTokenIdentifier> 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 a0146e7..e808af6 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.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 @@
    * for each namenode,
    * execute a {@link Balancer} to work through all datanodes once.  
    */
-  static int run(List<InetSocketAddress> namenodes, final Parameters p,
+  static int run(Collection<URI> namenodes, final Parameters p,
       Configuration conf) throws IOException, InterruptedException {
     final long sleeptime = 2000*conf.getLong(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
@@ -1393,8 +1393,8 @@
     final List<NameNodeConnector> connectors
         = new ArrayList<NameNodeConnector>(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 @@
       try {
         checkReplicationPolicyCompatibility(conf);
 
-        final List<InetSocketAddress> namenodes = DFSUtil.getNNServiceRpcAddresses(conf);
+        final Collection<URI> 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 83822e4..c4208b7 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.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 @@
   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 @@
   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 @@
 
   @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<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
-        new HashMap<Class<? extends Exception>, RetryPolicy>();
-    RetryPolicy methodPolicy = RetryPolicies.retryByException(
-        timeoutPolicy, exceptionToPolicyMap);
-    Map<String,RetryPolicy> methodNameToPolicyMap =
-        new HashMap<String, RetryPolicy>();
-    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 58725a6..ce3ff8b 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 @@
   /**
    * 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 c8f36a0..1c9b2aa 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.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.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.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 @@
   /** 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 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 @@
 
   /** 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<Block> postponedMisreplicatedBlocks = Sets.newHashSet();
 
   //
   // Keeps a TreeSet for every named node. Each treeset contains
@@ -316,49 +347,15 @@
       out.println("Metasave: Blocks waiting for replication: " + 
                   neededReplications.size());
       for (Block block : neededReplications) {
-        List<DatanodeDescriptor> containingNodes =
-                                          new ArrayList<DatanodeDescriptor>();
-        List<DatanodeDescriptor> containingLiveReplicasNodes =
-          new ArrayList<DatanodeDescriptor>();
-        
-        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<DatanodeDescriptor> corruptNodes = 
-                                      corruptReplicas.getNodes(block);
-        
-        for (Iterator<DatanodeDescriptor> 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 @@
     // 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<DatanodeDescriptor> containingNodes =
+                                      new ArrayList<DatanodeDescriptor>();
+    List<DatanodeDescriptor> containingLiveReplicasNodes =
+      new ArrayList<DatanodeDescriptor>();
+    
+    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<DatanodeDescriptor> corruptNodes = 
+                                  corruptReplicas.getNodes(block);
+    
+    for (Iterator<DatanodeDescriptor> 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 @@
     
     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 @@
    * 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 @@
     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.<p>
    * The block is converted only if the file has blocks and the last one
@@ -508,6 +576,14 @@
       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 @@
     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<DatanodeDescriptor> it = blocksMap.nodeIterator(blk);
           it.hasNext();) {
         final DatanodeDescriptor d = it.next();
@@ -608,6 +684,12 @@
           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 @@
 
     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 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 @@
           + " 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 @@
     }
   }
 
+  private void postponeBlock(Block blk) {
+    if (postponedMisreplicatedBlocks.add(blk)) {
+      postponedMisreplicatedBlocksCount++;
+    }
+  }
+  
+  
   void updateState() {
     pendingReplicationBlocksCount = pendingReplications.size();
     underReplicatedBlocksCount = neededReplications.size();
@@ -930,7 +1030,7 @@
    *
    * @return number of blocks scheduled for replication during this iteration.
    */
-  private int computeReplicationWork(int blocksToProcess) throws IOException {
+  int computeReplicationWork(int blocksToProcess) throws IOException {
     List<List<Block>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
@@ -981,8 +1081,10 @@
             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 @@
         srcNode = node;
     }
     if(numReplicas != null)
-      numReplicas.initialize(live, decommissioned, corrupt, excess);
+      numReplicas.initialize(live, decommissioned, corrupt, excess, 0);
     return srcNode;
   }
 
@@ -1314,7 +1416,7 @@
 
       // 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 @@
       } 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 @@
         + ", processing time: " + (endTime - startTime) + " msecs");
   }
 
+  /**
+   * Rescan the list of blocks which were previously postponed.
+   */
+  private void rescanPostponedMisreplicatedBlocks() {
+    for (Iterator<Block> 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 @@
     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 @@
       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 @@
    * @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 @@
           + " 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 @@
 
     // 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 @@
     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<ReportedBlockInfo> queue = pendingDNMessages.takeBlockQueue(b);
+    if (queue == null) {
+      // Nothing to re-process
+      return;
+    }
+    processQueuedMessages(queue);
+  }
+  
+  private void processQueuedMessages(Iterable<ReportedBlockInfo> 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 @@
     // 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 @@
       + 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 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 @@
     for (Iterator<DatanodeDescriptor> 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<Block> excessBlocks = excessReplicateMap.get(cur
           .getStorageID());
       if (excessBlocks == null || !excessBlocks.contains(block)) {
@@ -2151,13 +2449,19 @@
     // 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<BlockInfo> toAdd = new LinkedList<BlockInfo>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
-    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 @@
     }
   }
 
-  /** 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<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
     while (nodeIter.hasNext()) {
@@ -2241,7 +2566,7 @@
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
         corrupt++;
       } else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-        count++;
+        decommissioned++;
       } else {
         LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
             .getStorageID());
@@ -2251,8 +2576,11 @@
           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 int getActiveBlockCount() {
-    return blocksMap.size() - (int)invalidateBlocks.numBlocks();
+    return blocksMap.size();
   }
 
   public DatanodeDescriptor[] getNodes(BlockInfo block) {
@@ -2397,10 +2725,17 @@
   }
 
   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 @@
       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 @@
     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 @@
     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 @@
       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 ac1a7e6..058d2e3 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 @@
     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 d927f05..984456f 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 @@
     boolean contains(E e) {
       return blockq.contains(e);
     }
+
+    synchronized void clear() {
+      blockq.clear();
+    }
   }
 
   private volatile BlockInfo blockList = null;
@@ -103,6 +107,24 @@
   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 <newbandwidth>", at which point the
@@ -129,6 +151,10 @@
   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 @@
     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 @@
     this.lastUpdate = System.currentTimeMillis();
     this.xceiverCount = xceiverCount;
     this.volumeFailures = volFailures;
+    this.heartbeatedSinceFailover = true;
     rollBlocksScheduled(lastUpdate);
   }
 
@@ -564,5 +599,41 @@
     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 5d795e7..8c59ccb 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 @@
       }
     }
 
-    return null;
+    return new DatanodeCommand[0];
   }
 
   /**
@@ -947,4 +947,27 @@
       }
     }
   }
+  
+  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 2c6b46f..5c7e0bd 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 @@
     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 52f6258..9e5c8df 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 @@
   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 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 0000000..b7da116
--- /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<Block, Queue<ReportedBlockInfo>> 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<ReportedBlockInfo> takeBlockQueue(Block block) {
+    Queue<ReportedBlockInfo> queue = queueByBlockId.remove(block);
+    if (queue != null) {
+      count -= queue.size();
+    }
+    return queue;
+  }
+
+
+  private Queue<ReportedBlockInfo> getBlockQueue(Block block) {
+    Queue<ReportedBlockInfo> 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<Block, Queue<ReportedBlockInfo>> 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<ReportedBlockInfo> takeAll() {
+    List<ReportedBlockInfo> rbis = Lists.newArrayListWithCapacity(
+        count);
+    for (Queue<ReportedBlockInfo> 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 e07cf9b..e200ed0 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 @@
     }
   }
 
+
+  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 3de9067..c76d24c 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 @@
      * <p> Locking is not supported by all file systems.
      * E.g., NFS does not consistently support exclusive locks.
      * 
-     * <p> If locking is supported we guarantee exculsive access to the
+     * <p> 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 642551e..1f4e974 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.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 @@
    * @param names collection of strings to convert to URIs
    * @return collection of URIs
    */
-  public static Collection<URI> stringCollectionAsURIs(
+  public static List<URI> stringCollectionAsURIs(
                                   Collection<String> names) {
-    Collection<URI> uris = new ArrayList<URI>(names.size());
+    List<URI> uris = new ArrayList<URI>(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 5b1ed7c..27567b5 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.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.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:
- * <ul>
- * <li> Pre-registration handshake with namenode</li>
- * <li> Registration with namenode</li>
- * <li> Send periodic heartbeats to the namenode</li>
- * <li> Handle commands received from the namenode</li>
- * </ul>
+ * 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 @@
    */
   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<ReceivedDeletedBlockInfo> receivedAndDeletedBlockList 
-    = new LinkedList<ReceivedDeletedBlockInfo>();
-  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<BPServiceActor> bpServices =
+    new CopyOnWriteArrayList<BPServiceActor>();
+
+  /**
+   * 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<InetSocketAddress> 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<InetSocketAddress> addrs) throws IOException {
+    Set<InetSocketAddress> oldAddrs = Sets.newHashSet();
+    for (BPServiceActor actor : bpServices) {
+      oldAddrs.add(actor.getNNSocketAddress());
+    }
+    Set<InetSocketAddress> 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 @@
     }
   }
   
-  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 @@
         storageId = "unknown";
       }
       return "Block pool <registering> (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 <code>versionRequest</code> 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<URI> 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
-   * <p>
-   * 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<BPServiceActor> 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 @@
    * @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 @@
       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 @@
     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 @@
     }
     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 0000000..75f32cb
--- /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:
+ * <ul>
+ * <li> Pre-registration handshake with namenode</li>
+ * <li> Registration with namenode</li>
+ * <li> Send periodic heartbeats to the namenode</li>
+ * <li> Handle commands received from the namenode</li>
+ * </ul>
+ */
+@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<Long, ReceivedDeletedBlockInfo> 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 <code>versionRequest</code> 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<URI> 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
+   * <p>
+   * 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 0000000..3355ee2
--- /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<String, BPOfferService> bpByNameserviceId =
+    Maps.newHashMap();
+  private final Map<String, BPOfferService> bpByBlockPoolId =
+    Maps.newHashMap();
+  private final List<BPOfferService> 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<BPOfferService> 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<Object>() {
+            public Object run() throws Exception {
+              for (BPOfferService bpos : offerServices) {
+                bpos.start();
+              }
+              return null;
+            }
+          });
+    } catch (InterruptedException ex) {
+      IOException ioe = new IOException();
+      ioe.initCause(ex.getCause());
+      throw ioe;
+    }
+  }
+  
+  void joinAll() {
+    for (BPOfferService bpos: this.getAllNamenodeThreads()) {
+      bpos.join();
+    }
+  }
+  
+  void refreshNamenodes(Configuration conf)
+      throws IOException {
+    LOG.info("Refresh request received for nameservices: "
+        + conf.get(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES));
+    
+    Map<String, Map<String, InetSocketAddress>> newAddressMap = 
+      DFSUtil.getNNServiceRpcAddresses(conf);
+    
+    synchronized (refreshNamenodesLock) {
+      doRefreshNamenodes(newAddressMap);
+    }
+  }
+  
+  private void doRefreshNamenodes(
+      Map<String, Map<String, InetSocketAddress>> addrMap) throws IOException {
+    assert Thread.holdsLock(refreshNamenodesLock);
+
+    Set<String> toRefresh = Sets.newHashSet();
+    Set<String> toAdd = Sets.newHashSet();
+    Set<String> toRemove;
+    
+    synchronized (this) {
+      // Step 1. For each of the new nameservices, figure out whether
+      // it's an update of the set of NNs for an existing NS,
+      // or an entirely new nameservice.
+      for (String nameserviceId : addrMap.keySet()) {
+        if (bpByNameserviceId.containsKey(nameserviceId)) {
+          toRefresh.add(nameserviceId);
+        } else {
+          toAdd.add(nameserviceId);
+        }
+      }
+      
+      // Step 2. Any nameservices we currently have but are no longer present
+      // need to be removed.
+      toRemove = Sets.newHashSet(Sets.difference(
+          bpByNameserviceId.keySet(), addrMap.keySet()));
+      
+      assert toRefresh.size() + toAdd.size() ==
+        addrMap.size() :
+          "toAdd: " + Joiner.on(",").useForNull("<default>").join(toAdd) +
+          "  toRemove: " + Joiner.on(",").useForNull("<default>").join(toRemove) +
+          "  toRefresh: " + Joiner.on(",").useForNull("<default>").join(toRefresh);
+
+      
+      // Step 3. Start new nameservices
+      if (!toAdd.isEmpty()) {
+        LOG.info("Starting BPOfferServices for nameservices: " +
+            Joiner.on(",").useForNull("<default>").join(toAdd));
+      
+        for (String nsToAdd : toAdd) {
+          ArrayList<InetSocketAddress> addrs =
+            Lists.newArrayList(addrMap.get(nsToAdd).values());
+          BPOfferService bpos = createBPOS(addrs);
+          bpByNameserviceId.put(nsToAdd, bpos);
+          offerServices.add(bpos);
+        }
+      }
+      startAll();
+    }
+
+    // Step 4. Shut down old nameservices. This happens outside
+    // of the synchronized(this) lock since they need to call
+    // back to .remove() from another thread
+    if (!toRemove.isEmpty()) {
+      LOG.info("Stopping BPOfferServices for nameservices: " +
+          Joiner.on(",").useForNull("<default>").join(toRemove));
+      
+      for (String nsToRemove : toRemove) {
+        BPOfferService bpos = bpByNameserviceId.get(nsToRemove);
+        bpos.stop();
+        bpos.join();
+        // they will call remove on their own
+      }
+    }
+    
+    // Step 5. Update nameservices whose NN list has changed
+    if (!toRefresh.isEmpty()) {
+      LOG.info("Refreshing list of NNs for nameservices: " +
+          Joiner.on(",").useForNull("<default>").join(toRefresh));
+      
+      for (String nsToRefresh : toRefresh) {
+        BPOfferService bpos = bpByNameserviceId.get(nsToRefresh);
+        ArrayList<InetSocketAddress> addrs =
+          Lists.newArrayList(addrMap.get(nsToRefresh).values());
+        bpos.refreshNNList(addrs);
+      }
+    }
+  }
+
+  /**
+   * Extracted out for test purposes.
+   */
+  protected BPOfferService createBPOS(List<InetSocketAddress> nnAddrs) {
+    return new BPOfferService(nnAddrs, dn);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 153fd93..fd25c1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -153,6 +153,7 @@
         switch (stage) {
         case PIPELINE_SETUP_CREATE:
           replicaInfo = datanode.data.createRbw(block);
+          datanode.notifyNamenodeReceivingBlock(block);
           break;
         case PIPELINE_SETUP_STREAMING_RECOVERY:
           replicaInfo = datanode.data.recoverRbw(
@@ -166,6 +167,7 @@
                 block.getLocalBlock());
           }
           block.setGenerationStamp(newGs);
+          datanode.notifyNamenodeReceivingBlock(block);
           break;
         case PIPELINE_SETUP_APPEND_RECOVERY:
           replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
@@ -174,6 +176,7 @@
                 block.getLocalBlock());
           }
           block.setGenerationStamp(newGs);
+          datanode.notifyNamenodeReceivingBlock(block);
           break;
         case TRANSFER_RBW:
         case TRANSFER_FINALIZED:
@@ -320,7 +323,6 @@
   private void verifyChunks( byte[] dataBuf, int dataOff, int len, 
                              byte[] checksumBuf, int checksumOff ) 
                              throws IOException {
-    DatanodeProtocol nn = datanode.getBPNamenode(block.getBlockPoolId());
     while (len > 0) {
       int chunkLen = Math.min(len, bytesPerChecksum);
       
@@ -331,9 +333,7 @@
           try {
             LOG.info("report corrupt block " + block + " from datanode " +
                       srcDataNode + " to namenode");
-            LocatedBlock lb = new LocatedBlock(block, 
-                                            new DatanodeInfo[] {srcDataNode});
-            nn.reportBadBlocks(new LocatedBlock[] {lb});
+            datanode.reportRemoteBadBlock(srcDataNode, block);
           } catch (IOException e) {
             LOG.warn("Failed to report bad block " + block + 
                       " from datanode " + srcDataNode + " to namenode");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 5681525..098809c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -45,7 +45,6 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
 
 import java.io.BufferedOutputStream;
@@ -86,6 +85,7 @@
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -164,6 +164,8 @@
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
 import com.google.protobuf.BlockingService;
 
 
@@ -230,143 +232,6 @@
     return NetUtils.createSocketAddr(target);
   }
   
-  /**
-   * Manages he 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 final Map<String, BPOfferService> bpMapping;
-    private final Map<InetSocketAddress, BPOfferService> nameNodeThreads;
- 
-    //This lock is used only to ensure exclusion of refreshNamenodes
-    private final Object refreshNamenodesLock = new Object();
-    
-    BlockPoolManager(Configuration conf)
-        throws IOException {
-      bpMapping = new HashMap<String, BPOfferService>();
-      nameNodeThreads = new HashMap<InetSocketAddress, BPOfferService>();
-  
-      List<InetSocketAddress> isas = DFSUtil.getNNServiceRpcAddresses(conf);
-      for(InetSocketAddress isa : isas) {
-        BPOfferService bpos = new BPOfferService(isa, DataNode.this);
-        nameNodeThreads.put(bpos.getNNSocketAddress(), bpos);
-      }
-    }
-    
-    synchronized void addBlockPool(BPOfferService t) {
-      if (nameNodeThreads.get(t.getNNSocketAddress()) == null) {
-        throw new IllegalArgumentException(
-            "Unknown BPOfferService thread for namenode address:"
-                + t.getNNSocketAddress());
-      }
-      if (t.getBlockPoolId() == null) {
-        throw new IllegalArgumentException("Null blockpool id");
-      }
-      bpMapping.put(t.getBlockPoolId(), t);
-    }
-    
-    /**
-     * Returns the array of BPOfferService objects. 
-     * Caution: The BPOfferService returned could be shutdown any time.
-     */
-    synchronized BPOfferService[] getAllNamenodeThreads() {
-      BPOfferService[] bposArray = new BPOfferService[nameNodeThreads.values()
-          .size()];
-      return nameNodeThreads.values().toArray(bposArray);
-    }
-    
-    synchronized BPOfferService get(InetSocketAddress addr) {
-      return nameNodeThreads.get(addr);
-    }
-    
-    synchronized BPOfferService get(String bpid) {
-      return bpMapping.get(bpid);
-    }
-    
-    synchronized void remove(BPOfferService t) {
-      nameNodeThreads.remove(t.getNNSocketAddress());
-      bpMapping.remove(t.getBlockPoolId());
-    }
-    
-    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<Object>() {
-              public Object run() throws Exception {
-                for (BPOfferService bpos : nameNodeThreads.values()) {
-                  bpos.start();
-                }
-                return null;
-              }
-            });
-      } catch (InterruptedException ex) {
-        IOException ioe = new IOException();
-        ioe.initCause(ex.getCause());
-        throw ioe;
-      }
-    }
-    
-    void joinAll() {
-      for (BPOfferService bpos: this.getAllNamenodeThreads()) {
-        bpos.join();
-      }
-    }
-    
-    void refreshNamenodes(Configuration conf)
-        throws IOException {
-      LOG.info("Refresh request received for nameservices: "
-          + conf.get(DFS_FEDERATION_NAMESERVICES));
-      List<InetSocketAddress> newAddresses = 
-        DFSUtil.getNNServiceRpcAddresses(conf);
-      List<BPOfferService> toShutdown = new ArrayList<BPOfferService>();
-      List<InetSocketAddress> toStart = new ArrayList<InetSocketAddress>();
-      synchronized (refreshNamenodesLock) {
-        synchronized (this) {
-          for (InetSocketAddress nnaddr : nameNodeThreads.keySet()) {
-            if (!(newAddresses.contains(nnaddr))) {
-              toShutdown.add(nameNodeThreads.get(nnaddr));
-            }
-          }
-          for (InetSocketAddress nnaddr : newAddresses) {
-            if (!(nameNodeThreads.containsKey(nnaddr))) {
-              toStart.add(nnaddr);
-            }
-          }
-
-          for (InetSocketAddress nnaddr : toStart) {
-            BPOfferService bpos = new BPOfferService(nnaddr, DataNode.this);
-            nameNodeThreads.put(bpos.getNNSocketAddress(), bpos);
-          }
-        }
-
-        for (BPOfferService bpos : toShutdown) {
-          bpos.stop();
-          bpos.join();
-        }
-        
-        // stoping the BPOSes causes them to call remove() on their own when they
-        // clean up.
-        
-        // Now start the threads that are not already running.
-        startAll();
-      }
-    }
-  }
-  
   volatile boolean shouldRun = true;
   private BlockPoolManager blockPoolManager;
   public volatile FSDatasetInterface<? extends FSVolumeInterface> data = null;
@@ -653,7 +518,18 @@
     if(bpos != null) {
       bpos.notifyNamenodeReceivedBlock(block, delHint); 
     } else {
-      LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
+      LOG.error("Cannot find BPOfferService for reporting block received for bpid="
+          + block.getBlockPoolId());
+    }
+  }
+  
+  // calls specific to BP
+  protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
+    BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
+    if(bpos != null) {
+      bpos.notifyNamenodeReceivingBlock(block); 
+    } else {
+      LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
           + block.getBlockPoolId());
     }
   }
@@ -664,18 +540,66 @@
     if (bpos != null) {
       bpos.notifyNamenodeDeletedBlock(block);
     } else {
-      LOG.warn("Cannot find BPOfferService for reporting block deleted for bpid="
+      LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
           + block.getBlockPoolId());
     }
   }
   
+  /**
+   * Report a bad block which is hosted on the local DN.
+   */
   public void reportBadBlocks(ExtendedBlock block) throws IOException{
-    BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
-    if(bpos == null || bpos.bpNamenode == null) {
-      throw new IOException("cannot locate OfferService thread for bp="+block.getBlockPoolId());
-    }
+    BPOfferService bpos = getBPOSForBlock(block);
     bpos.reportBadBlocks(block);
   }
+
+  /**
+   * Report a bad block on another DN (eg if we received a corrupt replica
+   * from a remote host).
+   * @param srcDataNode the DN hosting the bad block
+   * @param block the block itself
+   */
+  public void reportRemoteBadBlock(DatanodeInfo srcDataNode, ExtendedBlock block)
+      throws IOException {
+    BPOfferService bpos = getBPOSForBlock(block);
+    bpos.reportRemoteBadBlock(srcDataNode, block);
+  }
+  
+  /**
+   * Try to send an error report to the NNs associated with the given
+   * block pool.
+   * @param bpid the block pool ID
+   * @param errCode error code to send
+   * @param errMsg textual message to send
+   */
+  void trySendErrorReport(String bpid, int errCode, String errMsg) {
+    BPOfferService bpos = blockPoolManager.get(bpid);
+    if (bpos == null) {
+      throw new IllegalArgumentException("Bad block pool: " + bpid);
+    }
+    bpos.trySendErrorReport(errCode, errMsg);
+  }
+
+
+  
+  /**
+   * Return the BPOfferService instance corresponding to the given block.
+   * @param block
+   * @return the BPOS
+   * @throws IOException if no such BPOS can be found
+   */
+  private BPOfferService getBPOSForBlock(ExtendedBlock block)
+      throws IOException {
+    Preconditions.checkNotNull(block);
+    BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
+    if (bpos == null) {
+      throw new IOException("cannot locate OfferService thread for bp="+
+          block.getBlockPoolId());
+    }
+    return bpos;
+  }
+
+
   
   // used only for testing
   void setHeartbeatsDisabledForTests(
@@ -728,7 +652,8 @@
 
     metrics = DataNodeMetrics.create(conf, getMachineName());
 
-    blockPoolManager = new BlockPoolManager(conf);
+    blockPoolManager = new BlockPoolManager(this);
+    blockPoolManager.refreshNamenodes(conf);
   }
   
   /**
@@ -961,11 +886,15 @@
   
   /**
    * get BP registration by machine and port name (host:port)
-   * @param mName
+   * @param mName - the name that the NN used
    * @return BP registration 
    * @throws IOException 
    */
   DatanodeRegistration getDNRegistrationByMachineName(String mName) {
+    // TODO: all the BPs should have the same name as each other, they all come
+    // from getName() here! and the use cases only are in tests where they just
+    // call with getName(). So we could probably just make this method return
+    // the first BPOS's registration. See HDFS-2609.
     BPOfferService [] bposArray = blockPoolManager.getAllNamenodeThreads();
     for (BPOfferService bpos : bposArray) {
       if(bpos.bpRegistration.getName().equals(mName))
@@ -1011,20 +940,6 @@
       throw new IOException(ie.getMessage());
     }
   }
-
-  /**
-   * get the name node address based on the block pool id
-   * @param bpid block pool ID
-   * @return namenode address corresponding to the bpid
-   */
-  public InetSocketAddress getNameNodeAddr(String bpid) {
-    BPOfferService bp = blockPoolManager.get(bpid);
-    if (bp != null) {
-      return bp.getNNSocketAddress();
-    }
-    LOG.warn("No name node address found for block pool ID " + bpid);
-    return null;
-  }
   
   public InetSocketAddress getSelfAddr() {
     return selfAddr;
@@ -1251,12 +1166,7 @@
 
     //inform NameNodes
     for(BPOfferService bpos: blockPoolManager.getAllNamenodeThreads()) {
-      DatanodeProtocolClientSideTranslatorPB nn = bpos.bpNamenode;
-      try {
-        nn.errorReport(bpos.bpRegistration, dpError, errMsgr);
-      } catch(IOException e) {
-        LOG.warn("Error reporting disk failure to NameNode", e);
-      }
+      bpos.trySendErrorReport(dpError, errMsgr);
     }
     
     if(hasEnoughResources) {
@@ -1273,6 +1183,10 @@
   public int getXceiverCount() {
     return threadGroup == null ? 0 : threadGroup.activeCount();
   }
+  
+  int getXmitsInProgress() {
+    return xmitsInProgress.get();
+  }
     
   UpgradeManagerDatanode getUpgradeManagerDatanode(String bpid) {
     BPOfferService bpos = blockPoolManager.get(bpid);
@@ -1285,15 +1199,15 @@
   private void transferBlock( ExtendedBlock block, 
                               DatanodeInfo xferTargets[] 
                               ) throws IOException {
-    DatanodeProtocolClientSideTranslatorPB nn = getBPNamenode(block
-        .getBlockPoolId());
+    BPOfferService bpos = getBPOSForBlock(block);
     DatanodeRegistration bpReg = getDNRegistrationForBP(block.getBlockPoolId());
     
     if (!data.isValidBlock(block)) {
       // block does not exist or is under-construction
       String errStr = "Can't send invalid block " + block;
       LOG.info(errStr);
-      nn.errorReport(bpReg, DatanodeProtocol.INVALID_BLOCK, errStr);
+      
+      bpos.trySendErrorReport(DatanodeProtocol.INVALID_BLOCK, errStr);
       return;
     }
 
@@ -1301,9 +1215,7 @@
     long onDiskLength = data.getLength(block);
     if (block.getNumBytes() > onDiskLength) {
       // Shorter on-disk len indicates corruption so report NN the corrupt block
-      nn.reportBadBlocks(new LocatedBlock[]{
-          new LocatedBlock(block, new DatanodeInfo[] {
-              new DatanodeInfo(bpReg)})});
+      bpos.reportBadBlocks(block);
       LOG.warn("Can't replicate block " + block
           + " because on-disk length " + onDiskLength 
           + " is shorter than NameNode recorded length " + block.getNumBytes());
@@ -1861,6 +1773,13 @@
                                           long newLength) throws IOException {
     ReplicaInfo r = data.updateReplicaUnderRecovery(oldBlock,
         recoveryId, newLength);
+    // Notify the namenode of the updated block info. This is important
+    // for HA, since otherwise the standby node may lose track of the
+    // block locations until the next block report.
+    ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
+    newBlock.setGenerationStamp(recoveryId);
+    newBlock.setNumBytes(newLength);
+    notifyNamenodeReceivedBlock(newBlock, "");
     return new ExtendedBlock(oldBlock.getBlockPoolId(), r);
   }
 
@@ -1935,23 +1854,32 @@
    * @return Namenode corresponding to the bpid
    * @throws IOException
    */
-  public DatanodeProtocolClientSideTranslatorPB getBPNamenode(String bpid)
+  public DatanodeProtocolClientSideTranslatorPB getActiveNamenodeForBP(String bpid)
       throws IOException {
     BPOfferService bpos = blockPoolManager.get(bpid);
     if (bpos == null) {
       throw new IOException("No block pool offer service for bpid=" + bpid);
-    } else if (bpos.bpNamenode == null) {
-      throw new IOException("cannot find a namenode proxy for bpid=" + bpid);
     }
-    return bpos.bpNamenode;
+    
+    DatanodeProtocolClientSideTranslatorPB activeNN = bpos.getActiveNN();
+    if (activeNN == null) {
+      throw new IOException(
+          "Block pool " + bpid + " has not recognized an active NN");
+    }
+    return activeNN;
   }
 
   /** Block synchronization */
   void syncBlock(RecoveringBlock rBlock,
                          List<BlockRecord> syncList) throws IOException {
     ExtendedBlock block = rBlock.getBlock();
-    DatanodeProtocolClientSideTranslatorPB nn = getBPNamenode(block
-        .getBlockPoolId());
+    DatanodeProtocolClientSideTranslatorPB nn =
+      getActiveNamenodeForBP(block.getBlockPoolId());
+    if (nn == null) {
+      throw new IOException(
+          "Unable to synchronize block " + rBlock + ", since this DN "
+          + " has not acknowledged any NN as active.");
+    }
     
     long recoveryId = rBlock.getNewGenerationStamp();
     if (LOG.isDebugEnabled()) {
@@ -2172,14 +2100,19 @@
 
   /**
    * Returned information is a JSON representation of a map with 
-   * name node host name as the key and block pool Id as the value
+   * name node host name as the key and block pool Id as the value.
+   * Note that, if there are multiple NNs in an NA nameservice,
+   * a given block pool may be represented twice.
    */
   @Override // DataNodeMXBean
   public String getNamenodeAddresses() {
     final Map<String, String> info = new HashMap<String, String>();
     for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
-      if (bpos != null && bpos.bpThread != null) {
-        info.put(bpos.getNNSocketAddress().getHostName(), bpos.getBlockPoolId());
+      if (bpos != null) {
+        for (BPServiceActor actor : bpos.getBPServiceActors()) {
+          info.put(actor.getNNSocketAddress().getHostName(),
+              bpos.getBlockPoolId());
+        }
       }
     }
     return JSON.toString(info);
@@ -2228,11 +2161,18 @@
 
   /**
    * @param addr rpc address of the namenode
-   * @return true - if BPOfferService corresponding to the namenode is alive
+   * @return true if the datanode is connected to a NameNode at the
+   * given address
    */
-  public boolean isBPServiceAlive(InetSocketAddress addr) {
-    BPOfferService bp = blockPoolManager.get(addr);
-    return bp != null ? bp.isAlive() : false;
+  public boolean isConnectedToNN(InetSocketAddress addr) {
+    for (BPOfferService bpos : getAllBpOs()) {
+      for (BPServiceActor bpsa : bpos.getBPServiceActors()) {
+        if (addr.equals(bpsa.getNNSocketAddress())) {
+          return bpsa.isAlive();
+        }
+      }
+    }
+    return false;
   }
   
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
index 408a6af..89272b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java
@@ -107,6 +107,14 @@
     
   }
   
+  synchronized long countPendingDeletions() {
+    long count = 0;
+    for (ThreadPoolExecutor exec : executors.values()) {
+      count += exec.getTaskCount() - exec.getCompletedTaskCount();
+    }
+    return count;
+  }
+  
   /**
    * Execute the task sometime in the future, using ThreadPools.
    */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
index 478fb56..9ada40fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java
@@ -92,7 +92,7 @@
           "UpgradeManagerDatanode.currentUpgrades is not null.";
         assert upgradeDaemon == null : 
           "UpgradeManagerDatanode.upgradeDaemon is not null.";
-        DatanodeProtocol nn = dataNode.getBPNamenode(bpid);
+        DatanodeProtocol nn = dataNode.getActiveNamenodeForBP(bpid);
         nn.processUpgradeCommand(broadcastCommand);
         return true;
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
index ddb1d60..49d2621 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java
@@ -45,7 +45,7 @@
   }
   
   protected DatanodeProtocol getNamenode() throws IOException {
-    return dataNode.getBPNamenode(bpid);
+    return dataNode.getActiveNamenodeForBP(bpid);
   }
 
   void setDatanode(DataNode dataNode, String bpid) {
@@ -92,14 +92,7 @@
             + " Name-node version = " + nsInfo.getLayoutVersion() + ".";
     DataNode.LOG.fatal( errorMsg );
     String bpid = nsInfo.getBlockPoolID();
-    DatanodeProtocol nn = dataNode.getBPNamenode(bpid);
-    try {
-      nn.errorReport(dataNode.getDNRegistrationForBP(bpid),
-                                    DatanodeProtocol.NOTIFY, errorMsg);
-    } catch(SocketTimeoutException e) {  // namenode is busy
-      DataNode.LOG.info("Problem connecting to server: " 
-                        + dataNode.getNameNodeAddr(nsInfo.getBlockPoolID()));
-    }
+    dataNode.trySendErrorReport(bpid, DatanodeProtocol.NOTIFY, errorMsg);
     throw new IOException(errorMsg);
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
index fc1fe14..ece013f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
@@ -217,7 +217,7 @@
       int logVersion = storage.getLayoutVersion();
       backupInputStream.setBytes(data, logVersion);
 
-      int numLoaded = logLoader.loadEditRecords(logVersion, backupInputStream, 
+      long numLoaded = logLoader.loadEditRecords(logVersion, backupInputStream, 
                                                 true, lastAppliedTxId + 1);
       if (numLoaded != numTxns) {
         throw new IOException("Batch of txns starting at txnid " +
@@ -310,7 +310,7 @@
             + " txns from in-progress stream " + stream);
         
         FSEditLogLoader loader = new FSEditLogLoader(namesystem);
-        int numLoaded = loader.loadFSEdits(stream, lastAppliedTxId + 1);
+        long numLoaded = loader.loadFSEdits(stream, lastAppliedTxId + 1);
         lastAppliedTxId += numLoaded;
         assert numLoaded == remainingTxns :
           "expected to load " + remainingTxns + " but loaded " +
@@ -345,7 +345,7 @@
   synchronized void namenodeStartedLogSegment(long txid)
       throws IOException {
     LOG.info("NameNode started a new log segment at txid " + txid);
-    if (editLog.isOpen()) {
+    if (editLog.isSegmentOpen()) {
       if (editLog.getLastWrittenTxId() == txid - 1) {
         // We are in sync with the NN, so end and finalize the current segment
         editLog.endCurrentLogSegment(false);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
index c655ee7..de75b76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupJournalManager.java
@@ -58,7 +58,7 @@
   }
 
   @Override
-  public long getNumberOfTransactions(long fromTxnId) 
+  public long getNumberOfTransactions(long fromTxnId, boolean inProgressOk)
       throws IOException, CorruptionException {
     // This JournalManager is never used for input. Therefore it cannot
     // return any transactions
@@ -66,7 +66,8 @@
   }
   
   @Override
-  public EditLogInputStream getInputStream(long fromTxnId) throws IOException {
+  public EditLogInputStream getInputStream(long fromTxnId, boolean inProgressOk)
+      throws IOException {
     // This JournalManager is never used for input. Therefore it cannot
     // return any transactions
     throw new IOException("Unsupported operation");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index 0515470..9cad4eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -26,13 +26,13 @@
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -41,7 +41,8 @@
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -69,7 +70,7 @@
   private static final String BN_SERVICE_RPC_ADDRESS_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY;
 
   /** Name-node proxy */
-  NamenodeProtocolTranslatorPB namenode;
+  NamenodeProtocol namenode;
   /** Name-node RPC address */
   String nnRpcAddress;
   /** Name-node HTTP address */
@@ -89,13 +90,13 @@
   // Common NameNode methods implementation for backup node.
   /////////////////////////////////////////////////////
   @Override // NameNode
-  protected InetSocketAddress getRpcServerAddress(Configuration conf) throws IOException {
+  protected InetSocketAddress getRpcServerAddress(Configuration conf) {
     String addr = conf.get(BN_ADDRESS_NAME_KEY, BN_ADDRESS_DEFAULT);
     return NetUtils.createSocketAddr(addr);
   }
   
   @Override
-  protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) throws IOException {
+  protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) {
     String addr = conf.get(BN_SERVICE_RPC_ADDRESS_KEY);
     if (addr == null || addr.isEmpty()) {
       return null;
@@ -143,6 +144,7 @@
                  CommonConfigurationKeys.FS_TRASH_INTERVAL_DEFAULT);
     NamespaceInfo nsInfo = handshake(conf);
     super.initialize(conf);
+
     if (false == namesystem.isInSafeMode()) {
       namesystem.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
     }
@@ -189,7 +191,7 @@
     }
     // Stop the RPC client
     if (namenode != null) {
-      IOUtils.cleanup(LOG, namenode);
+      RPC.stopProxy(namenode);
     }
     namenode = null;
     // Stop the checkpoint manager
@@ -197,6 +199,11 @@
       checkpointManager.interrupt();
       checkpointManager = null;
     }
+
+    // Abort current log segment - otherwise the NN shutdown code
+    // will close it gracefully, which is incorrect.
+    getFSImage().getEditLog().abortCurrentLogSegment();
+
     // Stop name-node threads
     super.stop();
   }
@@ -221,58 +228,31 @@
           this.clientRpcServer);
       nnRpcAddress = nn.nnRpcAddress;
     }
-  
-    /////////////////////////////////////////////////////
-    // NamenodeProtocol implementation for backup node.
-    /////////////////////////////////////////////////////
-    @Override // NamenodeProtocol
-    public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
-    throws IOException {
-      throw new UnsupportedActionException("getBlocks");
-    }
-  
-    // Only active name-node can register other nodes.
-    @Override // NamenodeProtocol
-    public NamenodeRegistration register(NamenodeRegistration registration
-    ) throws IOException {
-      throw new UnsupportedActionException("register");
-    }
-  
-    @Override // NamenodeProtocol
-    public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
-    throws IOException {
-      throw new UnsupportedActionException("startCheckpoint");
-    }
-  
-    @Override // NamenodeProtocol
-    public void endCheckpoint(NamenodeRegistration registration,
-                              CheckpointSignature sig) throws IOException {
-      throw new UnsupportedActionException("endCheckpoint");
-    }  
-  
+
     /////////////////////////////////////////////////////
     // BackupNodeProtocol implementation for backup node.
     /////////////////////////////////////////////////////
-  
+    @Override
+    public void startLogSegment(NamenodeRegistration registration, long txid)
+        throws IOException {
+      namesystem.checkOperation(OperationCategory.JOURNAL);
+      verifyRequest(registration);
+      
+      getBNImage().namenodeStartedLogSegment(txid);
+    }
+    
     @Override
     public void journal(NamenodeRegistration nnReg,
         long firstTxId, int numTxns,
         byte[] records) throws IOException {
+      namesystem.checkOperation(OperationCategory.JOURNAL);
       verifyRequest(nnReg);
       if(!nnRpcAddress.equals(nnReg.getAddress()))
         throw new IOException("Journal request from unexpected name-node: "
-            + nnReg.getAddress() + " expecting " + clientRpcAddress);
+            + nnReg.getAddress() + " expecting " + nnRpcAddress);
       getBNImage().journal(firstTxId, numTxns, records);
     }
-  
-    @Override
-    public void startLogSegment(NamenodeRegistration registration, long txid)
-        throws IOException {
-      verifyRequest(registration);
-    
-      getBNImage().namenodeStartedLogSegment(txid);
-    }
-    
+
     private BackupImage getBNImage() {
       return (BackupImage)nn.getFSImage();
     }
@@ -295,8 +275,9 @@
   private NamespaceInfo handshake(Configuration conf) throws IOException {
     // connect to name node
     InetSocketAddress nnAddress = NameNode.getServiceAddress(conf, true);
-    this.namenode = new NamenodeProtocolTranslatorPB(nnAddress, conf,
-        UserGroupInformation.getCurrentUser());
+    this.namenode = NameNodeProxies.createNonHAProxy(conf, nnAddress,
+        NamenodeProtocol.class, UserGroupInformation.getCurrentUser(),
+        true).getProxy();
     this.nnRpcAddress = NetUtils.getHostPortString(nnAddress);
     this.nnHttpAddress = NetUtils.getHostPortString(super.getHttpServerAddress(conf));
     // get version and id info from the name-node
@@ -409,6 +390,28 @@
   }
   
   @Override
+  protected NameNodeHAContext createHAContext() {
+    return new BNHAContext();
+  }
+  
+  private class BNHAContext extends NameNodeHAContext {
+    @Override // NameNode
+    public void checkOperation(OperationCategory op)
+        throws StandbyException {
+      if (op == OperationCategory.UNCHECKED ||
+          op == OperationCategory.CHECKPOINT) {
+        return;
+      }
+      if (OperationCategory.JOURNAL != op &&
+          !(OperationCategory.READ == op && allowStaleStandbyReads)) {
+        String msg = "Operation category " + op
+            + " is not supported at the BackupNode";
+        throw new StandbyException(msg);
+      }
+    }
+  }
+  
+  @Override
   protected String getNameServiceId(Configuration conf) {
     return DFSUtil.getBackupNameServiceId(conf);
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
new file mode 100644
index 0000000..8b3cf04
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
@@ -0,0 +1,78 @@
+/**
+ * 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.namenode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.collect.ImmutableList;
+
+@InterfaceAudience.Private
+public class CheckpointConf {
+  private static final Log LOG = LogFactory.getLog(CheckpointConf.class);
+  
+  /** How often to checkpoint regardless of number of txns */
+  private final long checkpointPeriod;    // in seconds
+  
+  /** How often to poll the NN to check checkpointTxnCount */
+  private final long checkpointCheckPeriod; // in seconds
+  
+  /** checkpoint once every this many transactions, regardless of time */
+  private final long checkpointTxnCount;
+
+  
+  public CheckpointConf(Configuration conf) {
+    checkpointCheckPeriod = conf.getLong(
+        DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
+        DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT);
+        
+    checkpointPeriod = conf.getLong(DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 
+                                    DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
+    checkpointTxnCount = conf.getLong(DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 
+                                  DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
+    warnForDeprecatedConfigs(conf);
+  }
+  
+  private static void warnForDeprecatedConfigs(Configuration conf) {
+    for (String key : ImmutableList.of(
+          "fs.checkpoint.size",
+          "dfs.namenode.checkpoint.size")) {
+      if (conf.get(key) != null) {
+        LOG.warn("Configuration key " + key + " is deprecated! Ignoring..." +
+            " Instead please specify a value for " +
+            DFS_NAMENODE_CHECKPOINT_TXNS_KEY);
+      }
+    }
+  }
+
+  public long getPeriod() {
+    return checkpointPeriod;
+  }
+
+  public long getCheckPeriod() {
+    return Math.min(checkpointCheckPeriod, checkpointPeriod);
+  }
+
+  public long getTxnCount() {
+    return checkpointTxnCount;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
index 39d2abae..6ae931f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
@@ -29,7 +29,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
@@ -58,17 +57,16 @@
 
   private BackupNode backupNode;
   volatile boolean shouldRun;
-  private long checkpointPeriod;    // in seconds
-  // Transactions count to trigger the checkpoint
-  private long checkpointTxnCount; 
 
   private String infoBindAddress;
 
+  private CheckpointConf checkpointConf;
+
   private BackupImage getFSImage() {
     return (BackupImage)backupNode.getFSImage();
   }
 
-  private NamenodeProtocol getNamenode(){
+  private NamenodeProtocol getRemoteNamenodeProxy(){
     return backupNode.namenode;
   }
 
@@ -89,26 +87,24 @@
   /**
    * Initialize checkpoint.
    */
-  @SuppressWarnings("deprecation")
   private void initialize(Configuration conf) throws IOException {
     // Create connection to the namenode.
     shouldRun = true;
 
     // Initialize other scheduling parameters from the configuration
-    checkpointPeriod = conf.getLong(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 
-                                    DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
-    checkpointTxnCount = conf.getLong(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 
-                                  DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
-    SecondaryNameNode.warnForDeprecatedConfigs(conf);
+    checkpointConf = new CheckpointConf(conf);
 
     // Pull out exact http address for posting url to avoid ip aliasing issues
     String fullInfoAddr = conf.get(DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, 
                                    DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT);
     infoBindAddress = fullInfoAddr.substring(0, fullInfoAddr.indexOf(":"));
 
-    LOG.info("Checkpoint Period : " + checkpointPeriod + " secs " +
-             "(" + checkpointPeriod/60 + " min)");
-    LOG.info("Transactions count is  : " + checkpointTxnCount + ", to trigger checkpoint");
+    LOG.info("Checkpoint Period : " +
+             checkpointConf.getPeriod() + " secs " +
+             "(" + checkpointConf.getPeriod()/60 + " min)");
+    LOG.info("Transactions count is  : " +
+             checkpointConf.getTxnCount() +
+             ", to trigger checkpoint");
   }
 
   /**
@@ -125,8 +121,8 @@
   public void run() {
     // Check the size of the edit log once every 5 minutes.
     long periodMSec = 5 * 60;   // 5 minutes
-    if(checkpointPeriod < periodMSec) {
-      periodMSec = checkpointPeriod;
+    if(checkpointConf.getPeriod() < periodMSec) {
+      periodMSec = checkpointConf.getPeriod();
     }
     periodMSec *= 1000;
 
@@ -142,7 +138,7 @@
           shouldCheckpoint = true;
         } else {
           long txns = countUncheckpointedTxns();
-          if(txns >= checkpointTxnCount)
+          if(txns >= checkpointConf.getTxnCount())
             shouldCheckpoint = true;
         }
         if(shouldCheckpoint) {
@@ -165,7 +161,7 @@
   }
 
   private long countUncheckpointedTxns() throws IOException {
-    long curTxId = getNamenode().getTransactionID();
+    long curTxId = getRemoteNamenodeProxy().getTransactionID();
     long uncheckpointedTxns = curTxId -
       getFSImage().getStorage().getMostRecentCheckpointTxId();
     assert uncheckpointedTxns >= 0;
@@ -183,7 +179,7 @@
     bnImage.freezeNamespaceAtNextRoll();
     
     NamenodeCommand cmd = 
-      getNamenode().startCheckpoint(backupNode.getRegistration());
+      getRemoteNamenodeProxy().startCheckpoint(backupNode.getRegistration());
     CheckpointCommand cpCmd = null;
     switch(cmd.getAction()) {
       case NamenodeProtocol.ACT_SHUTDOWN:
@@ -207,7 +203,7 @@
     long lastApplied = bnImage.getLastAppliedTxId();
     LOG.debug("Doing checkpoint. Last applied: " + lastApplied);
     RemoteEditLogManifest manifest =
-      getNamenode().getEditLogManifest(bnImage.getLastAppliedTxId() + 1);
+      getRemoteNamenodeProxy().getEditLogManifest(bnImage.getLastAppliedTxId() + 1);
 
     if (!manifest.getLogs().isEmpty()) {
       RemoteEditLog firstRemoteLog = manifest.getLogs().get(0);
@@ -243,11 +239,16 @@
     
     long txid = bnImage.getLastAppliedTxId();
     
-    backupNode.namesystem.dir.setReady();
-    backupNode.namesystem.setBlockTotal();
-    
-    bnImage.saveFSImageInAllDirs(backupNode.getNamesystem(), txid);
-    bnStorage.writeAll();
+    backupNode.namesystem.writeLock();
+    try {
+      backupNode.namesystem.dir.setReady();
+      backupNode.namesystem.setBlockTotal();
+      
+      bnImage.saveFSImageInAllDirs(backupNode.getNamesystem(), txid);
+      bnStorage.writeAll();
+    } finally {
+      backupNode.namesystem.writeUnlock();
+    }
 
     if(cpCmd.needToReturnImage()) {
       TransferFsImage.uploadImageFromStorage(
@@ -255,7 +256,7 @@
           bnStorage, txid);
     }
 
-    getNamenode().endCheckpoint(backupNode.getRegistration(), sig);
+    getRemoteNamenodeProxy().endCheckpoint(backupNode.getRegistration(), sig);
 
     if (backupNode.getRole() == NamenodeRole.BACKUP) {
       bnImage.convergeJournalSpool();
@@ -286,7 +287,7 @@
           log.getStartTxId(), log.getEndTxId());
       if (log.getStartTxId() > dstImage.getLastAppliedTxId()) {
         editsStreams.add(new EditLogFileInputStream(f, log.getStartTxId(), 
-                                                    log.getEndTxId()));
+                                                    log.getEndTxId(), true));
        }
     }
     LOG.info("Checkpointer about to load edits from " +
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
index 10601b1..3ffc852 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
@@ -39,6 +39,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.JsonNode;
@@ -66,9 +67,10 @@
   ClusterStatus generateClusterHealthReport() {
     ClusterStatus cs = new ClusterStatus();
     Configuration conf = new Configuration();
-    List<InetSocketAddress> isas = null;
+    List<ConfiguredNNAddress> nns = null;
     try {
-      isas = DFSUtil.getNNServiceRpcAddresses(conf);
+      nns = DFSUtil.flattenAddressMap(
+          DFSUtil.getNNServiceRpcAddresses(conf));
     } catch (Exception e) {
       // Could not build cluster status
       cs.setError(e);
@@ -76,7 +78,8 @@
     }
     
     // Process each namenode and add it to ClusterStatus
-    for (InetSocketAddress isa : isas) {
+    for (ConfiguredNNAddress cnn : nns) {
+      InetSocketAddress isa = cnn.getAddress();
       NamenodeMXBeanHelper nnHelper = null;
       try {
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
@@ -102,9 +105,10 @@
   DecommissionStatus generateDecommissioningReport() {
     String clusterid = "";
     Configuration conf = new Configuration();
-    List<InetSocketAddress> isas = null;
+    List<ConfiguredNNAddress> cnns = null;
     try {
-      isas = DFSUtil.getNNServiceRpcAddresses(conf);
+      cnns = DFSUtil.flattenAddressMap(
+          DFSUtil.getNNServiceRpcAddresses(conf));
     } catch (Exception e) {
       // catch any exception encountered other than connecting to namenodes
       DecommissionStatus dInfo = new DecommissionStatus(clusterid, e);
@@ -122,7 +126,8 @@
       new HashMap<String, Exception>();
     
     List<String> unreportedNamenode = new ArrayList<String>();
-    for (InetSocketAddress isa : isas) {
+    for (ConfiguredNNAddress cnn : cnns) {
+      InetSocketAddress isa = cnn.getAddress();
       NamenodeMXBeanHelper nnHelper = null;
       try {
         nnHelper = new NamenodeMXBeanHelper(isa, conf);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
index 6459ffd..402dcdd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
@@ -26,8 +26,8 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.ipc.RemoteException;
@@ -77,7 +77,8 @@
       NameNodeHttpServer.getNameNodeAddressFromContext(context);
     Configuration conf = new HdfsConfiguration(
         NameNodeHttpServer.getConfFromContext(context));
-    return DFSUtil.createNamenode(nnAddr, conf);
+    return NameNodeProxies.createProxy(conf, NameNode.getUri(nnAddr),
+        ClientProtocol.class).getProxy();
   }
 
   protected UserGroupInformation getUGI(HttpServletRequest request,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
index 637400f..a0fb8fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
@@ -133,4 +133,9 @@
   public long getLastTxId() throws IOException {
     return HdfsConstants.INVALID_TXID;
   }
+
+  @Override
+  public boolean isInProgress() {
+    return true;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
index 8c3ad2e..bdb4c5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java
@@ -22,12 +22,14 @@
 import java.util.Arrays;
 
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * An implementation of the abstract class {@link EditLogOutputStream},
@@ -40,7 +42,7 @@
 class EditLogBackupOutputStream extends EditLogOutputStream {
   static int DEFAULT_BUFFER_SIZE = 256;
 
-  private JournalProtocolTranslatorPB backupNode;  // RPC proxy to backup node
+  private JournalProtocol backupNode;  // RPC proxy to backup node
   private NamenodeRegistration bnRegistration;  // backup node registration
   private NamenodeRegistration nnRegistration;  // active node registration
   private EditsDoubleBuffer doubleBuf;
@@ -55,8 +57,9 @@
     InetSocketAddress bnAddress =
       NetUtils.createSocketAddr(bnRegistration.getAddress());
     try {
-      this.backupNode =
-          new JournalProtocolTranslatorPB(bnAddress, new HdfsConfiguration());
+      this.backupNode = NameNodeProxies.createNonHAProxy(new HdfsConfiguration(),
+          bnAddress, JournalProtocol.class, UserGroupInformation.getCurrentUser(),
+          true).getProxy();
     } catch(IOException e) {
       Storage.LOG.error("Error connecting to: " + bnAddress, e);
       throw e;
@@ -93,14 +96,14 @@
       throw new IOException("BackupEditStream has " + size +
                           " records still to be flushed and cannot be closed.");
     } 
-    IOUtils.cleanup(Storage.LOG, backupNode); // stop the RPC threads
+    RPC.stopProxy(backupNode); // stop the RPC threads
     doubleBuf.close();
     doubleBuf = null;
   }
 
   @Override
   public void abort() throws IOException {
-    IOUtils.cleanup(Storage.LOG, backupNode);
+    RPC.stopProxy(backupNode);
     doubleBuf = null;
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
index 952e4a7..22c1297 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
@@ -41,6 +41,7 @@
   private final int logVersion;
   private final FSEditLogOp.Reader reader;
   private final FSEditLogLoader.PositionTrackingInputStream tracker;
+  private final boolean isInProgress;
   
   /**
    * Open an EditLogInputStream for the given file.
@@ -53,7 +54,7 @@
    */
   EditLogFileInputStream(File name)
       throws LogHeaderCorruptException, IOException {
-    this(name, HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID);
+    this(name, HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID, false);
   }
 
   /**
@@ -66,7 +67,8 @@
    * @throws IOException if an actual IO error occurs while reading the
    *         header
    */
-  EditLogFileInputStream(File name, long firstTxId, long lastTxId)
+  EditLogFileInputStream(File name, long firstTxId, long lastTxId,
+      boolean isInProgress)
       throws LogHeaderCorruptException, IOException {
     file = name;
     fStream = new FileInputStream(name);
@@ -84,6 +86,25 @@
     reader = new FSEditLogOp.Reader(in, logVersion);
     this.firstTxId = firstTxId;
     this.lastTxId = lastTxId;
+    this.isInProgress = isInProgress;
+  }
+
+  /**
+   * Skip over a number of transactions. Subsequent calls to
+   * {@link EditLogFileInputStream#readOp()} will begin after these skipped
+   * transactions. If more transactions are requested to be skipped than remain
+   * in the edit log, all edit log ops in the log will be skipped and subsequent
+   * calls to {@link EditLogInputStream#readOp} will return null.
+   * 
+   * @param transactionsToSkip number of transactions to skip over.
+   * @throws IOException if there's an error while reading an operation
+   */
+  public void skipTransactions(long transactionsToSkip) throws IOException {
+    assert firstTxId != HdfsConstants.INVALID_TXID &&
+        lastTxId != HdfsConstants.INVALID_TXID;
+    for (long i = 0; i < transactionsToSkip; i++) {
+      reader.readOp();
+    }
   }
 
   @Override
@@ -133,6 +154,11 @@
   }
   
   @Override
+  public boolean isInProgress() {
+    return isInProgress;
+  }
+  
+  @Override
   public String toString() {
     return getName();
   }
@@ -142,11 +168,11 @@
     try {
       in = new EditLogFileInputStream(file);
     } catch (LogHeaderCorruptException corrupt) {
-      // If it's missing its header, this is equivalent to no transactions
+      // If the header is malformed or the wrong value, this indicates a corruption
       FSImage.LOG.warn("Log at " + file + " has no valid header",
           corrupt);
-      return new FSEditLogLoader.EditLogValidation(0, HdfsConstants.INVALID_TXID, 
-                                                   HdfsConstants.INVALID_TXID);
+      return new FSEditLogLoader.EditLogValidation(0,
+          HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID, true);
     }
     
     try {
@@ -172,14 +198,13 @@
       throw new LogHeaderCorruptException(
           "Reached EOF when reading log header");
     }
-    if (logVersion < HdfsConstants.LAYOUT_VERSION) { // future version
+    if (logVersion < HdfsConstants.LAYOUT_VERSION || // future version
+        logVersion > Storage.LAST_UPGRADABLE_LAYOUT_VERSION) { // unsupported
       throw new LogHeaderCorruptException(
           "Unexpected version of the file system log file: "
           + logVersion + ". Current version = "
           + HdfsConstants.LAYOUT_VERSION + ".");
     }
-    assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
-      "Unsupported version " + logVersion;
     return logVersion;
   }
   
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
index bdc0bd2..f7e1f01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -27,6 +28,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.io.IOUtils;
 
@@ -36,7 +38,8 @@
  * An implementation of the abstract class {@link EditLogOutputStream}, which
  * stores edits in a local file.
  */
-class EditLogFileOutputStream extends EditLogOutputStream {
+@InterfaceAudience.Private
+public class EditLogFileOutputStream extends EditLogOutputStream {
   private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
 
   private File file;
@@ -96,11 +99,23 @@
   public void create() throws IOException {
     fc.truncate(0);
     fc.position(0);
-    doubleBuf.getCurrentBuf().writeInt(HdfsConstants.LAYOUT_VERSION);
+    writeHeader(doubleBuf.getCurrentBuf());
     setReadyToFlush();
     flush();
   }
 
+  /**
+   * Write header information for this EditLogFileOutputStream to the provided
+   * DataOutputSream.
+   * 
+   * @param out the output stream to write the header to.
+   * @throws IOException in the event of error writing to the stream.
+   */
+  @VisibleForTesting
+  public static void writeHeader(DataOutputStream out) throws IOException {
+    out.writeInt(HdfsConstants.LAYOUT_VERSION);
+  }
+
   @Override
   public void close() throws IOException {
     if (fp == null) {
@@ -204,6 +219,11 @@
   File getFile() {
     return file;
   }
+  
+  @Override
+  public String toString() {
+    return "EditLogFileOutputStream(" + file + ")";
+  }
 
   /**
    * @return true if this stream is currently open.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputException.java
new file mode 100644
index 0000000..56edf8c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputException.java
@@ -0,0 +1,45 @@
+/**
+ * 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.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Thrown when there's a failure to read an edit log op from disk when loading
+ * edits.
+ */
+@InterfaceAudience.Private
+public class EditLogInputException extends IOException {
+
+  private static final long serialVersionUID = 1L;
+  
+  private final long numEditsLoaded;
+  
+  public EditLogInputException(String message, Throwable cause,
+      long numEditsLoaded) {
+    super(message, cause);
+    this.numEditsLoaded = numEditsLoaded;
+  }
+  
+  public long getNumEditsLoaded() {
+    return numEditsLoaded;
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
index 3ad19951..7a7f8d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
@@ -22,6 +22,9 @@
 import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
 /**
  * A generic abstract class to support reading edits log data from 
  * persistent storage.
@@ -79,4 +82,9 @@
    * Return the size of the current edits log.
    */
   public abstract long length() throws IOException;
+  
+  /**
+   * Return true if this stream is in progress, false if it is finalized.
+   */
+  public abstract boolean isInProgress();
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 45ce9df..ab0f4c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -261,113 +261,32 @@
    */
   INode unprotectedAddFile( String path, 
                             PermissionStatus permissions,
-                            BlockInfo[] blocks, 
                             short replication,
                             long modificationTime,
                             long atime,
                             long preferredBlockSize,
+                            boolean underConstruction,
                             String clientName,
                             String clientMachine)
       throws UnresolvedLinkException {
     INode newNode;
     assert hasWriteLock();
-    if (blocks == null)
-      newNode = new INodeDirectory(permissions, modificationTime);
-    else if(blocks.length == 0 || blocks[blocks.length-1].getBlockUCState()
-        == BlockUCState.UNDER_CONSTRUCTION) {
+    if (underConstruction) {
       newNode = new INodeFileUnderConstruction(
-          permissions, blocks.length, replication,
+          permissions, replication,
           preferredBlockSize, modificationTime, clientName, 
           clientMachine, null);
     } else {
-      newNode = new INodeFile(permissions, blocks.length, replication,
+      newNode = new INodeFile(permissions, 0, replication,
                               modificationTime, atime, preferredBlockSize);
     }
-    writeLock();
+
     try {
-      try {
-        newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE);
-        if(newNode != null && blocks != null) {
-          int nrBlocks = blocks.length;
-          // Add file->block mapping
-          INodeFile newF = (INodeFile)newNode;
-          for (int i = 0; i < nrBlocks; i++) {
-            newF.setBlock(i, getBlockManager().addINode(blocks[i], newF));
-          }
-        }
-      } catch (IOException e) {
-        return null;
-      }
-      return newNode;
-    } finally {
-      writeUnlock();
+      newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE);
+    } catch (IOException e) {
+      return null;
     }
-
-  }
-
-  /**
-   * Update files in-memory data structures with new block information.
-   * @throws IOException 
-   */
-  void updateFile(INodeFile file,
-                  String path,
-                  BlockInfo[] blocks, 
-                  long mtime,
-                  long atime) throws IOException {
-
-    // Update the salient file attributes.
-    file.setAccessTime(atime);
-    file.setModificationTimeForce(mtime);
-
-    // Update its block list
-    BlockInfo[] oldBlocks = file.getBlocks();
-
-    // Are we only updating the last block's gen stamp.
-    boolean isGenStampUpdate = oldBlocks.length == blocks.length;
-
-    // First, update blocks in common
-    BlockInfo oldBlock = null;
-    for (int i = 0; i < oldBlocks.length && i < blocks.length; i++) {
-      oldBlock = oldBlocks[i];
-      Block newBlock = blocks[i];
-
-      boolean isLastBlock = i == oldBlocks.length - 1;
-      if (oldBlock.getBlockId() != newBlock.getBlockId() ||
-          (oldBlock.getGenerationStamp() != newBlock.getGenerationStamp() && 
-              !(isGenStampUpdate && isLastBlock))) {
-        throw new IOException("Mismatched block IDs or generation stamps, " + 
-            "attempting to replace block " + oldBlock + " with " + newBlock +
-            " as block # " + i + "/" + blocks.length + " of " + path);
-      }
-
-      oldBlock.setNumBytes(newBlock.getNumBytes());
-      oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
-    }
-
-    if (blocks.length < oldBlocks.length) {
-      // We're removing a block from the file, e.g. abandonBlock(...)
-      if (!file.isUnderConstruction()) {
-        throw new IOException("Trying to remove a block from file " +
-            path + " which is not under construction.");
-      }
-      if (blocks.length != oldBlocks.length - 1) {
-        throw new IOException("Trying to remove more than one block from file "
-            + path);
-      }
-      unprotectedRemoveBlock(path,
-          (INodeFileUnderConstruction)file, oldBlocks[oldBlocks.length - 1]);
-    } else if (blocks.length > oldBlocks.length) {
-      // We're adding blocks
-      // First complete last old Block
-      getBlockManager().completeBlock(file, oldBlocks.length-1, true);
-      // Add the new blocks
-      for (int i = oldBlocks.length; i < blocks.length; i++) {
-        // addBlock();
-        BlockInfo newBI = blocks[i];
-        getBlockManager().addINode(newBI, file);
-        file.addBlock(newBI);
-      }
-    }
+    return newNode;
   }
 
   INodeDirectory addToParent(byte[] src, INodeDirectory parentINode,
@@ -450,7 +369,7 @@
 
     writeLock();
     try {
-      fsImage.getEditLog().logOpenFile(path, file);
+      fsImage.getEditLog().logUpdateBlocks(path, file);
       if(NameNode.stateChangeLog.isDebugEnabled()) {
         NameNode.stateChangeLog.debug("DIR* FSDirectory.persistBlocks: "
             +path+" with "+ file.getBlocks().length 
@@ -460,7 +379,7 @@
       writeUnlock();
     }
   }
-
+  
   /**
    * Close file.
    */
@@ -483,7 +402,7 @@
   }
 
   /**
-   * Remove a block to the file.
+   * Remove a block from the file.
    */
   boolean removeBlock(String path, INodeFileUnderConstruction fileNode, 
                       Block block) throws IOException {
@@ -499,7 +418,7 @@
     }
     return true;
   }
-
+  
   void unprotectedRemoveBlock(String path, INodeFileUnderConstruction fileNode, 
       Block block) throws IOException {
     // modify file-> block and blocksMap
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 80c6088..7c630d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -62,22 +62,36 @@
 
   /**
    * State machine for edit log.
+   * 
+   * In a non-HA setup:
+   * 
    * The log starts in UNITIALIZED state upon construction. Once it's
-   * initialized, it is usually in IN_SEGMENT state, indicating that edits
-   * may be written. In the middle of a roll, or while saving the namespace,
-   * it briefly enters the BETWEEN_LOG_SEGMENTS state, indicating that the
-   * previous segment has been closed, but the new one has not yet been opened.
+   * initialized, it is usually in IN_SEGMENT state, indicating that edits may
+   * be written. In the middle of a roll, or while saving the namespace, it
+   * briefly enters the BETWEEN_LOG_SEGMENTS state, indicating that the previous
+   * segment has been closed, but the new one has not yet been opened.
+   * 
+   * In an HA setup:
+   * 
+   * The log starts in UNINITIALIZED state upon construction. Once it's
+   * initialized, it sits in the OPEN_FOR_READING state the entire time that the
+   * NN is in standby. Upon the NN transition to active, the log will be CLOSED,
+   * and then move to being BETWEEN_LOG_SEGMENTS, much as if the NN had just
+   * started up, and then will move to IN_SEGMENT so it can begin writing to the
+   * log. The log states will then revert to behaving as they do in a non-HA
+   * setup.
    */
   private enum State {
     UNINITIALIZED,
     BETWEEN_LOG_SEGMENTS,
     IN_SEGMENT,
+    OPEN_FOR_READING,
     CLOSED;
   }  
   private State state = State.UNINITIALIZED;
   
   //initialize
-  private JournalSet journalSet;
+  private JournalSet journalSet = null;
   private EditLogOutputStream editLogStream = null;
 
   // a monotonically increasing counter that represents transactionIds.
@@ -112,7 +126,12 @@
   private NNStorage storage;
   private Configuration conf;
   
-  private Collection<URI> editsDirs;
+  private List<URI> editsDirs;
+  
+  /**
+   * The edit directories that are shared between primary and secondary.
+   */
+  private List<URI> sharedEditsDirs;
 
   private static class TransactionId {
     public long txid;
@@ -151,11 +170,11 @@
    * @param storage Storage object used by namenode
    * @param editsDirs List of journals to use
    */
-  FSEditLog(Configuration conf, NNStorage storage, Collection<URI> editsDirs) {
+  FSEditLog(Configuration conf, NNStorage storage, List<URI> editsDirs) {
     init(conf, storage, editsDirs);
   }
   
-  private void init(Configuration conf, NNStorage storage, Collection<URI> editsDirs) {
+  private void init(Configuration conf, NNStorage storage, List<URI> editsDirs) {
     isSyncRunning = false;
     this.conf = conf;
     this.storage = storage;
@@ -165,19 +184,44 @@
     // If this list is empty, an error will be thrown on first use
     // of the editlog, as no journals will exist
     this.editsDirs = Lists.newArrayList(editsDirs);
+
+    this.sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
+  }
+  
+  public synchronized void initJournalsForWrite() {
+    Preconditions.checkState(state == State.UNINITIALIZED ||
+        state == State.CLOSED, "Unexpected state: %s", state);
     
+    initJournals(this.editsDirs);
+    state = State.BETWEEN_LOG_SEGMENTS;
+  }
+  
+  public synchronized void initSharedJournalsForRead() {
+    if (state == State.OPEN_FOR_READING) {
+      LOG.warn("Initializing shared journals for READ, already open for READ",
+          new Exception());
+      return;
+    }
+    Preconditions.checkState(state == State.UNINITIALIZED ||
+        state == State.CLOSED);
+    
+    initJournals(this.sharedEditsDirs);
+    state = State.OPEN_FOR_READING;
+  }
+  
+  private synchronized void initJournals(List<URI> dirs) {
     int minimumRedundantJournals = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY,
         DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT);
 
     journalSet = new JournalSet(minimumRedundantJournals);
-    for (URI u : this.editsDirs) {
+    for (URI u : dirs) {
       boolean required = FSNamesystem.getRequiredNamespaceEditsDirs(conf)
           .contains(u);
       if (u.getScheme().equals(NNStorage.LOCAL_URI_SCHEME)) {
         StorageDirectory sd = storage.getStorageDirectory(u);
         if (sd != null) {
-          journalSet.add(new FileJournalManager(sd), required);
+          journalSet.add(new FileJournalManager(sd, storage), required);
         }
       } else {
         journalSet.add(createJournal(u), required);
@@ -187,7 +231,6 @@
     if (journalSet.isEmpty()) {
       LOG.error("No edits directories configured!");
     } 
-    state = State.BETWEEN_LOG_SEGMENTS;
   }
 
   /**
@@ -202,18 +245,51 @@
    * Initialize the output stream for logging, opening the first
    * log segment.
    */
-  synchronized void open() throws IOException {
-    Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS);
+  synchronized void openForWrite() throws IOException {
+    Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS,
+        "Bad state: %s", state);
 
-    startLogSegment(getLastWrittenTxId() + 1, true);
+    long segmentTxId = getLastWrittenTxId() + 1;
+    // Safety check: we should never start a segment if there are
+    // newer txids readable.
+    EditLogInputStream s = journalSet.getInputStream(segmentTxId, true);
+    try {
+      Preconditions.checkState(s == null,
+          "Cannot start writing at txid %s when there is a stream " +
+          "available for read: %s", segmentTxId, s);
+    } finally {
+      IOUtils.closeStream(s);
+    }
+    
+    startLogSegment(segmentTxId, true);
     assert state == State.IN_SEGMENT : "Bad state: " + state;
   }
   
-  synchronized boolean isOpen() {
+  /**
+   * @return true if the log is currently open in write mode, regardless
+   * of whether it actually has an open segment.
+   */
+  synchronized boolean isOpenForWrite() {
+    return state == State.IN_SEGMENT ||
+      state == State.BETWEEN_LOG_SEGMENTS;
+  }
+  
+  /**
+   * @return true if the log is open in write mode and has a segment open
+   * ready to take edits.
+   */
+  synchronized boolean isSegmentOpen() {
     return state == State.IN_SEGMENT;
   }
 
   /**
+   * @return true if the log is open in read mode.
+   */
+  public synchronized boolean isOpenForRead() {
+    return state == State.OPEN_FOR_READING;
+  }
+
+  /**
    * Shutdown the file store.
    */
   synchronized void close() {
@@ -242,7 +318,8 @@
    */
   void logEdit(final FSEditLogOp op) {
     synchronized (this) {
-      assert state != State.CLOSED;
+      assert isOpenForWrite() :
+        "bad state: " + state;
       
       // wait if an automatic sync is scheduled
       waitIfAutoSyncScheduled();
@@ -329,7 +406,7 @@
   /**
    * Return the transaction ID of the last transaction written to the log.
    */
-  synchronized long getLastWrittenTxId() {
+  public synchronized long getLastWrittenTxId() {
     return txid;
   }
   
@@ -337,7 +414,7 @@
    * @return the first transaction ID in the current log segment
    */
   synchronized long getCurSegmentTxId() {
-    Preconditions.checkState(state == State.IN_SEGMENT,
+    Preconditions.checkState(isSegmentOpen(),
         "Bad state: %s", state);
     return curSegmentTxId;
   }
@@ -549,6 +626,13 @@
     logEdit(op);
   }
   
+  public void logUpdateBlocks(String path, INodeFileUnderConstruction file) {
+    UpdateBlocksOp op = UpdateBlocksOp.getInstance()
+      .setPath(path)
+      .setBlocks(file.getBlocks());
+    logEdit(op);
+  }
+  
   /** 
    * Add create directory record to edit log
    */
@@ -724,16 +808,25 @@
    * Used only by unit tests.
    */
   @VisibleForTesting
-  List<JournalAndStream> getJournals() {
+  synchronized List<JournalAndStream> getJournals() {
     return journalSet.getAllJournalStreams();
   }
   
   /**
+   * Used only by tests.
+   */
+  @VisibleForTesting
+  synchronized public JournalSet getJournalSet() {
+    return journalSet;
+  }
+  
+  /**
    * Used only by unit tests.
    */
   @VisibleForTesting
   synchronized void setRuntimeForTesting(Runtime runtime) {
     this.runtime = runtime;
+    this.journalSet.setRuntimeForTesting(runtime);
   }
 
   /**
@@ -796,7 +889,7 @@
       editLogStream = journalSet.startLogSegment(segmentTxId);
     } catch (IOException ex) {
       throw new IOException("Unable to start log segment " +
-          segmentTxId + ": no journals successfully started.");
+          segmentTxId + ": too few journals successfully started.", ex);
     }
     
     curSegmentTxId = segmentTxId;
@@ -815,7 +908,7 @@
    */
   synchronized void endCurrentLogSegment(boolean writeEndTxn) {
     LOG.info("Ending log segment " + curSegmentTxId);
-    Preconditions.checkState(state == State.IN_SEGMENT,
+    Preconditions.checkState(isSegmentOpen(),
         "Bad state: %s", state);
     
     if (writeEndTxn) {
@@ -847,6 +940,7 @@
       if (editLogStream != null) {
         editLogStream.abort();
         editLogStream = null;
+        state = State.BETWEEN_LOG_SEGMENTS;
       }
     } catch (IOException e) {
       LOG.warn("All journals failed to abort", e);
@@ -856,17 +950,14 @@
   /**
    * Archive any log files that are older than the given txid.
    */
-  public void purgeLogsOlderThan(final long minTxIdToKeep) {
-    synchronized (this) {
-      // synchronized to prevent findbugs warning about inconsistent
-      // synchronization. This will be JIT-ed out if asserts are
-      // off.
-      assert curSegmentTxId == HdfsConstants.INVALID_TXID || // on format this is no-op
-        minTxIdToKeep <= curSegmentTxId :
-        "cannot purge logs older than txid " + minTxIdToKeep +
-        " when current segment starts at " + curSegmentTxId;
-    }
+  public synchronized void purgeLogsOlderThan(final long minTxIdToKeep) {
+    assert curSegmentTxId == HdfsConstants.INVALID_TXID || // on format this is no-op
+      minTxIdToKeep <= curSegmentTxId :
+      "cannot purge logs older than txid " + minTxIdToKeep +
+      " when current segment starts at " + curSegmentTxId;
 
+    // This could be improved to not need synchronization. But currently,
+    // journalSet is not threadsafe, so we need to synchronize this method.
     try {
       journalSet.purgeLogsOlderThan(minTxIdToKeep);
     } catch (IOException ex) {
@@ -898,8 +989,8 @@
 
 
   // sets the initial capacity of the flush buffer.
-  public void setOutputBufferCapacity(int size) {
-      journalSet.setOutputBufferCapacity(size);
+  synchronized void setOutputBufferCapacity(int size) {
+    journalSet.setOutputBufferCapacity(size);
   }
 
   /**
@@ -975,32 +1066,45 @@
   /**
    * Run recovery on all journals to recover any unclosed segments
    */
-  void recoverUnclosedStreams() {
+  synchronized void recoverUnclosedStreams() {
+    Preconditions.checkState(
+        state == State.BETWEEN_LOG_SEGMENTS,
+        "May not recover segments - wrong state: %s", state);
     try {
       journalSet.recoverUnfinalizedSegments();
     } catch (IOException ex) {
       // All journals have failed, it is handled in logSync.
     }
   }
+  
+  Collection<EditLogInputStream> selectInputStreams(long fromTxId,
+      long toAtLeastTxId) throws IOException {
+    return selectInputStreams(fromTxId, toAtLeastTxId, true);
+  }
 
   /**
    * Select a list of input streams to load.
+   * 
    * @param fromTxId first transaction in the selected streams
    * @param toAtLeast the selected streams must contain this transaction
+   * @param inProgessOk set to true if in-progress streams are OK
    */
-  Collection<EditLogInputStream> selectInputStreams(long fromTxId,
-      long toAtLeastTxId) throws IOException {
+  public synchronized Collection<EditLogInputStream> selectInputStreams(long fromTxId,
+      long toAtLeastTxId, boolean inProgressOk) throws IOException {
     List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
-    EditLogInputStream stream = journalSet.getInputStream(fromTxId);
+    EditLogInputStream stream = journalSet.getInputStream(fromTxId, inProgressOk);
     while (stream != null) {
-      fromTxId = stream.getLastTxId() + 1;
       streams.add(stream);
-      stream = journalSet.getInputStream(fromTxId);
+      // We're now looking for a higher range, so reset the fromTxId
+      fromTxId = stream.getLastTxId() + 1;
+      stream = journalSet.getInputStream(fromTxId, inProgressOk);
     }
+    
     if (fromTxId <= toAtLeastTxId) {
       closeAllStreams(streams);
-      throw new IOException("No non-corrupt logs for txid " 
-                            + fromTxId);
+      throw new IOException(String.format("Gap in transactions. Expected to "
+          + "be able to read up until at least txid %d but unable to find any "
+          + "edit logs containing txid %d", toAtLeastTxId, fromTxId));
     }
     return streams;
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index b93942d..7c24107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -28,6 +28,7 @@
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
@@ -36,6 +37,7 @@
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
@@ -54,9 +56,12 @@
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.Holder;
+import org.apache.hadoop.io.IOUtils;
+
 import com.google.common.base.Joiner;
 
 @InterfaceAudience.Private
@@ -73,40 +78,32 @@
    * This is where we apply edits that we've been writing to disk all
    * along.
    */
-  int loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
-  throws IOException {
-    long startTime = now();
-    int numEdits = loadFSEdits(edits, true, expectedStartingTxId);
-    FSImage.LOG.info("Edits file " + edits.getName() 
-        + " of size " + edits.length() + " edits # " + numEdits 
-        + " loaded in " + (now()-startTime)/1000 + " seconds.");
-    return numEdits;
-  }
-
-  int loadFSEdits(EditLogInputStream edits, boolean closeOnExit,
-                  long expectedStartingTxId)
+  long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
       throws IOException {
-    int numEdits = 0;
+    long numEdits = 0;
     int logVersion = edits.getVersion();
 
+    fsNamesys.writeLock();
     try {
+      long startTime = now();
       numEdits = loadEditRecords(logVersion, edits, false, 
                                  expectedStartingTxId);
+      FSImage.LOG.info("Edits file " + edits.getName() 
+          + " of size " + edits.length() + " edits # " + numEdits 
+          + " loaded in " + (now()-startTime)/1000 + " seconds.");
     } finally {
-      if(closeOnExit) {
-        edits.close();
-      }
+      edits.close();
+      fsNamesys.writeUnlock();
     }
     
     return numEdits;
   }
 
-  @SuppressWarnings("deprecation")
-  int loadEditRecords(int logVersion, EditLogInputStream in, boolean closeOnExit,
+  long loadEditRecords(int logVersion, EditLogInputStream in, boolean closeOnExit,
                       long expectedStartingTxId)
-      throws IOException {
+      throws IOException, EditLogInputException {
     FSDirectory fsDir = fsNamesys.dir;
-    int numEdits = 0;
+    long numEdits = 0;
 
     EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts =
       new EnumMap<FSEditLogOpCodes, Holder<Integer>>(FSEditLogOpCodes.class);
@@ -120,9 +117,20 @@
     long txId = expectedStartingTxId - 1;
     try {
       try {
-        FSEditLogOp op;
-        while ((op = in.readOp()) != null) {
-          recentOpcodeOffsets[numEdits % recentOpcodeOffsets.length] =
+        while (true) {
+          FSEditLogOp op;
+          try {
+            if ((op = in.readOp()) == null) {
+              break;
+            }
+          } catch (IOException ioe) {
+            long badTxId = txId + 1; // because txId hasn't been incremented yet
+            String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets, badTxId);
+            FSImage.LOG.error(errorMessage);
+            throw new EditLogInputException(errorMessage,
+                ioe, numEdits);
+          }
+          recentOpcodeOffsets[(int)(numEdits % recentOpcodeOffsets.length)] =
             in.getPosition();
           if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
             long expectedTxId = txId + 1;
@@ -133,310 +141,442 @@
             }
           }
 
-          numEdits++;
           incrOpCount(op.opCode, opCounts);
-          switch (op.opCode) {
-          case OP_ADD:
-          case OP_CLOSE: {
-            AddCloseOp addCloseOp = (AddCloseOp)op;
-
-            // versions > 0 support per file replication
-            // get name and replication
-            final short replication  = fsNamesys.getBlockManager(
-                ).adjustReplication(addCloseOp.replication);
-
-            long blockSize = addCloseOp.blockSize;
-            BlockInfo blocks[] = new BlockInfo[addCloseOp.blocks.length];
-            for (int i = 0; i < addCloseOp.blocks.length; i++) {
-              if(addCloseOp.opCode == FSEditLogOpCodes.OP_ADD
-                 && i == addCloseOp.blocks.length-1) {
-                blocks[i] = new BlockInfoUnderConstruction(addCloseOp.blocks[i],
-                                                           replication);
-              } else {
-                blocks[i] = new BlockInfo(addCloseOp.blocks[i], replication);
-              }
-            }
-
-            PermissionStatus permissions = fsNamesys.getUpgradePermission();
-            if (addCloseOp.permissions != null) {
-              permissions = addCloseOp.permissions;
-            }
-
-
-            // Older versions of HDFS does not store the block size in inode.
-            // If the file has more than one block, use the size of the
-            // first block as the blocksize. Otherwise use the default
-            // block size.
-            if (-8 <= logVersion && blockSize == 0) {
-              if (blocks.length > 1) {
-                blockSize = blocks[0].getNumBytes();
-              } else {
-                long first = ((blocks.length == 1)? blocks[0].getNumBytes(): 0);
-                blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
-              }
-            }
-
-
-            // The open lease transaction re-creates a file if necessary.
-            // Delete the file if it already exists.
-            if (FSNamesystem.LOG.isDebugEnabled()) {
-              FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
-                  " numblocks : " + blocks.length +
-                  " clientHolder " + addCloseOp.clientName +
-                  " clientMachine " + addCloseOp.clientMachine);
-            }
-
-            // There are four cases here:
-            // 1. OP_ADD to create a new file
-            // 2. OP_ADD to update file blocks
-            // 3. OP_ADD to open file for append
-            // 4. OP_CLOSE to close the file
-
-            // See if the file already exists
-            INodeFile oldFile = fsDir.getFileINode(addCloseOp.path);
-            if (oldFile == null) { // OP_ADD for a new file
-              assert addCloseOp.opCode == FSEditLogOpCodes.OP_ADD : 
-                "Expected opcode OP_ADD, but got " + addCloseOp.opCode;
-              fsDir.unprotectedAddFile(
-                  addCloseOp.path, permissions, blocks, replication,
-                  addCloseOp.mtime, addCloseOp.atime, blockSize,
-                  addCloseOp.clientName, addCloseOp.clientMachine);
-            } else {
-              fsDir.updateFile(oldFile, addCloseOp.path, blocks,
-                  addCloseOp.mtime, addCloseOp.atime);
-              if(addCloseOp.opCode == FSEditLogOpCodes.OP_CLOSE) {  // OP_CLOSE
-                if (!oldFile.isUnderConstruction() &&
-                    logVersion <= LayoutVersion.BUGFIX_HDFS_2991_VERSION) {
-                  // There was a bug (HDFS-2991) in hadoop < 0.23.1 where OP_CLOSE
-                  // could show up twice in a row. But after that version, this
-                  // should be fixed, so we should treat it as an error.
-                  throw new IOException(
-                      "File is not under construction: " + addCloseOp.path);
-                }
-                fsNamesys.getBlockManager().completeBlock(
-                    oldFile, blocks.length-1, true);
-                
-                if (oldFile.isUnderConstruction()) {
-                  INodeFile newFile =
-                    ((INodeFileUnderConstruction)oldFile).convertToInodeFile();
-                  fsDir.replaceNode(addCloseOp.path, oldFile, newFile);
-                }
-              } else if(! oldFile.isUnderConstruction()) {  // OP_ADD for append
-                INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
-                    oldFile.getLocalNameBytes(),
-                    oldFile.getReplication(), 
-                    oldFile.getModificationTime(),
-                    oldFile.getPreferredBlockSize(),
-                    oldFile.getBlocks(),
-                    oldFile.getPermissionStatus(),
-                    addCloseOp.clientName,
-                    addCloseOp.clientMachine,
-                    null);
-                fsDir.replaceNode(addCloseOp.path, oldFile, cons);
-              }
-            }
-            // Update file lease
-            if(addCloseOp.opCode == FSEditLogOpCodes.OP_ADD) {
-              fsNamesys.leaseManager.addLease(addCloseOp.clientName, addCloseOp.path);
-            } else {  // Ops.OP_CLOSE
-              if (oldFile.isUnderConstruction()) {
-                fsNamesys.leaseManager.removeLease(
-                    ((INodeFileUnderConstruction)oldFile).getClientName(), addCloseOp.path);
-              }
-            }
-            break;
+          try {
+            applyEditLogOp(op, fsDir, logVersion);
+          } catch (Throwable t) {
+            // Catch Throwable because in the case of a truly corrupt edits log, any
+            // sort of error might be thrown (NumberFormat, NullPointer, EOF, etc.)
+            String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets, txId);
+            FSImage.LOG.error(errorMessage);
+            throw new IOException(errorMessage, t);
           }
-          case OP_SET_REPLICATION: {
-            SetReplicationOp setReplicationOp = (SetReplicationOp)op;
-            short replication = fsNamesys.getBlockManager().adjustReplication(
-                setReplicationOp.replication);
-            fsDir.unprotectedSetReplication(setReplicationOp.path,
-                                            replication, null);
-            break;
-          }
-          case OP_CONCAT_DELETE: {
-            ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
-            fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs,
-                concatDeleteOp.timestamp);
-            break;
-          }
-          case OP_RENAME_OLD: {
-            RenameOldOp renameOp = (RenameOldOp)op;
-            HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
-            fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
-                                      renameOp.timestamp);
-            fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
-            break;
-          }
-          case OP_DELETE: {
-            DeleteOp deleteOp = (DeleteOp)op;
-            fsDir.unprotectedDelete(deleteOp.path, deleteOp.timestamp);
-            break;
-          }
-          case OP_MKDIR: {
-            MkdirOp mkdirOp = (MkdirOp)op;
-            PermissionStatus permissions = fsNamesys.getUpgradePermission();
-            if (mkdirOp.permissions != null) {
-              permissions = mkdirOp.permissions;
-            }
-
-            fsDir.unprotectedMkdir(mkdirOp.path, permissions,
-                                   mkdirOp.timestamp);
-            break;
-          }
-          case OP_SET_GENSTAMP: {
-            SetGenstampOp setGenstampOp = (SetGenstampOp)op;
-            fsNamesys.setGenerationStamp(setGenstampOp.genStamp);
-            break;
-          }
-          case OP_SET_PERMISSIONS: {
-            SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
-            fsDir.unprotectedSetPermission(setPermissionsOp.src,
-                                           setPermissionsOp.permissions);
-            break;
-          }
-          case OP_SET_OWNER: {
-            SetOwnerOp setOwnerOp = (SetOwnerOp)op;
-            fsDir.unprotectedSetOwner(setOwnerOp.src, setOwnerOp.username,
-                                      setOwnerOp.groupname);
-            break;
-          }
-          case OP_SET_NS_QUOTA: {
-            SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
-            fsDir.unprotectedSetQuota(setNSQuotaOp.src,
-                                      setNSQuotaOp.nsQuota,
-                                      HdfsConstants.QUOTA_DONT_SET);
-            break;
-          }
-          case OP_CLEAR_NS_QUOTA: {
-            ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
-            fsDir.unprotectedSetQuota(clearNSQuotaOp.src,
-                                      HdfsConstants.QUOTA_RESET,
-                                      HdfsConstants.QUOTA_DONT_SET);
-            break;
-          }
-
-          case OP_SET_QUOTA:
-            SetQuotaOp setQuotaOp = (SetQuotaOp)op;
-            fsDir.unprotectedSetQuota(setQuotaOp.src,
-                                      setQuotaOp.nsQuota,
-                                      setQuotaOp.dsQuota);
-            break;
-
-          case OP_TIMES: {
-            TimesOp timesOp = (TimesOp)op;
-
-            fsDir.unprotectedSetTimes(timesOp.path,
-                                      timesOp.mtime,
-                                      timesOp.atime, true);
-            break;
-          }
-          case OP_SYMLINK: {
-            SymlinkOp symlinkOp = (SymlinkOp)op;
-            fsDir.unprotectedSymlink(symlinkOp.path, symlinkOp.value,
-                                     symlinkOp.mtime, symlinkOp.atime,
-                                     symlinkOp.permissionStatus);
-            break;
-          }
-          case OP_RENAME: {
-            RenameOp renameOp = (RenameOp)op;
-
-            HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
-            fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
-                                      renameOp.timestamp, renameOp.options);
-            fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
-            break;
-          }
-          case OP_GET_DELEGATION_TOKEN: {
-            GetDelegationTokenOp getDelegationTokenOp
-              = (GetDelegationTokenOp)op;
-
-            fsNamesys.getDelegationTokenSecretManager()
-              .addPersistedDelegationToken(getDelegationTokenOp.token,
-                                           getDelegationTokenOp.expiryTime);
-            break;
-          }
-          case OP_RENEW_DELEGATION_TOKEN: {
-            RenewDelegationTokenOp renewDelegationTokenOp
-              = (RenewDelegationTokenOp)op;
-            fsNamesys.getDelegationTokenSecretManager()
-              .updatePersistedTokenRenewal(renewDelegationTokenOp.token,
-                                           renewDelegationTokenOp.expiryTime);
-            break;
-          }
-          case OP_CANCEL_DELEGATION_TOKEN: {
-            CancelDelegationTokenOp cancelDelegationTokenOp
-              = (CancelDelegationTokenOp)op;
-            fsNamesys.getDelegationTokenSecretManager()
-                .updatePersistedTokenCancellation(
-                    cancelDelegationTokenOp.token);
-            break;
-          }
-          case OP_UPDATE_MASTER_KEY: {
-            UpdateMasterKeyOp updateMasterKeyOp = (UpdateMasterKeyOp)op;
-            fsNamesys.getDelegationTokenSecretManager()
-              .updatePersistedMasterKey(updateMasterKeyOp.key);
-            break;
-          }
-          case OP_REASSIGN_LEASE: {
-            ReassignLeaseOp reassignLeaseOp = (ReassignLeaseOp)op;
-
-            Lease lease = fsNamesys.leaseManager.getLease(
-                reassignLeaseOp.leaseHolder);
-            INodeFileUnderConstruction pendingFile =
-                (INodeFileUnderConstruction) fsDir.getFileINode(
-                    reassignLeaseOp.path);
-            fsNamesys.reassignLeaseInternal(lease,
-                reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
-            break;
-          }
-          case OP_START_LOG_SEGMENT:
-          case OP_END_LOG_SEGMENT: {
-            // no data in here currently.
-            break;
-          }
-          case OP_DATANODE_ADD:
-          case OP_DATANODE_REMOVE:
-            break;
-          default:
-            throw new IOException("Invalid operation read " + op.opCode);
-          }
+          numEdits++;
         }
-
       } catch (IOException ex) {
         check203UpgradeFailure(logVersion, ex);
       } finally {
         if(closeOnExit)
           in.close();
       }
-    } catch (Throwable t) {
-      // Catch Throwable because in the case of a truly corrupt edits log, any
-      // sort of error might be thrown (NumberFormat, NullPointer, EOF, etc.)
-      StringBuilder sb = new StringBuilder();
-      sb.append("Error replaying edit log at offset " + in.getPosition());
-      sb.append("On transaction ID ").append(txId);
-      if (recentOpcodeOffsets[0] != -1) {
-        Arrays.sort(recentOpcodeOffsets);
-        sb.append("\nRecent opcode offsets:");
-        for (long offset : recentOpcodeOffsets) {
-          if (offset != -1) {
-            sb.append(' ').append(offset);
-          }
-        }
-      }
-      String errorMessage = sb.toString();
-      FSImage.LOG.error(errorMessage);
-      throw new IOException(errorMessage, t);
     } finally {
       fsDir.writeUnlock();
       fsNamesys.writeUnlock();
-    }
-    if (FSImage.LOG.isDebugEnabled()) {
-      dumpOpCounts(opCounts);
+      if (FSImage.LOG.isDebugEnabled()) {
+        dumpOpCounts(opCounts);
+      }
     }
     return numEdits;
   }
+  
+  @SuppressWarnings("deprecation")
+  private void applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
+      int logVersion) throws IOException {
+    switch (op.opCode) {
+    case OP_ADD: {
+      AddCloseOp addCloseOp = (AddCloseOp)op;
+      if (FSNamesystem.LOG.isDebugEnabled()) {
+        FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
+            " numblocks : " + addCloseOp.blocks.length +
+            " clientHolder " + addCloseOp.clientName +
+            " clientMachine " + addCloseOp.clientMachine);
+      }
+      // There three cases here:
+      // 1. OP_ADD to create a new file
+      // 2. OP_ADD to update file blocks
+      // 3. OP_ADD to open file for append
 
+      // See if the file already exists (persistBlocks call)
+      INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
+      INodeFile newFile = oldFile;
+      if (oldFile == null) { // this is OP_ADD on a new file (case 1)
+        // versions > 0 support per file replication
+        // get name and replication
+        final short replication  = fsNamesys.getBlockManager(
+            ).adjustReplication(addCloseOp.replication);
+        PermissionStatus permissions = fsNamesys.getUpgradePermission();
+        if (addCloseOp.permissions != null) {
+          permissions = addCloseOp.permissions;
+        }
+        long blockSize = addCloseOp.blockSize;
+
+        // Versions of HDFS prior to 0.17 may log an OP_ADD transaction
+        // which includes blocks in it. When we update the minimum
+        // upgrade version to something more recent than 0.17, we can
+        // simplify this code by asserting that OP_ADD transactions
+        // don't have any blocks.
+        
+        // Older versions of HDFS does not store the block size in inode.
+        // If the file has more than one block, use the size of the
+        // first block as the blocksize. Otherwise use the default
+        // block size.
+        if (-8 <= logVersion && blockSize == 0) {
+          if (addCloseOp.blocks.length > 1) {
+            blockSize = addCloseOp.blocks[0].getNumBytes();
+          } else {
+            long first = ((addCloseOp.blocks.length == 1)?
+                addCloseOp.blocks[0].getNumBytes(): 0);
+            blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
+          }
+        }
+
+        // add to the file tree
+        newFile = (INodeFile)fsDir.unprotectedAddFile(
+            addCloseOp.path, permissions,
+            replication, addCloseOp.mtime,
+            addCloseOp.atime, blockSize,
+            true, addCloseOp.clientName, addCloseOp.clientMachine);
+        fsNamesys.leaseManager.addLease(addCloseOp.clientName, addCloseOp.path);
+
+      } else { // This is OP_ADD on an existing file
+        if (!oldFile.isUnderConstruction()) {
+          // This is case 3: a call to append() on an already-closed file.
+          if (FSNamesystem.LOG.isDebugEnabled()) {
+            FSNamesystem.LOG.debug("Reopening an already-closed file " +
+                "for append");
+          }
+          fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
+              addCloseOp.clientName, addCloseOp.clientMachine, null,
+              false);
+          newFile = getINodeFile(fsDir, addCloseOp.path);
+        }
+      }
+      // Fall-through for case 2.
+      // Regardless of whether it's a new file or an updated file,
+      // update the block list.
+      
+      // Update the salient file attributes.
+      newFile.setAccessTime(addCloseOp.atime);
+      newFile.setModificationTimeForce(addCloseOp.mtime);
+      updateBlocks(fsDir, addCloseOp, newFile);
+      break;
+    }
+    case OP_CLOSE: {
+      AddCloseOp addCloseOp = (AddCloseOp)op;
+      
+      if (FSNamesystem.LOG.isDebugEnabled()) {
+        FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
+            " numblocks : " + addCloseOp.blocks.length +
+            " clientHolder " + addCloseOp.clientName +
+            " clientMachine " + addCloseOp.clientMachine);
+      }
+
+      INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
+      if (oldFile == null) {
+        throw new IOException("Operation trying to close non-existent file " +
+            addCloseOp.path);
+      }
+      
+      // Update in-memory data structures
+      updateBlocks(fsDir, addCloseOp, oldFile);
+
+      // Now close the file
+      if (!oldFile.isUnderConstruction() &&
+          logVersion <= LayoutVersion.BUGFIX_HDFS_2991_VERSION) {
+        // There was a bug (HDFS-2991) in hadoop < 0.23.1 where OP_CLOSE
+        // could show up twice in a row. But after that version, this
+        // should be fixed, so we should treat it as an error.
+        throw new IOException(
+            "File is not under construction: " + addCloseOp.path);
+      }
+      // One might expect that you could use removeLease(holder, path) here,
+      // but OP_CLOSE doesn't serialize the holder. So, remove by path.
+      if (oldFile.isUnderConstruction()) {
+        INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
+        fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
+        INodeFile newFile = ucFile.convertToInodeFile();
+        fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
+      }
+      break;
+    }
+    case OP_UPDATE_BLOCKS: {
+      UpdateBlocksOp updateOp = (UpdateBlocksOp)op;
+      if (FSNamesystem.LOG.isDebugEnabled()) {
+        FSNamesystem.LOG.debug(op.opCode + ": " + updateOp.path +
+            " numblocks : " + updateOp.blocks.length);
+      }
+      INodeFile oldFile = getINodeFile(fsDir, updateOp.path);
+      if (oldFile == null) {
+        throw new IOException(
+            "Operation trying to update blocks in non-existent file " +
+            updateOp.path);
+      }
+      
+      // Update in-memory data structures
+      updateBlocks(fsDir, updateOp, oldFile);
+      break;
+    }
+      
+    case OP_SET_REPLICATION: {
+      SetReplicationOp setReplicationOp = (SetReplicationOp)op;
+      short replication = fsNamesys.getBlockManager().adjustReplication(
+          setReplicationOp.replication);
+      fsDir.unprotectedSetReplication(setReplicationOp.path,
+                                      replication, null);
+      break;
+    }
+    case OP_CONCAT_DELETE: {
+      ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
+      fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs,
+          concatDeleteOp.timestamp);
+      break;
+    }
+    case OP_RENAME_OLD: {
+      RenameOldOp renameOp = (RenameOldOp)op;
+      HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
+      fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
+                                renameOp.timestamp);
+      fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
+      break;
+    }
+    case OP_DELETE: {
+      DeleteOp deleteOp = (DeleteOp)op;
+      fsDir.unprotectedDelete(deleteOp.path, deleteOp.timestamp);
+      break;
+    }
+    case OP_MKDIR: {
+      MkdirOp mkdirOp = (MkdirOp)op;
+      PermissionStatus permissions = fsNamesys.getUpgradePermission();
+      if (mkdirOp.permissions != null) {
+        permissions = mkdirOp.permissions;
+      }
+
+      fsDir.unprotectedMkdir(mkdirOp.path, permissions,
+                             mkdirOp.timestamp);
+      break;
+    }
+    case OP_SET_GENSTAMP: {
+      SetGenstampOp setGenstampOp = (SetGenstampOp)op;
+      fsNamesys.setGenerationStamp(setGenstampOp.genStamp);
+      break;
+    }
+    case OP_SET_PERMISSIONS: {
+      SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
+      fsDir.unprotectedSetPermission(setPermissionsOp.src,
+                                     setPermissionsOp.permissions);
+      break;
+    }
+    case OP_SET_OWNER: {
+      SetOwnerOp setOwnerOp = (SetOwnerOp)op;
+      fsDir.unprotectedSetOwner(setOwnerOp.src, setOwnerOp.username,
+                                setOwnerOp.groupname);
+      break;
+    }
+    case OP_SET_NS_QUOTA: {
+      SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
+      fsDir.unprotectedSetQuota(setNSQuotaOp.src,
+                                setNSQuotaOp.nsQuota,
+                                HdfsConstants.QUOTA_DONT_SET);
+      break;
+    }
+    case OP_CLEAR_NS_QUOTA: {
+      ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
+      fsDir.unprotectedSetQuota(clearNSQuotaOp.src,
+                                HdfsConstants.QUOTA_RESET,
+                                HdfsConstants.QUOTA_DONT_SET);
+      break;
+    }
+
+    case OP_SET_QUOTA:
+      SetQuotaOp setQuotaOp = (SetQuotaOp)op;
+      fsDir.unprotectedSetQuota(setQuotaOp.src,
+                                setQuotaOp.nsQuota,
+                                setQuotaOp.dsQuota);
+      break;
+
+    case OP_TIMES: {
+      TimesOp timesOp = (TimesOp)op;
+
+      fsDir.unprotectedSetTimes(timesOp.path,
+                                timesOp.mtime,
+                                timesOp.atime, true);
+      break;
+    }
+    case OP_SYMLINK: {
+      SymlinkOp symlinkOp = (SymlinkOp)op;
+      fsDir.unprotectedSymlink(symlinkOp.path, symlinkOp.value,
+                               symlinkOp.mtime, symlinkOp.atime,
+                               symlinkOp.permissionStatus);
+      break;
+    }
+    case OP_RENAME: {
+      RenameOp renameOp = (RenameOp)op;
+
+      HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
+      fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
+                                renameOp.timestamp, renameOp.options);
+      fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
+      break;
+    }
+    case OP_GET_DELEGATION_TOKEN: {
+      GetDelegationTokenOp getDelegationTokenOp
+        = (GetDelegationTokenOp)op;
+
+      fsNamesys.getDelegationTokenSecretManager()
+        .addPersistedDelegationToken(getDelegationTokenOp.token,
+                                     getDelegationTokenOp.expiryTime);
+      break;
+    }
+    case OP_RENEW_DELEGATION_TOKEN: {
+      RenewDelegationTokenOp renewDelegationTokenOp
+        = (RenewDelegationTokenOp)op;
+      fsNamesys.getDelegationTokenSecretManager()
+        .updatePersistedTokenRenewal(renewDelegationTokenOp.token,
+                                     renewDelegationTokenOp.expiryTime);
+      break;
+    }
+    case OP_CANCEL_DELEGATION_TOKEN: {
+      CancelDelegationTokenOp cancelDelegationTokenOp
+        = (CancelDelegationTokenOp)op;
+      fsNamesys.getDelegationTokenSecretManager()
+          .updatePersistedTokenCancellation(
+              cancelDelegationTokenOp.token);
+      break;
+    }
+    case OP_UPDATE_MASTER_KEY: {
+      UpdateMasterKeyOp updateMasterKeyOp = (UpdateMasterKeyOp)op;
+      fsNamesys.getDelegationTokenSecretManager()
+        .updatePersistedMasterKey(updateMasterKeyOp.key);
+      break;
+    }
+    case OP_REASSIGN_LEASE: {
+      ReassignLeaseOp reassignLeaseOp = (ReassignLeaseOp)op;
+
+      Lease lease = fsNamesys.leaseManager.getLease(
+          reassignLeaseOp.leaseHolder);
+      INodeFileUnderConstruction pendingFile =
+          (INodeFileUnderConstruction) fsDir.getFileINode(
+              reassignLeaseOp.path);
+      fsNamesys.reassignLeaseInternal(lease,
+          reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
+      break;
+    }
+    case OP_START_LOG_SEGMENT:
+    case OP_END_LOG_SEGMENT: {
+      // no data in here currently.
+      break;
+    }
+    case OP_DATANODE_ADD:
+    case OP_DATANODE_REMOVE:
+      break;
+    default:
+      throw new IOException("Invalid operation read " + op.opCode);
+    }
+  }
+  
+  private static String formatEditLogReplayError(EditLogInputStream in,
+      long recentOpcodeOffsets[], long txid) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Error replaying edit log at offset " + in.getPosition());
+    sb.append(" on transaction ID ").append(txid);
+    if (recentOpcodeOffsets[0] != -1) {
+      Arrays.sort(recentOpcodeOffsets);
+      sb.append("\nRecent opcode offsets:");
+      for (long offset : recentOpcodeOffsets) {
+        if (offset != -1) {
+          sb.append(' ').append(offset);
+        }
+      }
+    }
+    return sb.toString();
+  }
+  
+  private static INodeFile getINodeFile(FSDirectory fsDir, String path)
+      throws IOException {
+    INode inode = fsDir.getINode(path);
+    if (inode != null) {
+      if (!(inode instanceof INodeFile)) {
+        throw new IOException("Operation trying to get non-file " + path);
+      }
+    }
+    return (INodeFile)inode;
+  }
+  
+  /**
+   * Update in-memory data structures with new block information.
+   * @throws IOException
+   */
+  private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
+      INodeFile file) throws IOException {
+    // Update its block list
+    BlockInfo[] oldBlocks = file.getBlocks();
+    Block[] newBlocks = op.getBlocks();
+    String path = op.getPath();
+    
+    // Are we only updating the last block's gen stamp.
+    boolean isGenStampUpdate = oldBlocks.length == newBlocks.length;
+    
+    // First, update blocks in common
+    for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) {
+      BlockInfo oldBlock = oldBlocks[i];
+      Block newBlock = newBlocks[i];
+      
+      boolean isLastBlock = i == newBlocks.length - 1;
+      if (oldBlock.getBlockId() != newBlock.getBlockId() ||
+          (oldBlock.getGenerationStamp() != newBlock.getGenerationStamp() && 
+              !(isGenStampUpdate && isLastBlock))) {
+        throw new IOException("Mismatched block IDs or generation stamps, " + 
+            "attempting to replace block " + oldBlock + " with " + newBlock +
+            " as block # " + i + "/" + newBlocks.length + " of " +
+            path);
+      }
+      
+      oldBlock.setNumBytes(newBlock.getNumBytes());
+      boolean changeMade =
+        oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
+      oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
+      
+      if (oldBlock instanceof BlockInfoUnderConstruction &&
+          (!isLastBlock || op.shouldCompleteLastBlock())) {
+        changeMade = true;
+        fsNamesys.getBlockManager().forceCompleteBlock(
+            (INodeFileUnderConstruction)file,
+            (BlockInfoUnderConstruction)oldBlock);
+      }
+      if (changeMade) {
+        // The state or gen-stamp of the block has changed. So, we may be
+        // able to process some messages from datanodes that we previously
+        // were unable to process.
+        fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
+      }
+    }
+    
+    if (newBlocks.length < oldBlocks.length) {
+      // We're removing a block from the file, e.g. abandonBlock(...)
+      if (!file.isUnderConstruction()) {
+        throw new IOException("Trying to remove a block from file " +
+            path + " which is not under construction.");
+      }
+      if (newBlocks.length != oldBlocks.length - 1) {
+        throw new IOException("Trying to remove more than one block from file "
+            + path);
+      }
+      fsDir.unprotectedRemoveBlock(path,
+          (INodeFileUnderConstruction)file, oldBlocks[oldBlocks.length - 1]);
+    } else if (newBlocks.length > oldBlocks.length) {
+      // We're adding blocks
+      for (int i = oldBlocks.length; i < newBlocks.length; i++) {
+        Block newBlock = newBlocks[i];
+        BlockInfo newBI;
+        if (!op.shouldCompleteLastBlock()) {
+          // TODO: shouldn't this only be true for the last block?
+          // what about an old-version fsync() where fsync isn't called
+          // until several blocks in?
+          newBI = new BlockInfoUnderConstruction(
+              newBlock, file.getReplication());
+        } else {
+          // OP_CLOSE should add finalized blocks. This code path
+          // is only executed when loading edits written by prior
+          // versions of Hadoop. Current versions always log
+          // OP_ADD operations as each block is allocated.
+          newBI = new BlockInfo(newBlock, file.getReplication());
+        }
+        fsNamesys.getBlockManager().addINode(newBI, file);
+        file.addBlock(newBI);
+        fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
+      }
+    }
+  }
 
   private static void dumpOpCounts(
       EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts) {
@@ -517,19 +657,21 @@
       FSImage.LOG.debug("Caught exception after reading " + numValid +
           " ops from " + in + " while determining its valid length.", t);
     }
-    return new EditLogValidation(lastPos, firstTxId, lastTxId);
+    return new EditLogValidation(lastPos, firstTxId, lastTxId, false);
   }
   
   static class EditLogValidation {
-    private long validLength;
-    private long startTxId;
-    private long endTxId;
+    private final long validLength;
+    private final long startTxId;
+    private final long endTxId;
+    private final boolean corruptionDetected;
      
-    EditLogValidation(long validLength, 
-                      long startTxId, long endTxId) {
+    EditLogValidation(long validLength, long startTxId, long endTxId,
+        boolean corruptionDetected) {
       this.validLength = validLength;
       this.startTxId = startTxId;
       this.endTxId = endTxId;
+      this.corruptionDetected = corruptionDetected;
     }
     
     long getValidLength() { return validLength; }
@@ -545,6 +687,8 @@
       }
       return (endTxId - startTxId) + 1;
     }
+    
+    boolean hasCorruptHeader() { return corruptionDetected; }
   }
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index f075770c..949554d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -101,6 +101,7 @@
                       new LogSegmentOp(OP_START_LOG_SEGMENT));
         instances.put(OP_END_LOG_SEGMENT,
                       new LogSegmentOp(OP_END_LOG_SEGMENT));
+        instances.put(OP_UPDATE_BLOCKS, new UpdateBlocksOp());
         return instances;
       }
   };
@@ -128,8 +129,14 @@
   abstract void writeFields(DataOutputStream out)
       throws IOException;
 
+  static interface BlockListUpdatingOp {
+    Block[] getBlocks();
+    String getPath();
+    boolean shouldCompleteLastBlock();
+  }
+  
   @SuppressWarnings("unchecked")
-  static abstract class AddCloseOp extends FSEditLogOp {
+  static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
     int length;
     String path;
     short replication;
@@ -151,6 +158,10 @@
       this.path = path;
       return (T)this;
     }
+    
+    public String getPath() {
+      return path;
+    }
 
     <T extends AddCloseOp> T setReplication(short replication) {
       this.replication = replication;
@@ -176,6 +187,10 @@
       this.blocks = blocks;
       return (T)this;
     }
+    
+    public Block[] getBlocks() {
+      return blocks;
+    }
 
     <T extends AddCloseOp> T setPermissionStatus(PermissionStatus permissions) {
       this.permissions = permissions;
@@ -347,6 +362,10 @@
       return (AddOp)opInstances.get().get(OP_ADD);
     }
 
+    public boolean shouldCompleteLastBlock() {
+      return false;
+    }
+
     @Override
     public String toString() {
       StringBuilder builder = new StringBuilder();
@@ -365,6 +384,10 @@
       return (CloseOp)opInstances.get().get(OP_CLOSE);
     }
 
+    public boolean shouldCompleteLastBlock() {
+      return true;
+    }
+
     @Override
     public String toString() {
       StringBuilder builder = new StringBuilder();
@@ -373,6 +396,68 @@
       return builder.toString();
     }
   }
+  
+  static class UpdateBlocksOp extends FSEditLogOp implements BlockListUpdatingOp {
+    String path;
+    Block[] blocks;
+    
+    private UpdateBlocksOp() {
+      super(OP_UPDATE_BLOCKS);
+    }
+    
+    static UpdateBlocksOp getInstance() {
+      return (UpdateBlocksOp)opInstances.get()
+        .get(OP_UPDATE_BLOCKS);
+    }
+    
+    
+    UpdateBlocksOp setPath(String path) {
+      this.path = path;
+      return this;
+    }
+    
+    public String getPath() {
+      return path;
+    }
+
+    UpdateBlocksOp setBlocks(Block[] blocks) {
+      this.blocks = blocks;
+      return this;
+    }
+    
+    public Block[] getBlocks() {
+      return blocks;
+    }
+
+    @Override
+    void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeCompactBlockArray(blocks, out);
+    }
+    
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      path = FSImageSerialization.readString(in);
+      this.blocks = FSImageSerialization.readCompactBlockArray(
+          in, logVersion);
+    }
+
+    @Override
+    public boolean shouldCompleteLastBlock() {
+      return false;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("UpdateBlocksOp [path=")
+        .append(path)
+        .append(", blocks=")
+        .append(Arrays.toString(blocks))
+        .append("]");
+      return sb.toString();
+    }
+  }
 
   static class SetReplicationOp extends FSEditLogOp {
     String path;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
index 220c267..1f809c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
@@ -55,7 +55,8 @@
   OP_UPDATE_MASTER_KEY          ((byte) 21),
   OP_REASSIGN_LEASE             ((byte) 22),
   OP_END_LOG_SEGMENT            ((byte) 23),
-  OP_START_LOG_SEGMENT          ((byte) 24);
+  OP_START_LOG_SEGMENT          ((byte) 24),
+  OP_UPDATE_BLOCKS              ((byte) 25);
 
   private byte opCode;
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
index 463fca5..7fb3d4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
@@ -56,6 +56,8 @@
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -68,7 +70,7 @@
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class FSImage implements Closeable {
-  protected static final Log LOG = LogFactory.getLog(FSImage.class.getName());
+  public static final Log LOG = LogFactory.getLog(FSImage.class.getName());
 
   protected FSEditLog editLog = null;
   private boolean isUpgradeFinalized = false;
@@ -112,7 +114,8 @@
    * @throws IOException if directories are invalid.
    */
   protected FSImage(Configuration conf,
-                    Collection<URI> imageDirs, Collection<URI> editsDirs)
+                    Collection<URI> imageDirs,
+                    List<URI> editsDirs)
       throws IOException {
     this.conf = conf;
 
@@ -123,6 +126,12 @@
     }
 
     this.editLog = new FSEditLog(conf, storage, editsDirs);
+    String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
+    if (!HAUtil.isHAEnabled(conf, nameserviceId)) {
+      editLog.initJournalsForWrite();
+    } else {
+      editLog.initSharedJournalsForRead();
+    }
     
     archivalManager = new NNStorageRetentionManager(conf, storage, editLog);
   }
@@ -251,6 +260,11 @@
       StorageState curState;
       try {
         curState = sd.analyzeStorage(startOpt, storage);
+        String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
+        if (curState != StorageState.NORMAL && HAUtil.isHAEnabled(conf, nameserviceId)) {
+          throw new IOException("Cannot start an HA namenode with name dirs " +
+              "that need recovery. Dir: " + sd + " state: " + curState);
+        }
         // sd is locked but not opened
         switch(curState) {
         case NON_EXISTENT:
@@ -324,9 +338,9 @@
         File prevDir = sd.getPreviousDir();
         File tmpDir = sd.getPreviousTmp();
         assert curDir.exists() : "Current directory must exist.";
-        assert !prevDir.exists() : "prvious directory must not exist.";
-        assert !tmpDir.exists() : "prvious.tmp directory must not exist.";
-        assert !editLog.isOpen() : "Edits log must not be open.";
+        assert !prevDir.exists() : "previous directory must not exist.";
+        assert !tmpDir.exists() : "previous.tmp directory must not exist.";
+        assert !editLog.isSegmentOpen() : "Edits log must not be open.";
 
         // rename current to tmp
         NNStorage.rename(curDir, tmpDir);
@@ -469,7 +483,7 @@
   void doImportCheckpoint(FSNamesystem target) throws IOException {
     Collection<URI> checkpointDirs =
       FSImage.getCheckpointDirs(conf, null);
-    Collection<URI> checkpointEditsDirs =
+    List<URI> checkpointEditsDirs =
       FSImage.getCheckpointEditsDirs(conf, null);
 
     if (checkpointDirs == null || checkpointDirs.isEmpty()) {
@@ -519,11 +533,9 @@
     return editLog;
   }
 
-  void openEditLog() throws IOException {
+  void openEditLogForWrite() throws IOException {
     assert editLog != null : "editLog must be initialized";
-    Preconditions.checkState(!editLog.isOpen(),
-        "edit log should not yet be open");
-    editLog.open();
+    editLog.openForWrite();
     storage.writeTransactionIdFileToStorage(editLog.getCurSegmentTxId());
   };
   
@@ -564,12 +576,19 @@
 
     Iterable<EditLogInputStream> editStreams = null;
 
-    editLog.recoverUnclosedStreams();
+    if (editLog.isOpenForWrite()) {
+      // We only want to recover streams if we're going into Active mode.
+      editLog.recoverUnclosedStreams();
+    }
 
     if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
                                getLayoutVersion())) {
+      // If we're open for write, we're either non-HA or we're the active NN, so
+      // we better be able to load all the edits. If we're the standby NN, it's
+      // OK to not be able to read all of edits right now.
+      long toAtLeastTxId = editLog.isOpenForWrite() ? inspector.getMaxSeenTxId() : 0;
       editStreams = editLog.selectInputStreams(imageFile.getCheckpointTxId() + 1,
-                                               inspector.getMaxSeenTxId());
+          toAtLeastTxId, false);
     } else {
       editStreams = FSImagePreTransactionalStorageInspector
         .getEditLogStreams(storage);
@@ -644,12 +663,12 @@
    * Load the specified list of edit files into the image.
    * @return the number of transactions loaded
    */
-  protected long loadEdits(Iterable<EditLogInputStream> editStreams,
-                           FSNamesystem target) throws IOException {
+  public long loadEdits(Iterable<EditLogInputStream> editStreams,
+      FSNamesystem target) throws IOException, EditLogInputException {
     LOG.debug("About to load edits:\n  " + Joiner.on("\n  ").join(editStreams));
 
     long startingTxId = getLastAppliedTxId() + 1;
-    int numLoaded = 0;
+    long numLoaded = 0;
 
     try {    
       FSEditLogLoader loader = new FSEditLogLoader(target);
@@ -657,17 +676,26 @@
       // Load latest edits
       for (EditLogInputStream editIn : editStreams) {
         LOG.info("Reading " + editIn + " expecting start txid #" + startingTxId);
-        int thisNumLoaded = loader.loadFSEdits(editIn, startingTxId);
-        startingTxId += thisNumLoaded;
-        numLoaded += thisNumLoaded;
-        lastAppliedTxId += thisNumLoaded;
+        long thisNumLoaded = 0;
+        try {
+          thisNumLoaded = loader.loadFSEdits(editIn, startingTxId);
+        } catch (EditLogInputException elie) {
+          thisNumLoaded = elie.getNumEditsLoaded();
+          throw elie;
+        } finally {
+          // Update lastAppliedTxId even in case of error, since some ops may
+          // have been successfully applied before the error.
+          lastAppliedTxId = startingTxId + thisNumLoaded - 1;
+          startingTxId += thisNumLoaded;
+          numLoaded += thisNumLoaded;
+        }
       }
     } finally {
       FSEditLog.closeAllStreams(editStreams);
+      // update the counts
+      target.dir.updateCountForINodeWithQuota();   
     }
-
-    // update the counts
-    target.dir.updateCountForINodeWithQuota();    
+    
     return numLoaded;
   }
 
@@ -688,8 +716,7 @@
   
   /**
    * Load in the filesystem image from file. It's a big list of
-   * filenames and blocks.  Return whether we should
-   * "re-save" and consolidate the edit-logs
+   * filenames and blocks.
    */
   private void loadFSImage(File curFile, MD5Hash expectedMd5,
       FSNamesystem target) throws IOException {
@@ -786,16 +813,16 @@
    * Save the contents of the FS image to a new image file in each of the
    * current storage directories.
    */
-  synchronized void saveNamespace(FSNamesystem source) throws IOException {
+  public synchronized void saveNamespace(FSNamesystem source) throws IOException {
     assert editLog != null : "editLog must be initialized";
     storage.attemptRestoreRemovedStorage();
 
-    boolean editLogWasOpen = editLog.isOpen();
+    boolean editLogWasOpen = editLog.isSegmentOpen();
     
     if (editLogWasOpen) {
       editLog.endCurrentLogSegment(true);
     }
-    long imageTxId = editLog.getLastWrittenTxId();
+    long imageTxId = getLastAppliedOrWrittenTxId();
     try {
       saveFSImageInAllDirs(source, imageTxId);
       storage.writeAll();
@@ -812,7 +839,7 @@
     
   }
   
-  void cancelSaveNamespace(String reason)
+  public void cancelSaveNamespace(String reason)
       throws InterruptedException {
     SaveNamespaceContext ctx = curSaveNamespaceContext;
     if (ctx != null) {
@@ -1061,7 +1088,7 @@
     return Util.stringCollectionAsURIs(dirNames);
   }
 
-  static Collection<URI> getCheckpointEditsDirs(Configuration conf,
+  static List<URI> getCheckpointEditsDirs(Configuration conf,
       String defaultName) {
     Collection<String> dirNames = 
       conf.getStringCollection(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY);
@@ -1095,4 +1122,16 @@
     return lastAppliedTxId;
   }
 
+  public long getLastAppliedOrWrittenTxId() {
+    return Math.max(lastAppliedTxId,
+        editLog != null ? editLog.getLastWrittenTxId() : 0);
+  }
+
+  public void updateLastAppliedTxIdFromWritten() {
+    this.lastAppliedTxId = editLog.getLastWrittenTxId();
+  }
+
+  public synchronized long getMostRecentCheckpointTxId() {
+    return storage.getMostRecentCheckpointTxId();
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index 5b480305..f508433 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -40,6 +40,7 @@
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
 
 /**
  * Static utility functions for serializing various pieces of data in the correct
@@ -277,6 +278,49 @@
       ustr.getLength(), (byte) Path.SEPARATOR_CHAR);
   }
 
+
+  /**
+   * Write an array of blocks as compactly as possible. This uses
+   * delta-encoding for the generation stamp and size, following
+   * the principle that genstamp increases relatively slowly,
+   * and size is equal for all but the last block of a file.
+   */
+  public static void writeCompactBlockArray(
+      Block[] blocks, DataOutputStream out) throws IOException {
+    WritableUtils.writeVInt(out, blocks.length);
+    Block prev = null;
+    for (Block b : blocks) {
+      long szDelta = b.getNumBytes() -
+          (prev != null ? prev.getNumBytes() : 0);
+      long gsDelta = b.getGenerationStamp() -
+          (prev != null ? prev.getGenerationStamp() : 0);
+      out.writeLong(b.getBlockId()); // blockid is random
+      WritableUtils.writeVLong(out, szDelta);
+      WritableUtils.writeVLong(out, gsDelta);
+      prev = b;
+    }
+  }
+  
+  public static Block[] readCompactBlockArray(
+      DataInputStream in, int logVersion) throws IOException {
+    int num = WritableUtils.readVInt(in);
+    if (num < 0) {
+      throw new IOException("Invalid block array length: " + num);
+    }
+    Block prev = null;
+    Block[] ret = new Block[num];
+    for (int i = 0; i < num; i++) {
+      long id = in.readLong();
+      long sz = WritableUtils.readVLong(in) +
+          ((prev != null) ? prev.getNumBytes() : 0);
+      long gs = WritableUtils.readVLong(in) +
+          ((prev != null) ? prev.getGenerationStamp() : 0);
+      ret[i] = new Block(id, sz, gs);
+      prev = ret[i];
+    }
+    return ret;
+  }
+
   /**
    * DatanodeImage is used to store persistent information
    * about datanodes into the fsImage.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 604fbb0..f22f808 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -32,6 +32,8 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
@@ -47,10 +49,15 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERSIST_BLOCKS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
@@ -68,6 +75,7 @@
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
 import java.net.URI;
@@ -80,6 +88,7 @@
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -108,7 +117,10 @@
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -147,9 +159,18 @@
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
+import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
+import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
+import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -157,6 +178,7 @@
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.annotation.Metric;
 import org.apache.hadoop.metrics2.annotation.Metrics;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -170,13 +192,12 @@
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
-import com.google.common.base.Preconditions;
-
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 
 /***************************************************
  * FSNamesystem does the actual bookkeeping work for the
@@ -194,7 +215,7 @@
 @Metrics(context="dfs")
 public class FSNamesystem implements Namesystem, FSClusterStats,
     FSNamesystemMBean, NameNodeMXBean {
-  static final Log LOG = LogFactory.getLog(FSNamesystem.class);
+  public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
 
   private static final ThreadLocal<StringBuilder> auditBuffer =
     new ThreadLocal<StringBuilder>() {
@@ -243,14 +264,18 @@
   static final int DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED = 100;
   static int BLOCK_DELETION_INCREMENT = 1000;
   private boolean isPermissionEnabled;
+  private boolean persistBlocks;
   private UserGroupInformation fsOwner;
   private String supergroup;
   private PermissionStatus defaultPermission;
+  private boolean standbyShouldCheckpoint;
   
   // Scan interval is not configurable.
   private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL =
     TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS);
   private DelegationTokenSecretManager dtSecretManager;
+  private boolean alwaysUseDelegationTokensForTests;
+  
 
   //
   // Stores the correct file name hierarchy
@@ -264,7 +289,6 @@
 
   LeaseManager leaseManager = new LeaseManager(this); 
 
-  Daemon lmthread = null;   // LeaseMonitor thread
   Daemon smmthread = null;  // SafeModeMonitor thread
   
   Daemon nnrmthread = null; // NamenodeResourceMonitor thread
@@ -300,7 +324,26 @@
   // lock to protect FSNamesystem.
   private ReentrantReadWriteLock fsLock;
 
-  
+  /**
+   * Used when this NN is in standby state to read from the shared edit log.
+   */
+  private EditLogTailer editLogTailer = null;
+
+  /**
+   * Used when this NN is in standby state to perform checkpoints.
+   */
+  private StandbyCheckpointer standbyCheckpointer;
+
+  /**
+   * Reference to the NN's HAContext object. This is only set once
+   * {@link #startCommonServices(Configuration, HAContext)} is called. 
+   */
+  private HAContext haContext;
+
+  private boolean haEnabled;
+
+  private final Configuration conf;
+    
   /**
    * Instantiates an FSNamesystem loaded from the image and edits
    * directories specified in the passed Configuration.
@@ -310,9 +353,10 @@
    * @return an FSNamesystem which contains the loaded namespace
    * @throws IOException if loading fails
    */
-  public static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
+  public static FSNamesystem loadFromDisk(Configuration conf)
+    throws IOException {
     Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
-    Collection<URI> namespaceEditsDirs = 
+    List<URI> namespaceEditsDirs = 
       FSNamesystem.getNamespaceEditsDirs(conf);
 
     if (namespaceDirs.size() == 1) {
@@ -329,7 +373,9 @@
 
     long loadStart = now();
     StartupOption startOpt = NameNode.getStartupOption(conf);
-    namesystem.loadFSImage(startOpt, fsImage);
+    String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
+    namesystem.loadFSImage(startOpt, fsImage,
+      HAUtil.isHAEnabled(conf, nameserviceId));
     long timeTakenToLoadFSImage = now() - loadStart;
     LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
     NameNode.getNameNodeMetrics().setFsImageLoadTime(
@@ -348,6 +394,7 @@
    * @throws IOException on bad configuration
    */
   FSNamesystem(Configuration conf, FSImage fsImage) throws IOException {
+    this.conf = conf;
     try {
       initialize(conf, fsImage);
     } catch(IOException e) {
@@ -375,7 +422,7 @@
     this.safeMode = new SafeModeInfo(conf);
   }
 
-  void loadFSImage(StartupOption startOpt, FSImage fsImage)
+  void loadFSImage(StartupOption startOpt, FSImage fsImage, boolean haEnabled)
       throws IOException {
     // format before starting up if requested
     if (startOpt == StartupOption.FORMAT) {
@@ -385,43 +432,71 @@
       startOpt = StartupOption.REGULAR;
     }
     boolean success = false;
+    writeLock();
     try {
-      if (fsImage.recoverTransitionRead(startOpt, this)) {
+      // We shouldn't be calling saveNamespace if we've come up in standby state.
+      if (fsImage.recoverTransitionRead(startOpt, this) && !haEnabled) {
         fsImage.saveNamespace(this);
       }
-      fsImage.openEditLog();
+      // This will start a new log segment and write to the seen_txid file, so
+      // we shouldn't do it when coming up in standby state
+      if (!haEnabled) {
+        fsImage.openEditLogForWrite();
+      }
       
       success = true;
     } finally {
       if (!success) {
         fsImage.close();
       }
+      writeUnlock();
     }
     dir.imageLoadComplete();
   }
 
-  void activateSecretManager() throws IOException {
+  private void startSecretManager() {
     if (dtSecretManager != null) {
-      dtSecretManager.startThreads();
+      try {
+        dtSecretManager.startThreads();
+      } catch (IOException e) {
+        // Inability to start secret manager
+        // can't be recovered from.
+        throw new RuntimeException(e);
+      }
     }
   }
   
-  /**
-   * Activate FSNamesystem daemons.
-   */
-  void activate(Configuration conf) throws IOException {
-    this.registerMBean(); // register the MBean for the FSNamesystemState
+  private void startSecretManagerIfNecessary() {
+    boolean shouldRun = shouldUseDelegationTokens() &&
+      !isInSafeMode() && getEditLog().isOpenForWrite();
+    boolean running = dtSecretManager.isRunning();
+    if (shouldRun && !running) {
+      startSecretManager();
+    }
+  }
 
+  private void stopSecretManager() {
+    if (dtSecretManager != null) {
+      dtSecretManager.stopThreads();
+    }
+  }
+  
+  /** 
+   * Start services common to both active and standby states
+   * @param haContext 
+   * @throws IOException
+   */
+  void startCommonServices(Configuration conf, HAContext haContext) throws IOException {
+    this.registerMBean(); // register the MBean for the FSNamesystemState
     writeLock();
+    this.haContext = haContext;
     try {
       nnResourceChecker = new NameNodeResourceChecker(conf);
       checkAvailableResources();
-
+      assert safeMode != null &&
+        !safeMode.isPopulatingReplQueues();
       setBlockTotal();
       blockManager.activate(conf);
-
-      this.lmthread = new Daemon(leaseManager.new Monitor());
-      lmthread.start();
       this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
       nnrmthread.start();
     } finally {
@@ -431,24 +506,169 @@
     registerMXBean();
     DefaultMetricsSystem.instance().register(this);
   }
-
-  public static Collection<URI> getNamespaceDirs(Configuration conf) {
-    return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
-  }
   
-  public static Collection<URI> getNamespaceEditsDirs(Configuration conf) {
-    Collection<URI> editsDirs = getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_KEY);
-    if (editsDirs.isEmpty()) {
-      // If this is the case, no edit dirs have been explicitly configured.
-      // Image dirs are to be used for edits too.
-      return getNamespaceDirs(conf);
-    } else {
-      return editsDirs;
+  /** 
+   * Stop services common to both active and standby states
+   * @throws IOException
+   */
+  void stopCommonServices() {
+    writeLock();
+    try {
+      if (blockManager != null) blockManager.close();
+      if (nnrmthread != null) nnrmthread.interrupt();
+    } finally {
+      writeUnlock();
     }
   }
   
+  /**
+   * Start services required in active state
+   * @throws IOException
+   */
+  void startActiveServices() throws IOException {
+    LOG.info("Starting services required for active state");
+    writeLock();
+    try {
+      FSEditLog editLog = dir.fsImage.getEditLog();
+      
+      if (!editLog.isOpenForWrite()) {
+        // During startup, we're already open for write during initialization.
+        editLog.initJournalsForWrite();
+        // May need to recover
+        editLog.recoverUnclosedStreams();
+        
+        LOG.info("Catching up to latest edits from old active before " +
+            "taking over writer role in edits logs.");
+        editLogTailer.catchupDuringFailover();
+        
+        LOG.info("Reprocessing replication and invalidation queues...");
+        blockManager.getDatanodeManager().markAllDatanodesStale();
+        blockManager.clearQueues();
+        blockManager.processAllPendingDNMessages();
+        blockManager.processMisReplicatedBlocks();
+        
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("NameNode metadata after re-processing " +
+              "replication and invalidation queues during failover:\n" +
+              metaSaveAsString());
+        }
+        
+        long nextTxId = dir.fsImage.getLastAppliedTxId() + 1;
+        LOG.info("Will take over writing edit logs at txnid " + 
+            nextTxId);
+        editLog.setNextTxId(nextTxId);
+
+        dir.fsImage.editLog.openForWrite();
+      }
+      if (haEnabled) {
+        // Renew all of the leases before becoming active.
+        // This is because, while we were in standby mode,
+        // the leases weren't getting renewed on this NN.
+        // Give them all a fresh start here.
+        leaseManager.renewAllLeases();
+      }
+      leaseManager.startMonitor();
+      startSecretManagerIfNecessary();
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  private boolean shouldUseDelegationTokens() {
+    return UserGroupInformation.isSecurityEnabled() ||
+      alwaysUseDelegationTokensForTests;
+  }
+
+  /** 
+   * Stop services required in active state
+   * @throws InterruptedException
+   */
+  void stopActiveServices() {
+    LOG.info("Stopping services started for active state");
+    writeLock();
+    try {
+      stopSecretManager();
+      if (leaseManager != null) {
+        leaseManager.stopMonitor();
+      }
+      if (dir != null && dir.fsImage != null) {
+        if (dir.fsImage.editLog != null) {
+          dir.fsImage.editLog.close();
+        }
+        // Update the fsimage with the last txid that we wrote
+        // so that the tailer starts from the right spot.
+        dir.fsImage.updateLastAppliedTxIdFromWritten();
+      }
+    } finally {
+      writeUnlock();
+    }
+  }
+  
+  /** Start services required in standby state */
+  void startStandbyServices() {
+    LOG.info("Starting services required for standby state");
+    if (!dir.fsImage.editLog.isOpenForRead()) {
+      // During startup, we're already open for read.
+      dir.fsImage.editLog.initSharedJournalsForRead();
+    }
+    editLogTailer = new EditLogTailer(this);
+    editLogTailer.start();
+    if (standbyShouldCheckpoint) {
+      standbyCheckpointer = new StandbyCheckpointer(conf, this);
+      standbyCheckpointer.start();
+    }
+  }
+
+
+  /**
+   * Called while the NN is in Standby state, but just about to be
+   * asked to enter Active state. This cancels any checkpoints
+   * currently being taken.
+   */
+  void prepareToStopStandbyServices() throws ServiceFailedException {
+    if (standbyCheckpointer != null) {
+      standbyCheckpointer.cancelAndPreventCheckpoints();
+    }
+  }
+
+  /** Stop services required in standby state */
+  void stopStandbyServices() throws IOException {
+    LOG.info("Stopping services started for standby state");
+    if (standbyCheckpointer != null) {
+      standbyCheckpointer.stop();
+    }
+    if (editLogTailer != null) {
+      editLogTailer.stop();
+    }
+    if (dir != null && dir.fsImage != null && dir.fsImage.editLog != null) {
+      dir.fsImage.editLog.close();
+    }
+  }
+  
+  
+  void checkOperation(OperationCategory op) throws StandbyException {
+    if (haContext != null) {
+      // null in some unit tests
+      haContext.checkOperation(op);
+    }
+  }
+  
+  public static Collection<URI> getNamespaceDirs(Configuration conf) {
+    return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
+  }
+
+  /**
+   * Get all edits dirs which are required. If any shared edits dirs are
+   * configured, these are also included in the set of required dirs.
+   * 
+   * @param conf the HDFS configuration.
+   * @return all required dirs.
+   */
   public static Collection<URI> getRequiredNamespaceEditsDirs(Configuration conf) {
-    return getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY);
+    Set<URI> ret = new HashSet<URI>();
+    ret.addAll(getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY));
+    ret.addAll(getSharedEditsDirs(conf));
+    return ret;
   }
 
   private static Collection<URI> getStorageDirs(Configuration conf,
@@ -481,6 +701,75 @@
     return Util.stringCollectionAsURIs(dirNames);
   }
 
+  /**
+   * Return an ordered list of edits directories to write to.
+   * The list is ordered such that all shared edits directories
+   * are ordered before non-shared directories, and any duplicates
+   * are removed. The order they are specified in the configuration
+   * is retained.
+   * @return Collection of shared edits directories.
+   * @throws IOException if multiple shared edits directories are configured
+   */
+  public static List<URI> getNamespaceEditsDirs(Configuration conf)
+      throws IOException {
+    // Use a LinkedHashSet so that order is maintained while we de-dup
+    // the entries.
+    LinkedHashSet<URI> editsDirs = new LinkedHashSet<URI>();
+    
+    List<URI> sharedDirs = getSharedEditsDirs(conf);
+
+    // Fail until multiple shared edits directories are supported (HDFS-2782)
+    if (sharedDirs.size() > 1) {
+      throw new IOException(
+          "Multiple shared edits directories are not yet supported");
+    }
+
+    // First add the shared edits dirs. It's critical that the shared dirs
+    // are added first, since JournalSet syncs them in the order they are listed,
+    // and we need to make sure all edits are in place in the shared storage
+    // before they are replicated locally. See HDFS-2874.
+    for (URI dir : sharedDirs) {
+      if (!editsDirs.add(dir)) {
+        LOG.warn("Edits URI " + dir + " listed multiple times in " + 
+            DFS_NAMENODE_SHARED_EDITS_DIR_KEY + ". Ignoring duplicates.");
+      }
+    }
+    
+    // Now add the non-shared dirs.
+    for (URI dir : getStorageDirs(conf, DFS_NAMENODE_EDITS_DIR_KEY)) {
+      if (!editsDirs.add(dir)) {
+        LOG.warn("Edits URI " + dir + " listed multiple times in " + 
+            DFS_NAMENODE_SHARED_EDITS_DIR_KEY + " and " +
+            DFS_NAMENODE_EDITS_DIR_KEY + ". Ignoring duplicates.");
+      }
+    }
+
+    if (editsDirs.isEmpty()) {
+      // If this is the case, no edit dirs have been explicitly configured.
+      // Image dirs are to be used for edits too.
+      return Lists.newArrayList(getNamespaceDirs(conf));
+    } else {
+      return Lists.newArrayList(editsDirs);
+    }
+  }
+  
+  /**
+   * Returns edit directories that are shared between primary and secondary.
+   * @param conf
+   * @return Collection of edit directories.
+   */
+  public static List<URI> getSharedEditsDirs(Configuration conf) {
+    // don't use getStorageDirs here, because we want an empty default
+    // rather than the dir in /tmp
+    Collection<String> dirNames = conf.getTrimmedStringCollection(
+        DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
+    return Util.stringCollectionAsURIs(dirNames);
+  }
+  
+  public Configuration getConf() {
+    return conf;
+  }
+
   @Override
   public void readLock() {
     this.fsLock.readLock().lock();
@@ -494,6 +783,10 @@
     this.fsLock.writeLock().lock();
   }
   @Override
+  public void writeLockInterruptibly() throws InterruptedException {
+    this.fsLock.writeLock().lockInterruptibly();
+  }
+  @Override
   public void writeUnlock() {
     this.fsLock.writeLock().unlock();
   }
@@ -526,6 +819,26 @@
                                                DFS_PERMISSIONS_ENABLED_DEFAULT);
     LOG.info("supergroup=" + supergroup);
     LOG.info("isPermissionEnabled=" + isPermissionEnabled);
+
+    this.persistBlocks = conf.getBoolean(DFS_PERSIST_BLOCKS_KEY,
+                                         DFS_PERSIST_BLOCKS_DEFAULT);
+    // block allocation has to be persisted in HA using a shared edits directory
+    // so that the standby has up-to-date namespace information
+    String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
+    this.haEnabled = HAUtil.isHAEnabled(conf, nameserviceId);  
+    this.persistBlocks |= haEnabled && HAUtil.usesSharedEditsDir(conf);
+    
+    // Sanity check the HA-related config.
+    if (nameserviceId != null) {
+      LOG.info("Determined nameservice ID: " + nameserviceId);
+    }
+    LOG.info("HA Enabled: " + haEnabled);
+    if (!haEnabled && HAUtil.usesSharedEditsDir(conf)) {
+      LOG.warn("Configured NNs:\n" + DFSUtil.nnAddressesAsString(conf));
+      throw new IOException("Invalid configuration: a shared edits dir " +
+          "must not be specified if HA is not enabled.");
+    }
+
     short filePermission = (short)conf.getInt(DFS_NAMENODE_UPGRADE_PERMISSION_KEY,
                                               DFS_NAMENODE_UPGRADE_PERMISSION_DEFAULT);
     this.defaultPermission = PermissionStatus.createImmutable(
@@ -546,6 +859,16 @@
         DFS_SUPPORT_APPEND_DEFAULT);
 
     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
+    
+    this.standbyShouldCheckpoint = conf.getBoolean(
+        DFS_HA_STANDBY_CHECKPOINTS_KEY,
+        DFS_HA_STANDBY_CHECKPOINTS_DEFAULT);
+    
+    // For testing purposes, allow the DT secret manager to be started regardless
+    // of whether security is enabled.
+    alwaysUseDelegationTokensForTests = 
+      conf.getBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
+          DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT);
   }
 
   /**
@@ -566,7 +889,7 @@
   }
 
   /**
-   * Version of {@see #getNamespaceInfo()} that is not protected by a lock.
+   * Version of @see #getNamespaceInfo() that is not protected by a lock.
    */
   NamespaceInfo unprotectedGetNamespaceInfo() {
     return new NamespaceInfo(dir.fsImage.getStorage().getNamespaceID(),
@@ -583,23 +906,16 @@
   void close() {
     fsRunning = false;
     try {
-      if (blockManager != null) blockManager.close();
+      stopCommonServices();
       if (smmthread != null) smmthread.interrupt();
-      if (dtSecretManager != null) dtSecretManager.stopThreads();
-      if (nnrmthread != null) nnrmthread.interrupt();
-    } catch (Exception e) {
-      LOG.warn("Exception shutting down FSNamesystem", e);
     } finally {
       // using finally to ensure we also wait for lease daemon
       try {
-        if (lmthread != null) {
-          lmthread.interrupt();
-          lmthread.join(3000);
-        }
+        stopActiveServices();
+        stopStandbyServices();
         if (dir != null) {
           dir.close();
         }
-      } catch (InterruptedException ie) {
       } catch (IOException ie) {
         LOG.error("Error closing FSDirectory", ie);
         IOUtils.cleanup(LOG, dir);
@@ -611,6 +927,18 @@
   public boolean isRunning() {
     return fsRunning;
   }
+  
+  @Override
+  public boolean isInStandbyState() {
+    if (haContext == null || haContext.getState() == null) {
+      // We're still starting up. In this case, if HA is
+      // on for the cluster, we always start in standby. Otherwise
+      // start in active.
+      return haEnabled;
+    }
+  
+    return haContext.getState() instanceof StandbyState;
+  }
 
   /**
    * Dump all metadata into specified file
@@ -622,14 +950,7 @@
       File file = new File(System.getProperty("hadoop.log.dir"), filename);
       PrintWriter out = new PrintWriter(new BufferedWriter(new FileWriter(file,
           true)));
-  
-      long totalInodes = this.dir.totalInodes();
-      long totalBlocks = this.getBlocksTotal();
-      out.println(totalInodes + " files and directories, " + totalBlocks
-          + " blocks = " + (totalInodes + totalBlocks) + " total");
-
-      blockManager.metaSave(out);
-
+      metaSave(out);
       out.flush();
       out.close();
     } finally {
@@ -637,11 +958,31 @@
     }
   }
 
+  private void metaSave(PrintWriter out) {
+    assert hasWriteLock();
+    long totalInodes = this.dir.totalInodes();
+    long totalBlocks = this.getBlocksTotal();
+    out.println(totalInodes + " files and directories, " + totalBlocks
+        + " blocks = " + (totalInodes + totalBlocks) + " total");
+
+    blockManager.metaSave(out);
+  }
+
+  private String metaSaveAsString() {
+    StringWriter sw = new StringWriter();
+    PrintWriter pw = new PrintWriter(sw);
+    metaSave(pw);
+    pw.flush();
+    return sw.toString();
+  }
+  
+
   long getDefaultBlockSize() {
     return serverDefaults.getBlockSize();
   }
 
-  FsServerDefaults getServerDefaults() {
+  FsServerDefaults getServerDefaults() throws StandbyException {
+    checkOperation(OperationCategory.READ);
     return serverDefaults;
   }
 
@@ -668,6 +1009,8 @@
     HdfsFileStatus resultingStat = null;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set permission for " + src, safeMode);
       }
@@ -697,6 +1040,8 @@
     HdfsFileStatus resultingStat = null;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set owner for " + src, safeMode);
       }
@@ -787,13 +1132,14 @@
       }  else { // second attempt is with  write lock
         writeLock(); // writelock is needed to set accesstime
       }
-
-      // if the namenode is in safemode, then do not update access time
-      if (isInSafeMode()) {
-        doAccessTime = false;
-      }
-
       try {
+        checkOperation(OperationCategory.READ);
+
+        // if the namenode is in safemode, then do not update access time
+        if (isInSafeMode()) {
+          doAccessTime = false;
+        }
+
         long now = now();
         INodeFile inode = dir.getFileINode(src);
         if (inode == null) {
@@ -861,6 +1207,7 @@
     HdfsFileStatus resultingStat = null;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot concat " + target, safeMode);
       }
@@ -992,6 +1339,8 @@
     }
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       // Write access is required to set access and modification times
       if (isPermissionEnabled) {
         checkPathAccess(src, FsAction.WRITE);
@@ -1022,6 +1371,8 @@
     HdfsFileStatus resultingStat = null;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (!createParent) {
         verifyParentDir(link);
       }
@@ -1091,6 +1442,8 @@
     final boolean isFile;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set replication for " + src, safeMode);
       }
@@ -1121,6 +1474,7 @@
       throws IOException, UnresolvedLinkException {
     readLock();
     try {
+      checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
         checkTraverse(filename);
       }
@@ -1163,6 +1517,8 @@
       FileNotFoundException, ParentNotDirectoryException, IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       startFileInternal(src, permissions, holder, clientMachine, flag,
           createParent, replication, blockSize);
     } finally {
@@ -1266,30 +1622,8 @@
           blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
 
       if (append && myFile != null) {
-        //
-        // Replace current node with a INodeUnderConstruction.
-        // Recreate in-memory lease record.
-        //
-        INodeFile node = (INodeFile) myFile;
-        INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
-                                        node.getLocalNameBytes(),
-                                        node.getReplication(),
-                                        node.getModificationTime(),
-                                        node.getPreferredBlockSize(),
-                                        node.getBlocks(),
-                                        node.getPermissionStatus(),
-                                        holder,
-                                        clientMachine,
-                                        clientNode);
-        dir.replaceNode(src, node, cons);
-        leaseManager.addLease(cons.getClientName(), src);
-        
-        // convert last block to under-construction
-        LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
-
-        // add append file record to log, record lease, etc.
-        getEditLog().logOpenFile(src, cons);
-        return ret;
+        return prepareFileForWrite(
+            src, myFile, holder, clientMachine, clientNode, true);
       } else {
        // Now we can add the name to the filesystem. This file has no
        // blocks associated with it.
@@ -1320,6 +1654,45 @@
     }
     return null;
   }
+  
+  /**
+   * Replace current node with a INodeUnderConstruction.
+   * Recreate in-memory lease record.
+   * 
+   * @param src path to the file
+   * @param file existing file object
+   * @param leaseHolder identifier of the lease holder on this file
+   * @param clientMachine identifier of the client machine
+   * @param clientNode if the client is collocated with a DN, that DN's descriptor
+   * @param writeToEditLog whether to persist this change to the edit log
+   * @return the last block locations if the block is partial or null otherwise
+   * @throws UnresolvedLinkException
+   * @throws IOException
+   */
+  public LocatedBlock prepareFileForWrite(String src, INode file,
+      String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
+      boolean writeToEditLog)
+      throws UnresolvedLinkException, IOException {
+    INodeFile node = (INodeFile) file;
+    INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
+                                    node.getLocalNameBytes(),
+                                    node.getReplication(),
+                                    node.getModificationTime(),
+                                    node.getPreferredBlockSize(),
+                                    node.getBlocks(),
+                                    node.getPermissionStatus(),
+                                    leaseHolder,
+                                    clientMachine,
+                                    clientNode);
+    dir.replaceNode(src, node, cons);
+    leaseManager.addLease(cons.getClientName(), src);
+    
+    LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
+    if (writeToEditLog) {
+      getEditLog().logOpenFile(src, cons);
+    }
+    return ret;
+  }
 
   /**
    * Recover lease;
@@ -1336,6 +1709,8 @@
       throws IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException(
             "Cannot recover the lease of " + src, safeMode);
@@ -1455,6 +1830,8 @@
     LocatedBlock lb = null;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       lb = startFileInternal(src, null, holder, clientMachine, 
                         EnumSet.of(CreateFlag.APPEND), 
                         false, blockManager.maxReplication, 0);
@@ -1519,6 +1896,8 @@
 
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot add block to " + src, safeMode);
       }
@@ -1552,6 +1931,7 @@
     // Allocate a new block and record it in the INode. 
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot add block to " + src, safeMode);
       }
@@ -1570,10 +1950,14 @@
       
       for (DatanodeDescriptor dn : targets) {
         dn.incBlocksScheduled();
-      }      
+      }
+      dir.persistBlocks(src, pendingFile);
     } finally {
       writeUnlock();
     }
+    if (persistBlocks) {
+      getEditLog().logSync();
+    }
 
     // Create next block
     LocatedBlock b = new LocatedBlock(getExtendedBlock(newBlock), targets, fileLength);
@@ -1594,6 +1978,7 @@
     final List<DatanodeDescriptor> chosen;
     readLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       //check safe mode
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot add datanode; src=" + src
@@ -1635,6 +2020,7 @@
       UnresolvedLinkException, IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       //
       // Remove the block from the pending creates list
       //
@@ -1652,10 +2038,15 @@
         NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
                                       + b + " is removed from pendingCreates");
       }
-      return true;
+      dir.persistBlocks(src, file);
     } finally {
       writeUnlock();
     }
+    if (persistBlocks) {
+      getEditLog().logSync();
+    }
+
+    return true;
   }
   
   // make sure that we still have the lease on this file.
@@ -1705,6 +2096,8 @@
     boolean success = false;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       success = completeFileInternal(src, holder, 
         ExtendedBlock.getLocalBlock(last));
     } finally {
@@ -1764,12 +2157,15 @@
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
   private Block allocateBlock(String src, INode[] inodes,
-      DatanodeDescriptor targets[]) throws QuotaExceededException {
+      DatanodeDescriptor targets[]) throws QuotaExceededException,
+      SafeModeException {
     assert hasWriteLock();
     Block b = new Block(DFSUtil.getRandom().nextLong(), 0, 0); 
     while(isValidBlock(b)) {
       b.setBlockId(DFSUtil.getRandom().nextLong());
     }
+    // Increment the generation stamp for every new block.
+    nextGenerationStamp();
     b.setGenerationStamp(getGenerationStamp());
     b = dir.addBlock(src, inodes, b, targets);
     NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
@@ -1841,6 +2237,8 @@
     }
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       status = renameToInternal(src, dst);
       if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(dst, false);
@@ -1896,6 +2294,8 @@
     }
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       renameToInternal(src, dst, options);
       if (auditLog.isInfoEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(dst, false); 
@@ -1973,6 +2373,7 @@
 
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot delete " + src, safeMode);
       }
@@ -2028,9 +2429,45 @@
     if (blocks == null) {
       return;
     }
-    for(Block b : blocks) {
+    
+    // In the case that we are a Standby tailing edits from the
+    // active while in safe-mode, we need to track the total number
+    // of blocks and safe blocks in the system.
+    boolean trackBlockCounts = isSafeModeTrackingBlocks();
+    int numRemovedComplete = 0, numRemovedSafe = 0;
+
+    for (Block b : blocks) {
+      if (trackBlockCounts) {
+        BlockInfo bi = blockManager.getStoredBlock(b);
+        if (bi.isComplete()) {
+          numRemovedComplete++;
+          if (bi.numNodes() >= blockManager.minReplication) {
+            numRemovedSafe++;
+          }
+        }
+      }
       blockManager.removeBlock(b);
     }
+    if (trackBlockCounts) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Adjusting safe-mode totals for deletion of " + src + ":" +
+            "decreasing safeBlocks by " + numRemovedSafe +
+            ", totalBlocks by " + numRemovedComplete);
+      }
+      adjustSafeModeBlockTotals(-numRemovedSafe, -numRemovedComplete);
+    }
+  }
+
+  /**
+   * @see SafeModeInfo#shouldIncrementallyTrackBlocks
+   */
+  private boolean isSafeModeTrackingBlocks() {
+    if (!haEnabled) {
+      // Never track blocks incrementally in non-HA code.
+      return false;
+    }
+    SafeModeInfo sm = this.safeMode;
+    return sm != null && sm.shouldIncrementallyTrackBlocks();
   }
 
   /**
@@ -2045,11 +2482,15 @@
    *
    * @return object containing information regarding the file
    *         or null if file not found
+   * @throws StandbyException 
    */
   HdfsFileStatus getFileInfo(String src, boolean resolveLink) 
-    throws AccessControlException, UnresolvedLinkException {
+    throws AccessControlException, UnresolvedLinkException,
+           StandbyException {
     readLock();
     try {
+      checkOperation(OperationCategory.READ);
+
       if (!DFSUtil.isValidName(src)) {
         throw new InvalidPathException("Invalid file name: " + src);
       }
@@ -2073,6 +2514,8 @@
     }
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       status = mkdirsInternal(src, permissions, createParent);
     } finally {
       writeUnlock();
@@ -2127,9 +2570,11 @@
   }
 
   ContentSummary getContentSummary(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException {
+      FileNotFoundException, UnresolvedLinkException, StandbyException {
     readLock();
     try {
+      checkOperation(OperationCategory.READ);
+
       if (isPermissionEnabled) {
         checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
       }
@@ -2148,6 +2593,7 @@
       throws IOException, UnresolvedLinkException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set quota on " + path, safeMode);
       }
@@ -2172,6 +2618,7 @@
                                   + src + " for " + clientName);
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot fsync file " + src, safeMode);
       }
@@ -2381,6 +2828,10 @@
     String src = "";
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+      // If a DN tries to commit to the standby, the recovery will
+      // fail, and the next retry will succeed on the new NN.
+  
       if (isInSafeMode()) {
         throw new SafeModeException(
           "Cannot commitBlockSynchronization while in safe mode",
@@ -2455,8 +2906,8 @@
         //remove lease, close file
         finalizeINodeFileUnderConstruction(src, pendingFile);
       } else if (supportAppends) {
-        // If this commit does not want to close the file, persist
-        // blocks only if append is supported 
+        // If this commit does not want to close the file, persist blocks
+        // only if append is supported or we're explicitly told to
         dir.persistBlocks(src, pendingFile);
       }
     } finally {
@@ -2481,6 +2932,8 @@
   void renewLease(String holder) throws IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
       }
@@ -2508,6 +2961,8 @@
     DirectoryListing dl;
     readLock();
     try {
+      checkOperation(OperationCategory.READ);
+
       if (isPermissionEnabled) {
         if (dir.isDir(src)) {
           checkPathAccess(src, FsAction.READ_EXECUTE);
@@ -2586,7 +3041,7 @@
    * @return an array of datanode commands 
    * @throws IOException
    */
-  DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
+  HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
       long capacity, long dfsUsed, long remaining, long blockPoolUsed,
       int xceiverCount, int xmitsInProgress, int failedVolumes) 
         throws IOException {
@@ -2597,28 +3052,40 @@
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
           nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
           xceiverCount, maxTransfer, failedVolumes);
-      if (cmds != null) {
-        return cmds;
+      if (cmds == null || cmds.length == 0) {
+        DatanodeCommand cmd = upgradeManager.getBroadcastCommand();
+        if (cmd != null) {
+          cmds = new DatanodeCommand[] {cmd};
+        }
       }
-
-      //check distributed upgrade
-      DatanodeCommand cmd = upgradeManager.getBroadcastCommand();
-      if (cmd != null) {
-        return new DatanodeCommand[] {cmd};
-      }
-      return null;
+      
+      return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
     } finally {
       readUnlock();
     }
   }
 
+  private NNHAStatusHeartbeat createHaStatusHeartbeat() {
+    HAState state = haContext.getState();
+    NNHAStatusHeartbeat.State hbState;
+    if (state instanceof ActiveState) {
+      hbState = NNHAStatusHeartbeat.State.ACTIVE;
+    } else if (state instanceof StandbyState) {
+      hbState = NNHAStatusHeartbeat.State.STANDBY;      
+    } else {
+      throw new AssertionError("Invalid state: " + state.getClass());
+    }
+    return new NNHAStatusHeartbeat(hbState,
+        getFSImage().getLastAppliedOrWrittenTxId());
+  }
+
   /**
    * Returns whether or not there were available resources at the last check of
    * resources.
    *
    * @return true if there were sufficient resources available, false otherwise.
    */
-  private boolean nameNodeHasResourcesAvailable() {
+  boolean nameNodeHasResourcesAvailable() {
     return hasResourcesAvailable;
   }
 
@@ -2626,7 +3093,7 @@
    * Perform resource checks and cache the results.
    * @throws IOException
    */
-  private void checkAvailableResources() throws IOException {
+  void checkAvailableResources() {
     Preconditions.checkState(nnResourceChecker != null,
         "nnResourceChecker not initialized");
     hasResourcesAvailable = nnResourceChecker.hasAvailableDiskSpace();
@@ -2665,11 +3132,11 @@
     }
   }
   
-  FSImage getFSImage() {
+  public FSImage getFSImage() {
     return dir.fsImage;
   }
 
-  FSEditLog getEditLog() {
+  public FSEditLog getEditLog() {
     return getFSImage().getEditLog();
   }    
 
@@ -2701,8 +3168,12 @@
   @Metric({"TransactionsSinceLastLogRoll",
       "Number of transactions since last edit log roll"})
   public long getTransactionsSinceLastLogRoll() {
-    return (getEditLog().getLastWrittenTxId() -
-        getEditLog().getCurSegmentTxId()) + 1;
+    if (isInStandbyState()) {
+      return 0;
+    } else {
+      return getEditLog().getLastWrittenTxId() -
+        getEditLog().getCurSegmentTxId() + 1;
+    }
   }
   
   @Metric({"LastWrittenTransactionId", "Transaction ID written to the edit log"})
@@ -2931,6 +3402,8 @@
     boolean initializedReplQueues = false;
     /** Was safemode entered automatically because available resources were low. */
     private boolean resourcesLow = false;
+    /** Should safemode adjust its block totals as blocks come in */
+    private boolean shouldIncrementallyTrackBlocks = false;
     
     /**
      * Creates SafeModeInfo when the name node enters
@@ -2959,6 +3432,18 @@
     }
 
     /**
+     * In the HA case, the StandbyNode can be in safemode while the namespace
+     * is modified by the edit log tailer. In this case, the number of total
+     * blocks changes as edits are processed (eg blocks are added and deleted).
+     * However, we don't want to do the incremental tracking during the
+     * startup-time loading process -- only once the initial total has been
+     * set after the image has been loaded.
+     */
+    private boolean shouldIncrementallyTrackBlocks() {
+      return shouldIncrementallyTrackBlocks;
+    }
+
+    /**
      * Creates SafeModeInfo when safe mode is entered manually, or because
      * available resources are low.
      *
@@ -2986,13 +3471,7 @@
      * @return true if in safe mode
      */
     private synchronized boolean isOn() {
-      try {
-        assert isConsistent() : " SafeMode: Inconsistent filesystem state: "
-          + "Total num of blocks, active blocks, or "
-          + "total safe blocks don't match.";
-      } catch(IOException e) {
-        System.err.print(StringUtils.stringifyException(e));
-      }
+      doConsistencyCheck();
       return this.reached >= 0;
     }
       
@@ -3031,8 +3510,9 @@
           return;
         }
       }
-      // if not done yet, initialize replication queues
-      if (!isPopulatingReplQueues()) {
+      // if not done yet, initialize replication queues.
+      // In the standby, do not populate repl queues
+      if (!isPopulatingReplQueues() && !isInStandbyState()) {
         initializeReplQueues();
       }
       long timeInSafemode = now() - systemStart;
@@ -3051,6 +3531,8 @@
           + nt.getNumOfLeaves() + " datanodes");
       NameNode.stateChangeLog.info("STATE* UnderReplicatedBlocks has "
           + blockManager.numOfUnderReplicatedBlocks() + " blocks");
+
+      startSecretManagerIfNecessary();
     }
 
     /**
@@ -3073,7 +3555,7 @@
      * initializing replication queues.
      */
     private synchronized boolean canInitializeReplQueues() {
-      return blockSafe >= blockReplQueueThreshold;
+      return !isInStandbyState() && blockSafe >= blockReplQueueThreshold;
     }
       
     /** 
@@ -3106,6 +3588,9 @@
      * Check and trigger safe mode if needed. 
      */
     private void checkMode() {
+      // Have to have write-lock since leaving safemode initializes
+      // repl queues, which requires write lock
+      assert hasWriteLock();
       if (needEnter()) {
         enter();
         // check if we are ready to initialize replication queues
@@ -3145,6 +3630,13 @@
       this.blockThreshold = (int) (blockTotal * threshold);
       this.blockReplQueueThreshold = 
         (int) (blockTotal * replQueueThreshold);
+      if (haEnabled) {
+        // After we initialize the block count, any further namespace
+        // modifications done while in safe mode need to keep track
+        // of the number of total blocks in the system.
+        this.shouldIncrementallyTrackBlocks = true;
+      }
+      
       checkMode();
     }
       
@@ -3154,9 +3646,10 @@
      * @param replication current replication 
      */
     private synchronized void incrementSafeBlockCount(short replication) {
-      if (replication == safeReplication)
+      if (replication == safeReplication) {
         this.blockSafe++;
-      checkMode();
+        checkMode();
+      }
     }
       
     /**
@@ -3165,9 +3658,11 @@
      * @param replication current replication 
      */
     private synchronized void decrementSafeBlockCount(short replication) {
-      if (replication == safeReplication-1)
+      if (replication == safeReplication-1) {
         this.blockSafe--;
-      checkMode();
+        assert blockSafe >= 0 || isManual();
+        checkMode();
+      }
     }
 
     /**
@@ -3285,16 +3780,45 @@
       
     /**
      * Checks consistency of the class state.
-     * This is costly and currently called only in assert.
-     * @throws IOException 
+     * This is costly so only runs if asserts are enabled.
      */
-    private boolean isConsistent() throws IOException {
+    private void doConsistencyCheck() {
+      boolean assertsOn = false;
+      assert assertsOn = true; // set to true if asserts are on
+      if (!assertsOn) return;
+      
       if (blockTotal == -1 && blockSafe == -1) {
-        return true; // manual safe mode
+        return; // manual safe mode
       }
       int activeBlocks = blockManager.getActiveBlockCount();
-      return (blockTotal == activeBlocks) ||
-        (blockSafe >= 0 && blockSafe <= blockTotal);
+      if ((blockTotal != activeBlocks) &&
+          !(blockSafe >= 0 && blockSafe <= blockTotal)) {
+        throw new AssertionError(
+            " SafeMode: Inconsistent filesystem state: "
+        + "SafeMode data: blockTotal=" + blockTotal
+        + " blockSafe=" + blockSafe + "; "
+        + "BlockManager data: active="  + activeBlocks);
+      }
+    }
+
+    private synchronized void adjustBlockTotals(int deltaSafe, int deltaTotal) {
+      if (!shouldIncrementallyTrackBlocks) {
+        return;
+      }
+      assert haEnabled;
+      
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Adjusting block totals from " +
+            blockSafe + "/" + blockTotal + " to " +
+            (blockSafe + deltaSafe) + "/" + (blockTotal + deltaTotal));
+      }
+      assert blockSafe + deltaSafe >= 0 : "Can't reduce blockSafe " +
+        blockSafe + " by " + deltaSafe + ": would be negative";
+      assert blockTotal + deltaTotal >= 0 : "Can't reduce blockTotal " +
+        blockTotal + " by " + deltaTotal + ": would be negative";
+      
+      blockSafe += deltaSafe;
+      setBlockTotal(blockTotal + deltaTotal);
     }
   }
     
@@ -3376,6 +3900,9 @@
 
   @Override
   public boolean isPopulatingReplQueues() {
+    if (isInStandbyState()) {
+      return false;
+    }
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -3398,13 +3925,30 @@
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
       return;
-    safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
+    BlockInfo storedBlock = blockManager.getStoredBlock(b);
+    if (storedBlock.isComplete()) {
+      safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
+    }
+  }
+  
+  /**
+   * Adjust the total number of blocks safe and expected during safe mode.
+   * If safe mode is not currently on, this is a no-op.
+   * @param deltaSafe the change in number of safe blocks
+   * @param deltaTotal the change i nnumber of total blocks expected
+   */
+  public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal) {
+    // safeMode is volatile, and may be set to null at any time
+    SafeModeInfo safeMode = this.safeMode;
+    if (safeMode == null)
+      return;
+    safeMode.adjustBlockTotals(deltaSafe, deltaTotal);
   }
 
   /**
    * Set the total number of blocks in the system. 
    */
-  void setBlockTotal() {
+  public void setBlockTotal() {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -3440,7 +3984,8 @@
           }
           assert node != null : "Found a lease for nonexisting file.";
           assert node.isUnderConstruction() :
-            "Found a lease for file that is not under construction.";
+            "Found a lease for file " + path + " that is not under construction." +
+            " lease=" + lease;
           INodeFileUnderConstruction cons = (INodeFileUnderConstruction) node;
           BlockInfo[] blocks = cons.getBlocks();
           if(blocks == null)
@@ -3465,21 +4010,32 @@
   void enterSafeMode(boolean resourcesLow) throws IOException {
     writeLock();
     try {
-    // Ensure that any concurrent operations have been fully synced
-    // before entering safe mode. This ensures that the FSImage
-    // is entirely stable on disk as soon as we're in safe mode.
-    getEditLog().logSyncAll();
-    if (!isInSafeMode()) {
-      safeMode = new SafeModeInfo(resourcesLow);
-      return;
-    }
-    if (resourcesLow) {
-      safeMode.setResourcesLow();
-    }
-    safeMode.setManual();
-    getEditLog().logSyncAll();
-    NameNode.stateChangeLog.info("STATE* Safe mode is ON. " 
-                                + safeMode.getTurnOffTip());
+      // Stop the secret manager, since rolling the master key would
+      // try to write to the edit log
+      stopSecretManager();
+
+      // Ensure that any concurrent operations have been fully synced
+      // before entering safe mode. This ensures that the FSImage
+      // is entirely stable on disk as soon as we're in safe mode.
+      boolean isEditlogOpenForWrite = getEditLog().isOpenForWrite();
+      // Before Editlog is in OpenForWrite mode, editLogStream will be null. So,
+      // logSyncAll call can be called only when Edlitlog is in OpenForWrite mode
+      if (isEditlogOpenForWrite) {
+        getEditLog().logSyncAll();
+      }
+      if (!isInSafeMode()) {
+        safeMode = new SafeModeInfo(resourcesLow);
+        return;
+      }
+      if (resourcesLow) {
+        safeMode.setResourcesLow();
+      }
+      safeMode.setManual();
+      if (isEditlogOpenForWrite) {
+        getEditLog().logSyncAll();
+      }
+      NameNode.stateChangeLog.info("STATE* Safe mode is ON. "
+          + safeMode.getTurnOffTip());
     } finally {
       writeUnlock();
     }
@@ -3520,6 +4076,7 @@
   CheckpointSignature rollEditLog() throws IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.JOURNAL);
       if (isInSafeMode()) {
         throw new SafeModeException("Log not rolled", safeMode);
       }
@@ -3536,6 +4093,8 @@
   throws IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.CHECKPOINT);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Checkpoint not started", safeMode);
       }
@@ -3552,6 +4111,8 @@
                             CheckpointSignature sig) throws IOException {
     readLock();
     try {
+      checkOperation(OperationCategory.CHECKPOINT);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Checkpoint not ended", safeMode);
       }
@@ -3704,6 +4265,34 @@
     return blockManager.getExcessBlocksCount();
   }
   
+  // HA-only metric
+  @Metric
+  public long getPostponedMisreplicatedBlocks() {
+    return blockManager.getPostponedMisreplicatedBlocksCount();
+  }
+
+  // HA-only metric
+  @Metric
+  public int getPendingDataNodeMessageCount() {
+    return blockManager.getPendingDataNodeMessageCount();
+  }
+  
+  // HA-only metric
+  @Metric
+  public String getHAState() {
+    return haContext.getState().toString();
+  }
+
+  // HA-only metric
+  @Metric
+  public long getMillisSinceLastLoadedEdits() {
+    if (isInStandbyState() && editLogTailer != null) {
+      return now() - editLogTailer.getLastLoadTimestamp();
+    } else {
+      return 0;
+    }
+  }
+  
   @Metric
   public int getBlockCapacity() {
     return blockManager.getCapacity();
@@ -3715,6 +4304,7 @@
   }
   
   private ObjectName mbeanName;
+
   /**
    * Register the FSNamesystem MBean using the name
    *        "hadoop:service=NameNode,name=FSNamesystemState"
@@ -3813,6 +4403,29 @@
   }
   
   /**
+   * Client is reporting some bad block locations.
+   */
+  void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    writeLock();
+    try {
+      checkOperation(OperationCategory.WRITE);
+      
+      NameNode.stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
+      for (int i = 0; i < blocks.length; i++) {
+        ExtendedBlock blk = blocks[i].getBlock();
+        DatanodeInfo[] nodes = blocks[i].getLocations();
+        for (int j = 0; j < nodes.length; j++) {
+          DatanodeInfo dn = nodes[j];
+          blockManager.findAndMarkBlockAsCorrupt(blk, dn,
+              "client machine reported it");
+        }
+      }
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  /**
    * Get a new generation stamp together with an access token for 
    * a block under construction
    * 
@@ -3829,6 +4442,8 @@
     LocatedBlock locatedBlock;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       // check vadility of parameters
       checkUCBlock(block, clientName);
   
@@ -3858,6 +4473,8 @@
       throws IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Pipeline not updated", safeMode);
       }
@@ -3873,7 +4490,7 @@
     } finally {
       writeUnlock();
     }
-    if (supportAppends) {
+    if (supportAppends || persistBlocks) {
       getEditLog().logSync();
     }
     LOG.info("updatePipeline(" + oldBlock + ") successfully to " + newBlock);
@@ -4067,6 +4684,8 @@
 
     readLock();
     try {
+      checkOperation(OperationCategory.READ);
+
       if (!isPopulatingReplQueues()) {
         throw new IOException("Cannot run listCorruptFileBlocks because " +
                               "replication queues have not been initialized.");
@@ -4159,6 +4778,8 @@
     Token<DelegationTokenIdentifier> token;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot issue delegation token", safeMode);
       }
@@ -4203,6 +4824,8 @@
     long expiryTime;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot renew delegation token", safeMode);
       }
@@ -4233,6 +4856,8 @@
       throws IOException {
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot cancel delegation token", safeMode);
       }
@@ -4266,16 +4891,14 @@
    * @param key new delegation key.
    */
   public void logUpdateMasterKey(DelegationKey key) throws IOException {
-    writeLock();
-    try {
-      if (isInSafeMode()) {
-        throw new SafeModeException(
-          "Cannot log master key update in safe mode", safeMode);
-      }
-      getEditLog().logUpdateMasterKey(key);
-    } finally {
-      writeUnlock();
-    }
+    
+    assert !isInSafeMode() :
+      "this should never be called while in safemode, since we stop " +
+      "the DT manager before entering safemode!";
+    // No need to hold FSN lock since we don't access any internal
+    // structures, and this is stopped before the FSN shuts itself
+    // down, etc.
+    getEditLog().logUpdateMasterKey(key);
     getEditLog().logSync();
   }
   
@@ -4545,9 +5168,32 @@
       byte[] password) throws InvalidToken {
     getDelegationTokenSecretManager().verifyToken(identifier, password);
   }
+  
+  public boolean isGenStampInFuture(long genStamp) {
+    return (genStamp > getGenerationStamp());
+  }
+  @VisibleForTesting
+  public EditLogTailer getEditLogTailer() {
+    return editLogTailer;
+  }
+  
+  @VisibleForTesting
+  void setFsLockForTests(ReentrantReadWriteLock lock) {
+    this.fsLock = lock;
+  }
+  
+  @VisibleForTesting
+  ReentrantReadWriteLock getFsLockForTests() {
+    return fsLock;
+  }
 
   @VisibleForTesting
   public SafeModeInfo getSafeModeInfoForTests() {
     return safeMode;
   }
+  
+  @VisibleForTesting
+  public void setNNResourceChecker(NameNodeResourceChecker nnResourceChecker) {
+    this.nnResourceChecker = nnResourceChecker;
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index eeb40c2..603dd00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -52,6 +52,7 @@
   private static final Log LOG = LogFactory.getLog(FileJournalManager.class);
 
   private final StorageDirectory sd;
+  private final NNStorage storage;
   private int outputBufferCapacity = 512*1024;
 
   private static final Pattern EDITS_REGEX = Pattern.compile(
@@ -60,14 +61,14 @@
     NameNodeFile.EDITS_INPROGRESS.getName() + "_(\\d+)");
 
   private File currentInProgress = null;
-  private long maxSeenTransaction = 0L;
 
   @VisibleForTesting
   StoragePurger purger
     = new NNStorageRetentionManager.DeletionStoragePurger();
 
-  public FileJournalManager(StorageDirectory sd) {
+  public FileJournalManager(StorageDirectory sd, NNStorage storage) {
     this.sd = sd;
+    this.storage = storage;
   }
 
   @Override 
@@ -76,11 +77,16 @@
   @Override
   synchronized public EditLogOutputStream startLogSegment(long txid) 
       throws IOException {
-    currentInProgress = NNStorage.getInProgressEditsFile(sd, txid);
-    EditLogOutputStream stm = new EditLogFileOutputStream(currentInProgress,
-        outputBufferCapacity);
-    stm.create();
-    return stm;
+    try {
+      currentInProgress = NNStorage.getInProgressEditsFile(sd, txid);
+      EditLogOutputStream stm = new EditLogFileOutputStream(currentInProgress,
+          outputBufferCapacity);
+      stm.create();
+      return stm;
+    } catch (IOException e) {
+      storage.reportErrorsOnDirectory(sd);
+      throw e;
+    }
   }
 
   @Override
@@ -90,13 +96,14 @@
 
     File dstFile = NNStorage.getFinalizedEditsFile(
         sd, firstTxId, lastTxId);
-    LOG.debug("Finalizing edits file " + inprogressFile + " -> " + dstFile);
+    LOG.info("Finalizing edits file " + inprogressFile + " -> " + dstFile);
     
     Preconditions.checkState(!dstFile.exists(),
         "Can't finalize edits file " + inprogressFile + " since finalized file " +
         "already exists");
     if (!inprogressFile.renameTo(dstFile)) {
-      throw new IOException("Unable to finalize edits file " + inprogressFile);
+      storage.reportErrorsOnDirectory(sd);
+      throw new IllegalStateException("Unable to finalize edits file " + inprogressFile);
     }
     if (inprogressFile.equals(currentInProgress)) {
       currentInProgress = null;
@@ -116,6 +123,7 @@
   @Override
   public void purgeLogsOlderThan(long minTxIdToKeep)
       throws IOException {
+    LOG.info("Purging logs older than " + minTxIdToKeep);
     File[] files = FileUtil.listFiles(sd.getCurrentDir());
     List<EditLogFile> editLogs = 
       FileJournalManager.matchEditLogs(files);
@@ -135,18 +143,18 @@
    */
   List<RemoteEditLog> getRemoteEditLogs(long firstTxId) throws IOException {
     File currentDir = sd.getCurrentDir();
-    List<EditLogFile> allLogFiles = matchEditLogs(
-        FileUtil.listFiles(currentDir));
+    List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<RemoteEditLog> ret = Lists.newArrayListWithCapacity(
         allLogFiles.size());
 
     for (EditLogFile elf : allLogFiles) {
-      if (elf.isCorrupt() || elf.isInProgress()) continue;
+      if (elf.hasCorruptHeader() || elf.isInProgress()) continue;
       if (elf.getFirstTxId() >= firstTxId) {
         ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
       } else if ((firstTxId > elf.getFirstTxId()) &&
                  (firstTxId <= elf.getLastTxId())) {
-        throw new IOException("Asked for firstTxId " + firstTxId
+        // Note that this behavior is different from getLogFiles below.
+        throw new IllegalStateException("Asked for firstTxId " + firstTxId
             + " which is in the middle of file " + elf.file);
       }
     }
@@ -154,6 +162,20 @@
     return ret;
   }
 
+  /**
+   * returns matching edit logs via the log directory. Simple helper function
+   * that lists the files in the logDir and calls matchEditLogs(File[])
+   * 
+   * @param logDir
+   *          directory to match edit logs in
+   * @return matched edit logs
+   * @throws IOException
+   *           IOException thrown for invalid logDir
+   */
+  static List<EditLogFile> matchEditLogs(File logDir) throws IOException {
+    return matchEditLogs(FileUtil.listFiles(logDir));
+  }
+  
   static List<EditLogFile> matchEditLogs(File[] filesInStorage) {
     List<EditLogFile> ret = Lists.newArrayList();
     for (File f : filesInStorage) {
@@ -169,7 +191,7 @@
           LOG.error("Edits file " + f + " has improperly formatted " +
                     "transaction ID");
           // skip
-        }          
+        }
       }
       
       // Check for in-progress edits
@@ -190,27 +212,37 @@
   }
 
   @Override
-  synchronized public EditLogInputStream getInputStream(long fromTxId) 
-      throws IOException {
+  synchronized public EditLogInputStream getInputStream(long fromTxId,
+      boolean inProgressOk) throws IOException {
     for (EditLogFile elf : getLogFiles(fromTxId)) {
-      if (elf.getFirstTxId() == fromTxId) {
+      if (elf.containsTxId(fromTxId)) {
+        if (!inProgressOk && elf.isInProgress()) {
+          continue;
+        }
         if (elf.isInProgress()) {
           elf.validateLog();
         }
         if (LOG.isTraceEnabled()) {
           LOG.trace("Returning edit stream reading from " + elf);
         }
-        return new EditLogFileInputStream(elf.getFile(), 
-            elf.getFirstTxId(), elf.getLastTxId());
+        EditLogFileInputStream elfis = new EditLogFileInputStream(elf.getFile(),
+            elf.getFirstTxId(), elf.getLastTxId(), elf.isInProgress());
+        long transactionsToSkip = fromTxId - elf.getFirstTxId();
+        if (transactionsToSkip > 0) {
+          LOG.info(String.format("Log begins at txid %d, but requested start "
+              + "txid is %d. Skipping %d edits.", elf.getFirstTxId(), fromTxId,
+              transactionsToSkip));
+          elfis.skipTransactions(transactionsToSkip);
+        }
+        return elfis;
       }
     }
 
-    throw new IOException("Cannot find editlog file with " + fromTxId
-        + " as first first txid");
+    throw new IOException("Cannot find editlog file containing " + fromTxId);
   }
 
   @Override
-  public long getNumberOfTransactions(long fromTxId) 
+  public long getNumberOfTransactions(long fromTxId, boolean inProgressOk)
       throws IOException, CorruptionException {
     long numTxns = 0L;
     
@@ -222,21 +254,25 @@
         LOG.warn("Gap in transactions in " + sd.getRoot() + ". Gap is "
             + fromTxId + " - " + (elf.getFirstTxId() - 1));
         break;
-      } else if (fromTxId == elf.getFirstTxId()) {
+      } else if (elf.containsTxId(fromTxId)) {
+        if (!inProgressOk && elf.isInProgress()) {
+          break;
+        }
+        
         if (elf.isInProgress()) {
           elf.validateLog();
         } 
 
-        if (elf.isCorrupt()) {
+        if (elf.hasCorruptHeader()) {
           break;
         }
+        numTxns += elf.getLastTxId() + 1 - fromTxId;
         fromTxId = elf.getLastTxId() + 1;
-        numTxns += fromTxId - elf.getFirstTxId();
         
         if (elf.isInProgress()) {
           break;
         }
-      } // else skip
+      }
     }
 
     if (LOG.isDebugEnabled()) {
@@ -244,7 +280,8 @@
                 + " txns from " + fromTxId);
     }
 
-    long max = findMaxTransaction();
+    long max = findMaxTransaction(inProgressOk);
+    
     // fromTxId should be greater than max, as it points to the next 
     // transaction we should expect to find. If it is less than or equal
     // to max, it means that a transaction with txid == max has not been found
@@ -261,23 +298,44 @@
   @Override
   synchronized public void recoverUnfinalizedSegments() throws IOException {
     File currentDir = sd.getCurrentDir();
-    List<EditLogFile> allLogFiles = matchEditLogs(currentDir.listFiles());
-    
-    // make sure journal is aware of max seen transaction before moving corrupt 
-    // files aside
-    findMaxTransaction();
+    LOG.info("Recovering unfinalized segments in " + currentDir);
+    List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
 
     for (EditLogFile elf : allLogFiles) {
       if (elf.getFile().equals(currentInProgress)) {
         continue;
       }
       if (elf.isInProgress()) {
-        elf.validateLog();
-
-        if (elf.isCorrupt()) {
-          elf.moveAsideCorruptFile();
+        // If the file is zero-length, we likely just crashed after opening the
+        // file, but before writing anything to it. Safe to delete it.
+        if (elf.getFile().length() == 0) {
+          LOG.info("Deleting zero-length edit log file " + elf);
+          if (!elf.getFile().delete()) {
+            throw new IOException("Unable to delete file " + elf.getFile());
+          }
           continue;
         }
+        
+        elf.validateLog();
+
+        if (elf.hasCorruptHeader()) {
+          elf.moveAsideCorruptFile();
+          throw new CorruptionException("In-progress edit log file is corrupt: "
+              + elf);
+        }
+        
+        // If the file has a valid header (isn't corrupt) but contains no
+        // transactions, we likely just crashed after opening the file and
+        // writing the header, but before syncing any transactions. Safe to
+        // delete the file.
+        if (elf.getNumTransactions() == 0) {
+          LOG.info("Deleting edit log file with zero transactions " + elf);
+          if (!elf.getFile().delete()) {
+            throw new IOException("Unable to delete " + elf.getFile());
+          }
+          continue;
+        }
+        
         finalizeLogSegment(elf.getFirstTxId(), elf.getLastTxId());
       }
     }
@@ -285,16 +343,12 @@
 
   private List<EditLogFile> getLogFiles(long fromTxId) throws IOException {
     File currentDir = sd.getCurrentDir();
-    List<EditLogFile> allLogFiles = matchEditLogs(currentDir.listFiles());
+    List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<EditLogFile> logFiles = Lists.newArrayList();
     
     for (EditLogFile elf : allLogFiles) {
-      if (fromTxId > elf.getFirstTxId()
-          && fromTxId <= elf.getLastTxId()) {
-        throw new IOException("Asked for fromTxId " + fromTxId
-            + " which is in middle of file " + elf.file);
-      }
-      if (fromTxId <= elf.getFirstTxId()) {
+      if (fromTxId <= elf.getFirstTxId() ||
+          elf.containsTxId(fromTxId)) {
         logFiles.add(elf);
       }
     }
@@ -306,21 +360,35 @@
 
   /** 
    * Find the maximum transaction in the journal.
-   * This gets stored in a member variable, as corrupt edit logs
-   * will be moved aside, but we still need to remember their first
-   * tranaction id in the case that it was the maximum transaction in
-   * the journal.
    */
-  private long findMaxTransaction()
+  private long findMaxTransaction(boolean inProgressOk)
       throws IOException {
+    boolean considerSeenTxId = true;
+    long seenTxId = NNStorage.readTransactionIdFile(sd);
+    long maxSeenTransaction = 0;
     for (EditLogFile elf : getLogFiles(0)) {
+      if (elf.isInProgress() && !inProgressOk) {
+        if (elf.getFirstTxId() != HdfsConstants.INVALID_TXID &&
+            elf.getFirstTxId() <= seenTxId) {
+          // don't look at the seen_txid file if in-progress logs are not to be
+          // examined, and the value in seen_txid falls within the in-progress
+          // segment.
+          considerSeenTxId = false;
+        }
+        continue;
+      }
+      
       if (elf.isInProgress()) {
         maxSeenTransaction = Math.max(elf.getFirstTxId(), maxSeenTransaction);
         elf.validateLog();
       }
       maxSeenTransaction = Math.max(elf.getLastTxId(), maxSeenTransaction);
     }
-    return maxSeenTransaction;
+    if (considerSeenTxId) {
+      return Math.max(maxSeenTransaction, seenTxId);
+    } else {
+      return maxSeenTransaction;
+    }
   }
 
   @Override
@@ -335,8 +403,9 @@
     private File file;
     private final long firstTxId;
     private long lastTxId;
+    private long numTx = -1;
 
-    private boolean isCorrupt = false;
+    private boolean hasCorruptHeader = false;
     private final boolean isInProgress;
 
     final static Comparator<EditLogFile> COMPARE_BY_START_TXID 
@@ -376,6 +445,10 @@
     long getLastTxId() {
       return lastTxId;
     }
+    
+    boolean containsTxId(long txId) {
+      return firstTxId <= txId && txId <= lastTxId;
+    }
 
     /** 
      * Count the number of valid transactions in a log.
@@ -384,11 +457,13 @@
      */
     void validateLog() throws IOException {
       EditLogValidation val = EditLogFileInputStream.validateEditLog(file);
-      if (val.getNumTransactions() == 0) {
-        markCorrupt();
-      } else {
-        this.lastTxId = val.getEndTxId();
-      }
+      this.numTx = val.getNumTransactions();
+      this.lastTxId = val.getEndTxId();
+      this.hasCorruptHeader = val.hasCorruptHeader();
+    }
+    
+    long getNumTransactions() {
+      return numTx;
     }
 
     boolean isInProgress() {
@@ -399,16 +474,12 @@
       return file;
     }
     
-    void markCorrupt() {
-      isCorrupt = true;
-    }
-    
-    boolean isCorrupt() {
-      return isCorrupt;
+    boolean hasCorruptHeader() {
+      return hasCorruptHeader;
     }
 
     void moveAsideCorruptFile() throws IOException {
-      assert isCorrupt;
+      assert hasCorruptHeader;
     
       File src = file;
       File dst = new File(src.getParent(), src.getName() + ".corrupt");
@@ -423,8 +494,9 @@
     @Override
     public String toString() {
       return String.format("EditLogFile(file=%s,first=%019d,last=%019d,"
-                           +"inProgress=%b,corrupt=%b)", file.toString(),
-                           firstTxId, lastTxId, isInProgress(), isCorrupt);
+                           +"inProgress=%b,hasCorruptHeader=%b,numTx=%d)",
+                           file.toString(), firstTxId, lastTxId,
+                           isInProgress(), hasCorruptHeader, numTx);
     }
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
index 8753b27..b986003 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
@@ -124,16 +124,18 @@
             final long txid = parsedParams.getTxId();
 
             if (! currentlyDownloadingCheckpoints.add(txid)) {
-              throw new IOException(
+              response.sendError(HttpServletResponse.SC_CONFLICT,
                   "Another checkpointer is already in the process of uploading a" +
                   " checkpoint made at transaction ID " + txid);
+              return null;
             }
 
             try {
               if (nnImage.getStorage().findImageFile(txid) != null) {
-                throw new IOException(
+                response.sendError(HttpServletResponse.SC_CONFLICT,
                     "Another checkpointer already uploaded an checkpoint " +
                     "for txid " + txid);
+                return null;
               }
               
               // issue a HTTP get request to download the new fsimage 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
index 0fab53c..c5c47fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
@@ -26,6 +26,8 @@
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
+import com.google.common.base.Joiner;
+
 /**
  * I-node for file being written.
  */
@@ -41,19 +43,7 @@
                              String clientName,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
-    this(permissions, 0, replication, preferredBlockSize, modTime,
-        clientName, clientMachine, clientNode);
-  }
-
-  INodeFileUnderConstruction(PermissionStatus permissions,
-                             int nrBlocks,
-                             short replication,
-                             long preferredBlockSize,
-                             long modTime,
-                             String clientName,
-                             String clientMachine,
-                             DatanodeDescriptor clientNode) {
-    super(permissions.applyUMask(UMASK), nrBlocks, replication,
+    super(permissions.applyUMask(UMASK), 0, replication,
         modTime, modTime, preferredBlockSize);
     this.clientName = clientName;
     this.clientMachine = clientMachine;
@@ -106,6 +96,9 @@
   // use the modification time as the access time
   //
   INodeFile convertToInodeFile() {
+    assert allBlocksComplete() :
+      "Can't finalize inode " + this + " since it contains " +
+      "non-complete blocks! Blocks are: " + blocksAsString();
     INodeFile obj = new INodeFile(getPermissionStatus(),
                                   getBlocks(),
                                   getReplication(),
@@ -115,6 +108,18 @@
     return obj;
     
   }
+  
+  /**
+   * @return true if all of the blocks in this file are marked as completed.
+   */
+  private boolean allBlocksComplete() {
+    for (BlockInfo b : blocks) {
+      if (!b.isComplete()) {
+        return false;
+      }
+    }
+    return true;
+  }
 
   /**
    * Remove a block from the block list. This block should be
@@ -153,4 +158,8 @@
     setBlock(numBlocks()-1, ucBlock);
     return ucBlock;
   }
+  
+  private String blocksAsString() {
+    return Joiner.on(",").join(this.blocks);
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
index d45de18..f9c622d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
@@ -48,20 +48,23 @@
    /**
    * Get the input stream starting with fromTxnId from this journal manager
    * @param fromTxnId the first transaction id we want to read
+   * @param inProgressOk whether or not in-progress streams should be returned
    * @return the stream starting with transaction fromTxnId
    * @throws IOException if a stream cannot be found.
    */
-  EditLogInputStream getInputStream(long fromTxnId) throws IOException;
+  EditLogInputStream getInputStream(long fromTxnId, boolean inProgressOk)
+    throws IOException;
 
   /**
    * Get the number of transaction contiguously available from fromTxnId.
    *
    * @param fromTxnId Transaction id to count from
+   * @param inProgressOk whether or not in-progress streams should be counted
    * @return The number of transactions available from fromTxnId
    * @throws IOException if the journal cannot be read.
    * @throws CorruptionException if there is a gap in the journal at fromTxnId.
    */
-  long getNumberOfTransactions(long fromTxnId) 
+  long getNumberOfTransactions(long fromTxnId, boolean inProgressOk)
       throws IOException, CorruptionException;
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
index b1accd8..d84d79d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
@@ -25,8 +25,10 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -35,8 +37,6 @@
 import com.google.common.collect.Multimaps;
 import com.google.common.collect.Sets;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-
 /**
  * Manages a collection of Journals. None of the methods are synchronized, it is
  * assumed that FSEditLog methods, that use this class, use proper
@@ -148,11 +148,17 @@
   
   private List<JournalAndStream> journals = Lists.newArrayList();
   final int minimumRedundantJournals;
+  private volatile Runtime runtime = Runtime.getRuntime();
   
   JournalSet(int minimumRedundantResources) {
     this.minimumRedundantJournals = minimumRedundantResources;
   }
   
+  @VisibleForTesting
+  public void setRuntimeForTesting(Runtime runtime) {
+    this.runtime = runtime;
+  }
+  
   @Override
   public EditLogOutputStream startLogSegment(final long txId) throws IOException {
     mapJournalsAndReportErrors(new JournalClosure() {
@@ -201,19 +207,25 @@
    *         or null if no more exist
    */
   @Override
-  public EditLogInputStream getInputStream(long fromTxnId) throws IOException {
+  public EditLogInputStream getInputStream(long fromTxnId, boolean inProgressOk)
+      throws IOException {
     JournalManager bestjm = null;
     long bestjmNumTxns = 0;
     CorruptionException corruption = null;
 
     for (JournalAndStream jas : journals) {
+      if (jas.isDisabled()) continue;
+      
       JournalManager candidate = jas.getManager();
       long candidateNumTxns = 0;
       try {
-        candidateNumTxns = candidate.getNumberOfTransactions(fromTxnId);
+        candidateNumTxns = candidate.getNumberOfTransactions(fromTxnId,
+            inProgressOk);
       } catch (CorruptionException ce) {
         corruption = ce;
       } catch (IOException ioe) {
+        LOG.warn("Unable to read input streams from JournalManager " + candidate,
+            ioe);
         continue; // error reading disk, just skip
       }
       
@@ -231,15 +243,20 @@
         return null;
       }
     }
-    return bestjm.getInputStream(fromTxnId);
+    return bestjm.getInputStream(fromTxnId, inProgressOk);
   }
   
   @Override
-  public long getNumberOfTransactions(long fromTxnId) throws IOException {
+  public long getNumberOfTransactions(long fromTxnId, boolean inProgressOk)
+      throws IOException {
     long num = 0;
     for (JournalAndStream jas: journals) {
-      if (jas.isActive()) {
-        long newNum = jas.getManager().getNumberOfTransactions(fromTxnId);
+      if (jas.isDisabled()) {
+        LOG.info("Skipping jas " + jas + " since it's disabled");
+        continue;
+      } else {
+        long newNum = jas.getManager().getNumberOfTransactions(fromTxnId,
+            inProgressOk);
         if (newNum > num) {
           num = newNum;
         }
@@ -298,13 +315,31 @@
    */
   private void mapJournalsAndReportErrors(
       JournalClosure closure, String status) throws IOException{
+
     List<JournalAndStream> badJAS = Lists.newLinkedList();
     for (JournalAndStream jas : journals) {
       try {
         closure.apply(jas);
       } catch (Throwable t) {
-        LOG.error("Error: " + status + " failed for (journal " + jas + ")", t);
-        badJAS.add(jas);
+        if (jas.isRequired()) {
+          String msg = "Error: " + status + " failed for required journal ("
+            + jas + ")";
+          LOG.fatal(msg, t);
+          // If we fail on *any* of the required journals, then we must not
+          // continue on any of the other journals. Abort them to ensure that
+          // retry behavior doesn't allow them to keep going in any way.
+          abortAllJournals();
+          // the current policy is to shutdown the NN on errors to shared edits
+          // dir. There are many code paths to shared edits failures - syncs,
+          // roll of edits etc. All of them go through this common function 
+          // where the isRequired() check is made. Applying exit policy here 
+          // to catch all code paths.
+          runtime.exit(1);
+          throw new IOException(msg);
+        } else {
+          LOG.error("Error: " + status + " failed for (journal " + jas + ")", t);
+          badJAS.add(jas);          
+        }
       }
     }
     disableAndReportErrorOnJournals(badJAS);
@@ -317,6 +352,17 @@
   }
   
   /**
+   * Abort all of the underlying streams.
+   */
+  private void abortAllJournals() {
+    for (JournalAndStream jas : journals) {
+      if (jas.isActive()) {
+        jas.abort();
+      }
+    }
+  }
+
+  /**
    * An implementation of EditLogOutputStream that applies a requested method on
    * all the journals that are currently active.
    */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 6f03452..71e6cbb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -34,6 +34,10 @@
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.util.Daemon;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 
@@ -82,6 +86,9 @@
   //
   private SortedMap<String, Lease> sortedLeasesByPath = new TreeMap<String, Lease>();
 
+  private Daemon lmthread;
+  private volatile boolean shouldRunMonitor;
+
   LeaseManager(FSNamesystem fsnamesystem) {this.fsnamesystem = fsnamesystem;}
 
   Lease getLease(String holder) {
@@ -146,6 +153,9 @@
     Lease lease = getLease(holder);
     if (lease != null) {
       removeLease(lease, src);
+    } else {
+      LOG.warn("Removing non-existent lease! holder=" + holder +
+          " src=" + src);
     }
   }
 
@@ -190,6 +200,15 @@
     }
   }
 
+  /**
+   * Renew all of the currently open leases.
+   */
+  synchronized void renewAllLeases() {
+    for (Lease l : leases.values()) {
+      renewLease(l);
+    }
+  }
+
   /************************************************************
    * A Lease governs all the locks held by a single client.
    * For each client there's a corresponding lease, whose
@@ -296,6 +315,11 @@
       paths.remove(oldpath);
       paths.add(newpath);
     }
+    
+    @VisibleForTesting
+    long getLastUpdate() {
+      return lastUpdate;
+    }
   }
 
   synchronized void changeLease(String src, String dst,
@@ -367,18 +391,18 @@
 
     /** Check leases periodically. */
     public void run() {
-      for(; fsnamesystem.isRunning(); ) {
-        fsnamesystem.writeLock();
+      for(; shouldRunMonitor && fsnamesystem.isRunning(); ) {
         try {
-          if (!fsnamesystem.isInSafeMode()) {
-            checkLeases();
+          fsnamesystem.writeLockInterruptibly();
+          try {
+            if (!fsnamesystem.isInSafeMode()) {
+              checkLeases();
+            }
+          } finally {
+            fsnamesystem.writeUnlock();
           }
-        } finally {
-          fsnamesystem.writeUnlock();
-        }
-
-
-        try {
+  
+  
           Thread.sleep(HdfsServerConstants.NAMENODE_LEASE_RECHECK_INTERVAL);
         } catch(InterruptedException ie) {
           if (LOG.isDebugEnabled()) {
@@ -437,4 +461,36 @@
         + "\n sortedLeasesByPath=" + sortedLeasesByPath
         + "\n}";
   }
+
+  void startMonitor() {
+    Preconditions.checkState(lmthread == null,
+        "Lease Monitor already running");
+    shouldRunMonitor = true;
+    lmthread = new Daemon(new Monitor());
+    lmthread.start();
+  }
+  
+  void stopMonitor() {
+    if (lmthread != null) {
+      shouldRunMonitor = false;
+      try {
+        lmthread.interrupt();
+        lmthread.join(3000);
+      } catch (InterruptedException ie) {
+        LOG.warn("Encountered exception ", ie);
+      }
+      lmthread = null;
+    }
+  }
+
+  /**
+   * Trigger the currently-running Lease monitor to re-check
+   * its leases immediately. This is for use by unit tests.
+   */
+  @VisibleForTesting
+  void triggerMonitorCheckNow() {
+    Preconditions.checkState(lmthread != null,
+        "Lease monitor is not running");
+    lmthread.interrupt();
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index 7bca8f4..3f157e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -28,6 +28,7 @@
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -161,7 +162,8 @@
     
     // this may modify the editsDirs, so copy before passing in
     setStorageDirectories(imageDirs, 
-                          Lists.newArrayList(editsDirs));
+                          Lists.newArrayList(editsDirs),
+                          FSNamesystem.getSharedEditsDirs(conf));
   }
 
   @Override // Storage
@@ -249,6 +251,16 @@
   List<StorageDirectory> getRemovedStorageDirs() {
     return this.removedStorageDirs;
   }
+  
+  /**
+   * See {@link NNStorage#setStorageDirectories(Collection, Collection, Collection)}
+   */
+  @VisibleForTesting
+  synchronized void setStorageDirectories(Collection<URI> fsNameDirs,
+                                          Collection<URI> fsEditsDirs)
+      throws IOException {
+    setStorageDirectories(fsNameDirs, fsEditsDirs, new ArrayList<URI>());
+  }
 
   /**
    * Set the storage directories which will be used. This should only ever be
@@ -265,7 +277,8 @@
    */
   @VisibleForTesting
   synchronized void setStorageDirectories(Collection<URI> fsNameDirs,
-                                          Collection<URI> fsEditsDirs)
+                                          Collection<URI> fsEditsDirs,
+                                          Collection<URI> sharedEditsDirs)
       throws IOException {
     this.storageDirs.clear();
     this.removedStorageDirs.clear();
@@ -289,7 +302,8 @@
       if(dirName.getScheme().compareTo(JournalType.FILE.name().toLowerCase())
           == 0){
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
-            dirType));
+            dirType,
+            !sharedEditsDirs.contains(dirName))); // Don't lock the dir if it's shared.
       }
     }
 
@@ -301,7 +315,7 @@
       if(dirName.getScheme().compareTo(JournalType.FILE.name().toLowerCase())
           == 0)
         this.addStorageDir(new StorageDirectory(new File(dirName.getPath()),
-                    NameNodeDirType.EDITS));
+                    NameNodeDirType.EDITS, !sharedEditsDirs.contains(dirName)));
     }
   }
 
@@ -458,7 +472,7 @@
   /**
    * @return the transaction ID of the last checkpoint.
    */
-  long getMostRecentCheckpointTxId() {
+  public long getMostRecentCheckpointTxId() {
     return mostRecentCheckpointTxId;
   }
   
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
index fe65100..fe75247 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
@@ -31,6 +31,7 @@
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -46,6 +47,7 @@
 public class NNStorageRetentionManager {
   
   private final int numCheckpointsToRetain;
+  private final long numExtraEditsToRetain;
   private static final Log LOG = LogFactory.getLog(
       NNStorageRetentionManager.class);
   private final NNStorage storage;
@@ -60,6 +62,15 @@
     this.numCheckpointsToRetain = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY,
         DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT);
+    this.numExtraEditsToRetain = conf.getLong(
+        DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY,
+        DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT);
+    Preconditions.checkArgument(numCheckpointsToRetain > 0,
+        "Must retain at least one checkpoint");
+    Preconditions.checkArgument(numExtraEditsToRetain >= 0,
+        DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY +
+        " must not be negative");
+    
     this.storage = storage;
     this.editLog = editLog;
     this.purger = purger;
@@ -79,8 +90,12 @@
     purgeCheckpointsOlderThan(inspector, minImageTxId);
     // If fsimage_N is the image we want to keep, then we need to keep
     // all txns > N. We can remove anything < N+1, since fsimage_N
-    // reflects the state up to and including N.
-    editLog.purgeLogsOlderThan(minImageTxId + 1);
+    // reflects the state up to and including N. However, we also
+    // provide a "cushion" of older txns that we keep, which is
+    // handy for HA, where a remote node may not have as many
+    // new images.
+    long purgeLogsFrom = Math.max(0, minImageTxId + 1 - numExtraEditsToRetain);
+    editLog.purgeLogsOlderThan(purgeLogsFrom);
   }
   
   private void purgeCheckpointsOlderThan(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index a0f4d4b..b62f0d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -33,22 +30,40 @@
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.HealthCheckFailedException;
+import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Trash;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.ha.ActiveState;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
+import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
+import org.apache.hadoop.hdfs.server.namenode.ha.StandbyState;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+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 org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 
@@ -96,6 +111,22 @@
   }
   
   /**
+   * Categories of operations supported by the namenode.
+   */
+  public static enum OperationCategory {
+    /** Operations that are state agnostic */
+    UNCHECKED,
+    /** Read operation that does not change the namespace state */
+    READ,
+    /** Write operation that changes the namespace state */
+    WRITE,
+    /** Operations related to checkpointing */
+    CHECKPOINT,
+    /** Operations related to {@link JournalProtocol} */
+    JOURNAL
+  }
+  
+  /**
    * HDFS federation configuration can have two types of parameters:
    * <ol>
    * <li>Parameter that is common for all the name services in the cluster.</li>
@@ -110,6 +141,7 @@
     DFS_NAMENODE_RPC_ADDRESS_KEY,
     DFS_NAMENODE_NAME_DIR_KEY,
     DFS_NAMENODE_EDITS_DIR_KEY,
+    DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_DIR_KEY,
     DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
     DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
@@ -124,14 +156,40 @@
     DFS_NAMENODE_BACKUP_SERVICE_RPC_ADDRESS_KEY
   };
   
-
+  public long getProtocolVersion(String protocol, 
+                                 long clientVersion) throws IOException {
+    if (protocol.equals(ClientProtocol.class.getName())) {
+      return ClientProtocol.versionID; 
+    } else if (protocol.equals(DatanodeProtocol.class.getName())){
+      return DatanodeProtocol.versionID;
+    } else if (protocol.equals(NamenodeProtocol.class.getName())){
+      return NamenodeProtocol.versionID;
+    } else if (protocol.equals(RefreshAuthorizationPolicyProtocol.class.getName())){
+      return RefreshAuthorizationPolicyProtocol.versionID;
+    } else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
+      return RefreshUserMappingsProtocol.versionID;
+    } else if (protocol.equals(GetUserMappingsProtocol.class.getName())){
+      return GetUserMappingsProtocol.versionID;
+    } else {
+      throw new IOException("Unknown protocol to name node: " + protocol);
+    }
+  }
+    
   public static final int DEFAULT_PORT = 8020;
-
   public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
   public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange");
+  public static final HAState ACTIVE_STATE = new ActiveState();
+  public static final HAState STANDBY_STATE = new StandbyState();
   
   protected FSNamesystem namesystem; 
+  protected final Configuration conf;
   protected NamenodeRole role;
+  private HAState state;
+  private final boolean haEnabled;
+  private final HAContext haContext;
+  protected boolean allowStaleStandbyReads;
+
+  
   /** httpServer */
   protected NameNodeHttpServer httpServer;
   private Thread emptier;
@@ -212,7 +270,7 @@
    * @param filesystemURI
    * @return address of file system
    */
-  static InetSocketAddress getAddress(URI filesystemURI) {
+  public static InetSocketAddress getAddress(URI filesystemURI) {
     String authority = filesystemURI.getAuthority();
     if (authority == null) {
       throw new IllegalArgumentException(String.format(
@@ -251,13 +309,11 @@
    * Given a configuration get the address of the service rpc server
    * If the service rpc is not configured returns null
    */
-  protected InetSocketAddress getServiceRpcServerAddress(Configuration conf)
-      throws IOException {
+  protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) {
     return NameNode.getServiceAddress(conf, false);
   }
 
-  protected InetSocketAddress getRpcServerAddress(Configuration conf)
-      throws IOException {
+  protected InetSocketAddress getRpcServerAddress(Configuration conf) {
     return getAddress(conf);
   }
   
@@ -335,11 +391,7 @@
       throw e;
     }
 
-    activate(conf);
-    LOG.info(getRole() + " up at: " + rpcServer.getRpcAddress());
-    if (rpcServer.getServiceRpcAddress() != null) {
-      LOG.info(getRole() + " service server is up at: " + rpcServer.getServiceRpcAddress()); 
-    }
+    startCommonServices(conf);
   }
   
   /**
@@ -373,19 +425,11 @@
     } 
   }
 
-  /**
-   * Activate name-node servers and threads.
-   */
-  void activate(Configuration conf) throws IOException {
-    if ((isRole(NamenodeRole.NAMENODE))
-        && (UserGroupInformation.isSecurityEnabled())) {
-      namesystem.activateSecretManager();
-    }
-    namesystem.activate(conf);
+  /** Start the services common to active and standby states */
+  private void startCommonServices(Configuration conf) throws IOException {
+    namesystem.startCommonServices(conf, haContext);
     startHttpServer(conf);
     rpcServer.start();
-    startTrashEmptier(conf);
-    
     plugins = conf.getInstances(DFS_NAMENODE_PLUGINS_KEY,
         ServicePlugin.class);
     for (ServicePlugin p: plugins) {
@@ -395,8 +439,28 @@
         LOG.warn("ServicePlugin " + p + " could not be started", t);
       }
     }
+    LOG.info(getRole() + " up at: " + rpcServer.getRpcAddress());
+    if (rpcServer.getServiceRpcAddress() != null) {
+      LOG.info(getRole() + " service server is up at: "
+          + rpcServer.getServiceRpcAddress());
+    }
   }
-
+  
+  private void stopCommonServices() {
+    if(namesystem != null) namesystem.close();
+    if(rpcServer != null) rpcServer.stop();
+    if (plugins != null) {
+      for (ServicePlugin p : plugins) {
+        try {
+          p.stop();
+        } catch (Throwable t) {
+          LOG.warn("ServicePlugin " + p + " could not be stopped", t);
+        }
+      }
+    }   
+    stopHttpServer();
+  }
+  
   private void startTrashEmptier(Configuration conf) throws IOException {
     long trashInterval 
       = conf.getLong(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, 
@@ -408,11 +472,26 @@
     this.emptier.start();
   }
   
+  private void stopTrashEmptier() {
+    if (this.emptier != null) {
+      emptier.interrupt();
+      emptier = null;
+    }
+  }
+  
   private void startHttpServer(final Configuration conf) throws IOException {
     httpServer = new NameNodeHttpServer(conf, this, getHttpServerAddress(conf));
     httpServer.start();
     setHttpServerAddress(conf);
   }
+  
+  private void stopHttpServer() {
+    try {
+      if (httpServer != null) httpServer.stop();
+    } catch (Exception e) {
+      LOG.error("Exception while stopping httpserver", e);
+    }
+  }
 
   /**
    * Start NameNode.
@@ -447,10 +526,23 @@
 
   protected NameNode(Configuration conf, NamenodeRole role) 
       throws IOException { 
+    this.conf = conf;
     this.role = role;
+    String nsId = getNameServiceId(conf);
+    String namenodeId = HAUtil.getNameNodeId(conf, nsId);
+    this.haEnabled = HAUtil.isHAEnabled(conf, nsId);
+    if (!haEnabled) {
+      state = ACTIVE_STATE;
+    } else {
+      state = STANDBY_STATE;
+    }
+    this.allowStaleStandbyReads = HAUtil.shouldAllowStandbyReads(conf);
+    this.haContext = createHAContext();
     try {
-      initializeGenericKeys(conf, getNameServiceId(conf));
+      initializeGenericKeys(conf, nsId, namenodeId);
       initialize(conf);
+      state.prepareToEnterState(haContext);
+      state.enterState(haContext);
     } catch (IOException e) {
       this.stop();
       throw e;
@@ -460,6 +552,10 @@
     }
   }
 
+  protected HAContext createHAContext() {
+    return new NameNodeHAContext();
+  }
+
   /**
    * Wait for service to finish.
    * (Normally, it runs forever.)
@@ -468,6 +564,7 @@
     try {
       this.rpcServer.join();
     } catch (InterruptedException ie) {
+      LOG.info("Caught interrupted exception ", ie);
     }
   }
 
@@ -480,23 +577,14 @@
         return;
       stopRequested = true;
     }
-    if (plugins != null) {
-      for (ServicePlugin p : plugins) {
-        try {
-          p.stop();
-        } catch (Throwable t) {
-          LOG.warn("ServicePlugin " + p + " could not be stopped", t);
-        }
-      }
-    }
     try {
-      if (httpServer != null) httpServer.stop();
-    } catch (Exception e) {
-      LOG.error("Exception while stopping httpserver", e);
+      if (state != null) {
+        state.exitState(haContext);
+      }
+    } catch (ServiceFailedException e) {
+      LOG.warn("Encountered exception while exiting state ", e);
     }
-    if(namesystem != null) namesystem.close();
-    if(emptier != null) emptier.interrupt();
-    if(rpcServer != null) rpcServer.stop();
+    stopCommonServices();
     if (metrics != null) {
       metrics.shutdown();
     }
@@ -561,6 +649,10 @@
   private static boolean format(Configuration conf,
                                 boolean isConfirmationNeeded)
       throws IOException {
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    String namenodeId = HAUtil.getNameNodeId(conf, nsId);
+    initializeGenericKeys(conf, nsId, namenodeId);
+
     if (!conf.getBoolean(DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY, 
                          DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT)) {
       throw new IOException("The option " + DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY
@@ -571,7 +663,7 @@
     }
     
     Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
-    Collection<URI> editDirsToFormat = 
+    List<URI> editDirsToFormat = 
                  FSNamesystem.getNamespaceEditsDirs(conf);
     for(Iterator<URI> it = dirsToFormat.iterator(); it.hasNext();) {
       File curDir = new File(it.next().getPath());
@@ -605,6 +697,10 @@
   private static boolean finalize(Configuration conf,
                                boolean isConfirmationNeeded
                                ) throws IOException {
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    String namenodeId = HAUtil.getNameNodeId(conf, nsId);
+    initializeGenericKeys(conf, nsId, namenodeId);
+
     FSNamesystem nsys = new FSNamesystem(conf, new FSImage(conf));
     System.err.print(
         "\"finalize\" will remove the previous state of the files system.\n"
@@ -721,6 +817,14 @@
       return null;
     }
     setStartupOption(conf, startOpt);
+    
+    if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf)) &&
+        (startOpt == StartupOption.UPGRADE ||
+         startOpt == StartupOption.ROLLBACK ||
+         startOpt == StartupOption.FINALIZE)) {
+      throw new HadoopIllegalArgumentException("Invalid startup option. " +
+          "Cannot perform DFS upgrade with HA enabled.");
+    }
 
     switch (startOpt) {
       case FORMAT:
@@ -761,16 +865,26 @@
    * @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 name service Id
-   * @see DFSUtil#setGenericConf(Configuration, String, String...)
+   * @param nameserviceId name service Id (to distinguish federated NNs)
+   * @param namenodeId the namenode ID (to distinguish HA NNs)
+   * @see DFSUtil#setGenericConf(Configuration, String, String, String...)
    */
-  public static void initializeGenericKeys(Configuration conf, String
-      nameserviceId) {
-    if ((nameserviceId == null) || nameserviceId.isEmpty()) {
+  public static void initializeGenericKeys(Configuration conf,
+      String nameserviceId, String namenodeId) {
+    if ((nameserviceId == null || nameserviceId.isEmpty()) && 
+        (namenodeId == null || namenodeId.isEmpty())) {
       return;
     }
     
-    DFSUtil.setGenericConf(conf, nameserviceId, NAMESERVICE_SPECIFIC_KEYS);
+    if (nameserviceId != null) {
+      conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
+    }
+    if (namenodeId != null) {
+      conf.set(DFS_HA_NAMENODE_ID_KEY, namenodeId);
+    }
+    
+    DFSUtil.setGenericConf(conf, nameserviceId, namenodeId,
+        NAMESERVICE_SPECIFIC_KEYS);
     if (conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY) != null) {
       URI defaultUri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
           + conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY));
@@ -799,4 +913,131 @@
       System.exit(-1);
     }
   }
+
+  synchronized void monitorHealth() 
+      throws HealthCheckFailedException, AccessControlException {
+    namesystem.checkSuperuserPrivilege();
+    if (!haEnabled) {
+      return; // no-op, if HA is not enabled
+    }
+    getNamesystem().checkAvailableResources();
+    if (!getNamesystem().nameNodeHasResourcesAvailable()) {
+      throw new HealthCheckFailedException(
+          "The NameNode has no resources available");
+    }
+  }
+  
+  synchronized void transitionToActive() 
+      throws ServiceFailedException, AccessControlException {
+    namesystem.checkSuperuserPrivilege();
+    if (!haEnabled) {
+      throw new ServiceFailedException("HA for namenode is not enabled");
+    }
+    state.setState(haContext, ACTIVE_STATE);
+  }
+  
+  synchronized void transitionToStandby() 
+      throws ServiceFailedException, AccessControlException {
+    namesystem.checkSuperuserPrivilege();
+    if (!haEnabled) {
+      throw new ServiceFailedException("HA for namenode is not enabled");
+    }
+    state.setState(haContext, STANDBY_STATE);
+  }
+
+  synchronized HAServiceState getServiceState() throws AccessControlException {
+    namesystem.checkSuperuserPrivilege();
+    if (state == null) {
+      return HAServiceState.INITIALIZING;
+    }
+    return state.getServiceState();
+  }
+
+  synchronized boolean readyToBecomeActive()
+      throws ServiceFailedException, AccessControlException {
+    namesystem.checkSuperuserPrivilege();
+    if (!haEnabled) {
+      throw new ServiceFailedException("HA for namenode is not enabled");
+    }
+    return !isInSafeMode();
+  }
+
+  
+  /**
+   * Class used as expose {@link NameNode} as context to {@link HAState}
+   * 
+   * TODO(HA):
+   * When entering and exiting state, on failing to start services,
+   * appropriate action is needed todo either shutdown the node or recover
+   * from failure.
+   */
+  protected class NameNodeHAContext implements HAContext {
+    @Override
+    public void setState(HAState s) {
+      state = s;
+    }
+
+    @Override
+    public HAState getState() {
+      return state;
+    }
+
+    @Override
+    public void startActiveServices() throws IOException {
+      namesystem.startActiveServices();
+      startTrashEmptier(conf);
+    }
+
+    @Override
+    public void stopActiveServices() throws IOException {
+      if (namesystem != null) {
+        namesystem.stopActiveServices();
+      }
+      stopTrashEmptier();
+    }
+
+    @Override
+    public void startStandbyServices() throws IOException {
+      namesystem.startStandbyServices();
+    }
+
+    @Override
+    public void prepareToStopStandbyServices() throws ServiceFailedException {
+      namesystem.prepareToStopStandbyServices();
+    }
+    
+    @Override
+    public void stopStandbyServices() throws IOException {
+      if (namesystem != null) {
+        namesystem.stopStandbyServices();
+      }
+    }
+    
+    @Override
+    public void writeLock() {
+      namesystem.writeLock();
+    }
+    
+    @Override
+    public void writeUnlock() {
+      namesystem.writeUnlock();
+    }
+    
+    /** Check if an operation of given category is allowed */
+    @Override
+    public void checkOperation(final OperationCategory op)
+        throws StandbyException {
+      state.checkOperation(haContext, op);
+    }
+    
+    @Override
+    public boolean allowStaleReads() {
+      return allowStaleStandbyReads;
+    }
+
+  }
+  
+  public boolean isStandbyState() {
+    return (state.equals(STANDBY_STATE));
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
index e4817c7..a024a55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java
@@ -46,7 +46,7 @@
  * are added by default, and arbitrary extra volumes may be configured as well.
  */
 @InterfaceAudience.Private
-class NameNodeResourceChecker {
+public class NameNodeResourceChecker {
   private static final Log LOG = LogFactory.getLog(NameNodeResourceChecker.class.getName());
 
   // Space (in bytes) reserved per volume.
@@ -176,8 +176,7 @@
    *         least one redundant volume and all of the required volumes, false
    *         otherwise.
    */
-  boolean hasAvailableDiskSpace()
-      throws IOException {
+  public boolean hasAvailableDiskSpace() {
     return NameNodeResourcePolicy.areResourcesAvailable(volumes.values(),
         minimumRedundantVolumes);
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
index 53cd867..3896165 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
@@ -37,9 +37,6 @@
    *        required to continue operation.
    * @return true if and only if there are sufficient NN resources to
    *         continue logging edits.
-   * @throws RuntimeException if the number of <bold>configured</bold>
-   *         redundant resources is fewer than the minimum number of available
-   *         redundant resources.
    */
   static boolean areResourcesAvailable(
       Collection<? extends CheckableNameNodeResource> resources,
@@ -63,12 +60,6 @@
       }
     }
     
-    if (redundantResourceCount < minimumRedundantResources) {
-      throw new RuntimeException("Need a minimum of " + minimumRedundantResources
-          + " for NN to operate but only " + redundantResourceCount
-          + " are configured.");
-    }
-    
     if (redundantResourceCount == 0) {
       // If there are no redundant resources, return true if there are any
       // required resources available.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 0750600..17b387c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -41,6 +41,13 @@
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HealthCheckFailedException;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceProtocolService;
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolPB;
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolServerSideTranslatorPB;
+
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -82,6 +89,7 @@
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
@@ -90,6 +98,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
+import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@@ -104,6 +113,7 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.net.Node;
@@ -127,7 +137,7 @@
   private static final Log stateChangeLog = NameNode.stateChangeLog;
   
   // Dependencies from other parts of NN.
-  private final FSNamesystem namesystem;
+  protected final FSNamesystem namesystem;
   protected final NameNode nn;
   private final NameNodeMetrics metrics;
   
@@ -183,6 +193,11 @@
         new GetUserMappingsProtocolServerSideTranslatorPB(this);
     BlockingService getUserMappingService = GetUserMappingsProtocolService
         .newReflectiveBlockingService(getUserMappingXlator);
+    
+    HAServiceProtocolServerSideTranslatorPB haServiceProtocolXlator = 
+        new HAServiceProtocolServerSideTranslatorPB(this);
+    BlockingService haPbService = HAServiceProtocolService
+        .newReflectiveBlockingService(haServiceProtocolXlator);
 	  
     WritableRpcEngine.ensureInitialized();
     
@@ -198,6 +213,8 @@
           dnSocketAddr.getHostName(), dnSocketAddr.getPort(), 
           serviceHandlerCount,
           false, conf, namesystem.getDelegationTokenSecretManager());
+      DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
+          serviceRpcServer);
       DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
           serviceRpcServer);
       DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
@@ -221,6 +238,8 @@
         clientNNPbService, socAddr.getHostName(),
             socAddr.getPort(), handlerCount, false, conf,
             namesystem.getDelegationTokenSecretManager());
+    DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
+        clientRpcServer);
     DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
         clientRpcServer);
     DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
@@ -287,7 +306,7 @@
       throw new IllegalArgumentException(
         "Unexpected not positive size: "+size);
     }
-
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getBlockManager().getBlocks(datanode, size); 
   }
 
@@ -300,6 +319,7 @@
   public void errorReport(NamenodeRegistration registration,
                           int errorCode, 
                           String msg) throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     verifyRequest(registration);
     LOG.info("Error report from " + registration + ": " + msg);
     if(errorCode == FATAL)
@@ -327,9 +347,6 @@
   @Override // NamenodeProtocol
   public void endCheckpoint(NamenodeRegistration registration,
                             CheckpointSignature sig) throws IOException {
-    verifyRequest(registration);
-    if(!nn.isRole(NamenodeRole.NAMENODE))
-      throw new IOException("Only an ACTIVE node can invoke endCheckpoint.");
     namesystem.endCheckpoint(registration, sig);
   }
 
@@ -478,10 +495,10 @@
     return namesystem.getAdditionalDatanode(src, blk,
         existings, excludeSet, numAdditionalNodes, clientName);
   }
-
   /**
    * The client needs to give up on the block.
    */
+  @Override // ClientProtocol
   public void abandonBlock(ExtendedBlock b, String src, String holder)
       throws IOException {
     if(stateChangeLog.isDebugEnabled()) {
@@ -509,18 +526,9 @@
    * mark the block as corrupt.  In the future we might 
    * check the blocks are actually corrupt. 
    */
-  @Override
+  @Override // ClientProtocol, DatanodeProtocol
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
-    for (int i = 0; i < blocks.length; i++) {
-      ExtendedBlock blk = blocks[i].getBlock();
-      DatanodeInfo[] nodes = blocks[i].getLocations();
-      for (int j = 0; j < nodes.length; j++) {
-        DatanodeInfo dn = nodes[j];
-        namesystem.getBlockManager().findAndMarkBlockAsCorrupt(blk, dn,
-            "client machine reported it");
-      }
-    }
+    namesystem.reportBadBlocks(blocks);
   }
 
   @Override // ClientProtocol
@@ -633,8 +641,7 @@
 
   @Override // ClientProtocol
   public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation)
-  throws IOException {
+      boolean needLocation) throws IOException {
     DirectoryListing files = namesystem.getListing(
         src, startAfter, needLocation);
     if (files != null) {
@@ -656,14 +663,16 @@
     return namesystem.getFileInfo(src, false);
   }
   
-  @Override
-  public long[] getStats() {
+  @Override // ClientProtocol
+  public long[] getStats() throws IOException {
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getStats();
   }
 
   @Override // ClientProtocol
   public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
-      throws IOException {
+  throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     DatanodeInfo results[] = namesystem.datanodeReport(type);
     if (results == null ) {
       throw new IOException("Cannot find datanode report");
@@ -673,28 +682,32 @@
     
   @Override // ClientProtocol
   public boolean setSafeMode(SafeModeAction action) throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     return namesystem.setSafeMode(action);
   }
 
   @Override // ClientProtocol
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException {
+  public boolean restoreFailedStorage(String arg) throws IOException { 
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     return namesystem.restoreFailedStorage(arg);
   }
 
   @Override // ClientProtocol
   public void saveNamespace() throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.saveNamespace();
   }
 
   @Override // ClientProtocol
   public void refreshNodes() throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.getBlockManager().getDatanodeManager().refreshNodes(
         new HdfsConfiguration());
   }
 
   @Override // NamenodeProtocol
-  public long getTransactionID() {
+  public long getTransactionID() throws IOException {
+    namesystem.checkOperation(OperationCategory.CHECKPOINT);
     return namesystem.getEditLog().getSyncTxId();
   }
 
@@ -703,32 +716,36 @@
     return namesystem.rollEditLog();
   }
   
-  @Override
+  @Override // NamenodeProtocol
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
   throws IOException {
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.getEditLog().getEditLogManifest(sinceTxId);
   }
     
   @Override // ClientProtocol
   public void finalizeUpgrade() throws IOException {
+    namesystem.checkOperation(OperationCategory.WRITE);
     namesystem.finalizeUpgrade();
   }
 
   @Override // ClientProtocol
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
       throws IOException {
+    namesystem.checkOperation(OperationCategory.READ);
     return namesystem.distributedUpgradeProgress(action);
   }
 
   @Override // ClientProtocol
   public void metaSave(String filename) throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.metaSave(filename);
   }
 
   @Override // ClientProtocol
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
-	String[] cookieTab = new String[] { cookie };
+    String[] cookieTab = new String[] { cookie };
     Collection<FSNamesystem.CorruptFileBlockInfo> fbs =
       namesystem.listCorruptFileBlocks(path, cookieTab);
 
@@ -743,11 +760,12 @@
   /**
    * Tell all datanodes to use a new, non-persistent bandwidth value for
    * dfs.datanode.balance.bandwidthPerSec.
-   * @param bandwidth Blanacer bandwidth in bytes per second for all datanodes.
+   * @param bandwidth Balancer bandwidth in bytes per second for all datanodes.
    * @throws IOException
    */
   @Override // ClientProtocol
   public void setBalancerBandwidth(long bandwidth) throws IOException {
+    namesystem.checkOperation(OperationCategory.UNCHECKED);
     namesystem.getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
   }
   
@@ -822,7 +840,7 @@
   }
 
   @Override // DatanodeProtocol
-  public DatanodeCommand[] sendHeartbeat(DatanodeRegistration nodeReg,
+  public HeartbeatResponse sendHeartbeat(DatanodeRegistration nodeReg,
       StorageReport[] report, int xmitsInProgress, int xceiverCount,
       int failedVolumes) throws IOException {
     verifyRequest(nodeReg);
@@ -844,7 +862,7 @@
     }
 
     namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
-    if (nn.getFSImage().isUpgradeFinalized())
+    if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState())
       return new FinalizeCommand(poolId);
     return null;
   }
@@ -858,7 +876,7 @@
           +"from "+nodeReg.getName()+" "+receivedAndDeletedBlocks.length
           +" blocks.");
     }
-    namesystem.getBlockManager().blockReceivedAndDeleted(
+    namesystem.getBlockManager().processIncrementalBlockReport(
         nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks());
   }
 
@@ -946,6 +964,35 @@
     return UserGroupInformation.createRemoteUser(user).getGroupNames();
   }
 
+  @Override // HAServiceProtocol
+  public synchronized void monitorHealth() 
+      throws HealthCheckFailedException, AccessControlException {
+    nn.monitorHealth();
+  }
+  
+  @Override // HAServiceProtocol
+  public synchronized void transitionToActive() 
+      throws ServiceFailedException, AccessControlException {
+    nn.transitionToActive();
+  }
+  
+  @Override // HAServiceProtocol
+  public synchronized void transitionToStandby() 
+      throws ServiceFailedException, AccessControlException {
+    nn.transitionToStandby();
+  }
+
+  @Override // HAServiceProtocol
+  public synchronized HAServiceState getServiceState() 
+      throws AccessControlException {
+    return nn.getServiceState();
+  }
+
+  @Override // HAServiceProtocol
+  public synchronized boolean readyToBecomeActive() 
+      throws ServiceFailedException, AccessControlException {
+    return nn.readyToBecomeActive();
+  }
 
   /**
    * Verify version.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
index 64b2723..44c0751 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
@@ -36,6 +36,7 @@
 import javax.servlet.jsp.JspWriter;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -309,7 +310,16 @@
 
       long bpUsed = fsnStats[6];
       float percentBpUsed = DFSUtil.getPercentUsed(bpUsed, total);
-      
+
+      // don't show under-replicated/missing blocks or corrupt files for SBN
+      // since the standby namenode doesn't compute replication queues 
+      String underReplicatedBlocks = "";
+      if (nn.getServiceState() == HAServiceState.ACTIVE) {
+    	  underReplicatedBlocks = rowTxt() 
+              + colTxt("Excludes missing blocks.")
+              + "Number of Under-Replicated Blocks" + colTxt() + ":" + colTxt()
+              + fsn.getBlockManager().getUnderReplicatedNotMissingBlocks(); 
+      }
       out.print("<div id=\"dfstable\"> <table>\n" + rowTxt() + colTxt()
           + "Configured Capacity" + colTxt() + ":" + colTxt()
           + StringUtils.byteDesc(total) + rowTxt() + colTxt() + "DFS Used"
@@ -344,10 +354,8 @@
           + rowTxt() + colTxt()
           + "<a href=\"dfsnodelist.jsp?whatNodes=DECOMMISSIONING\">"
           + "Decommissioning Nodes</a> "
-          + colTxt() + ":" + colTxt() + decommissioning.size() 
-          + rowTxt() + colTxt("Excludes missing blocks.")
-          + "Number of Under-Replicated Blocks" + colTxt() + ":" + colTxt()
-          + fsn.getBlockManager().getUnderReplicatedNotMissingBlocks()
+          + colTxt() + ":" + colTxt() + decommissioning.size()
+          + underReplicatedBlocks
           + "</table></div><br>\n");
 
       if (live.isEmpty() && dead.isEmpty()) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 6846e95..c453db5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -32,4 +32,10 @@
 
   /** @return the block pool ID */
   public String getBlockPoolId();
+
+  public boolean isInStandbyState();
+
+  public boolean isGenStampInFuture(long generationStamp);
+
+  public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
 }
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SaveNamespaceCancelledException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SaveNamespaceCancelledException.java
index 2731275..5b49f0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SaveNamespaceCancelledException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SaveNamespaceCancelledException.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import org.apache.hadoop.classification.InterfaceAudience;;
 
-class SaveNamespaceCancelledException extends IOException {
+@InterfaceAudience.Private
+public class SaveNamespaceCancelledException extends IOException {
   private static final long serialVersionUID = 1L;
 
   SaveNamespaceCancelledException(String cancelReason) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index 50dca62..c1ce79e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -45,10 +45,11 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
@@ -112,17 +113,9 @@
   private String infoBindAddress;
 
   private Collection<URI> checkpointDirs;
-  private Collection<URI> checkpointEditsDirs;
-  
-  /** How often to checkpoint regardless of number of txns */
-  private long checkpointPeriod;    // in seconds
-  
-  /** How often to poll the NN to check checkpointTxnCount */
-  private long checkpointCheckPeriod; // in seconds
-  
-  /** checkpoint once every this many transactions, regardless of time */
-  private long checkpointTxnCount;
+  private List<URI> checkpointEditsDirs;
 
+  private CheckpointConf checkpointConf;
   private FSNamesystem namesystem;
 
 
@@ -132,9 +125,9 @@
       + "\nName Node Address    : " + nameNodeAddr   
       + "\nStart Time           : " + new Date(starttime)
       + "\nLast Checkpoint Time : " + (lastCheckpointTime == 0? "--": new Date(lastCheckpointTime))
-      + "\nCheckpoint Period    : " + checkpointPeriod + " seconds"
-      + "\nCheckpoint Size      : " + StringUtils.byteDesc(checkpointTxnCount)
-                                    + " (= " + checkpointTxnCount + " bytes)" 
+      + "\nCheckpoint Period    : " + checkpointConf.getPeriod() + " seconds"
+      + "\nCheckpoint Size      : " + StringUtils.byteDesc(checkpointConf.getTxnCount())
+                                    + " (= " + checkpointConf.getTxnCount() + " bytes)" 
       + "\nCheckpoint Dirs      : " + checkpointDirs
       + "\nCheckpoint Edits Dirs: " + checkpointEditsDirs;
   }
@@ -174,16 +167,19 @@
   public SecondaryNameNode(Configuration conf,
       CommandLineOpts commandLineOpts) throws IOException {
     try {
-      NameNode.initializeGenericKeys(conf,
-          DFSUtil.getSecondaryNameServiceId(conf));
+      String nsId = DFSUtil.getSecondaryNameServiceId(conf);
+      if (HAUtil.isHAEnabled(conf, nsId)) {
+        throw new IOException(
+            "Cannot use SecondaryNameNode in an HA cluster." +
+            " The Standby Namenode will perform checkpointing.");
+      }
+      NameNode.initializeGenericKeys(conf, nsId, null);
       initialize(conf, commandLineOpts);
-    } catch(IOException e) {
+    } catch (IOException e) {
       shutdown();
-      LOG.fatal("Failed to start secondary namenode. ", e);
       throw e;
-    } catch(HadoopIllegalArgumentException e) {
+    } catch (HadoopIllegalArgumentException e) {
       shutdown();
-      LOG.fatal("Failed to start secondary namenode. ", e);
       throw e;
     }
   }
@@ -216,8 +212,9 @@
     nameNodeAddr = NameNode.getServiceAddress(conf, true);
 
     this.conf = conf;
-    this.namenode = new NamenodeProtocolTranslatorPB(nameNodeAddr, conf,
-        UserGroupInformation.getCurrentUser());
+    this.namenode = NameNodeProxies.createNonHAProxy(conf, nameNodeAddr, 
+        NamenodeProtocol.class, UserGroupInformation.getCurrentUser(),
+        true).getProxy();
 
     // initialize checkpoint directories
     fsName = getInfoServer();
@@ -231,16 +228,8 @@
     namesystem = new FSNamesystem(conf, checkpointImage);
 
     // Initialize other scheduling parameters from the configuration
-    checkpointCheckPeriod = conf.getLong(
-        DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
-        DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT);
-        
-    checkpointPeriod = conf.getLong(DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 
-                                    DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
-    checkpointTxnCount = conf.getLong(DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 
-                                  DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
-    warnForDeprecatedConfigs(conf);
-
+    checkpointConf = new CheckpointConf(conf);
+    
     // initialize the webserver for uploading files.
     // Kerberized SSL servers must be run from the host principal...
     UserGroupInformation httpUGI = 
@@ -296,21 +285,9 @@
     conf.set(DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, infoBindAddress + ":" +infoPort); 
     LOG.info("Secondary Web-server up at: " + infoBindAddress + ":" +infoPort);
     LOG.info("Secondary image servlet up at: " + infoBindAddress + ":" + imagePort);
-    LOG.info("Checkpoint Period   :" + checkpointPeriod + " secs " +
-             "(" + checkpointPeriod/60 + " min)");
-    LOG.info("Log Size Trigger    :" + checkpointTxnCount + " txns");
-  }
-
-  static void warnForDeprecatedConfigs(Configuration conf) {
-    for (String key : ImmutableList.of(
-          "fs.checkpoint.size",
-          "dfs.namenode.checkpoint.size")) {
-      if (conf.get(key) != null) {
-        LOG.warn("Configuration key " + key + " is deprecated! Ignoring..." +
-            " Instead please specify a value for " +
-            DFS_NAMENODE_CHECKPOINT_TXNS_KEY);
-      }
-    }
+    LOG.info("Checkpoint Period   :" + checkpointConf.getPeriod() + " secs " +
+             "(" + checkpointConf.getPeriod()/60 + " min)");
+    LOG.info("Log Size Trigger    :" + checkpointConf.getTxnCount() + " txns");
   }
 
   /**
@@ -332,36 +309,24 @@
   }
 
   public void run() {
-    if (UserGroupInformation.isSecurityEnabled()) {
-      UserGroupInformation ugi = null;
-      try { 
-        ugi = UserGroupInformation.getLoginUser();
-      } catch (IOException e) {
-        LOG.error("Exception while getting login user", e);
-        e.printStackTrace();
-        Runtime.getRuntime().exit(-1);
-      }
-      ugi.doAs(new PrivilegedAction<Object>() {
+    SecurityUtil.doAsLoginUserOrFatal(
+        new PrivilegedAction<Object>() {
         @Override
         public Object run() {
           doWork();
           return null;
         }
       });
-    } else {
-      doWork();
-    }
   }
   //
   // The main work loop
   //
   public void doWork() {
-
     //
     // Poll the Namenode (once every checkpointCheckPeriod seconds) to find the
     // number of transactions in the edit log that haven't yet been checkpointed.
     //
-    long period = Math.min(checkpointCheckPeriod, checkpointPeriod);
+    long period = checkpointConf.getCheckPeriod();
 
     while (shouldRun) {
       try {
@@ -380,7 +345,7 @@
         long now = System.currentTimeMillis();
 
         if (shouldCheckpointBasedOnCount() ||
-            now >= lastCheckpointTime + 1000 * checkpointPeriod) {
+            now >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {
           doCheckpoint();
           lastCheckpointTime = now;
         }
@@ -471,19 +436,10 @@
     }
 
     String configuredAddress = DFSUtil.getInfoServer(null, conf, true);
-    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 fsName.getHost() + ":" + sockAddr.getPort();
-    } else {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("configuredAddress = " + configuredAddress);
-      }
-      return configuredAddress;
-    }
+    String address = DFSUtil.substituteForWildcardAddress(configuredAddress,
+        fsName.getHost());
+    LOG.debug("Will connect to NameNode at HTTP address: " + address);
+    return address;
   }
   
   /**
@@ -574,13 +530,13 @@
       switch (opts.getCommand()) {
       case CHECKPOINT:
         long count = countUncheckpointedTxns();
-        if (count > checkpointTxnCount ||
+        if (count > checkpointConf.getTxnCount() ||
             opts.shouldForceCheckpoint()) {
           doCheckpoint();
         } else {
           System.err.println("EditLog size " + count + " transactions is " +
                              "smaller than configured checkpoint " +
-                             "interval " + checkpointTxnCount + " transactions.");
+                             "interval " + checkpointConf.getTxnCount() + " transactions.");
           System.err.println("Skipping checkpoint.");
         }
         break;
@@ -626,7 +582,7 @@
   }
 
   boolean shouldCheckpointBasedOnCount() throws IOException {
-    return countUncheckpointedTxns() >= checkpointTxnCount;
+    return countUncheckpointedTxns() >= checkpointConf.getTxnCount();
   }
 
   /**
@@ -642,7 +598,13 @@
     
     StringUtils.startupShutdownMessage(SecondaryNameNode.class, argv, LOG);
     Configuration tconf = new HdfsConfiguration();
-    SecondaryNameNode secondary = new SecondaryNameNode(tconf, opts);
+    SecondaryNameNode secondary = null;
+    try {
+      secondary = new SecondaryNameNode(tconf, opts);
+    } catch (IOException ioe) {
+      LOG.fatal("Failed to start secondary namenode", ioe);
+      System.exit(-1);
+    }
 
     if (opts.getCommand() != null) {
       int ret = secondary.processStartupCommand(opts);
@@ -759,7 +721,7 @@
      */
     CheckpointStorage(Configuration conf, 
                       Collection<URI> imageDirs,
-                      Collection<URI> editsDirs) throws IOException {
+                      List<URI> editsDirs) throws IOException {
       super(conf, imageDirs, editsDirs);
       
       // the 2NN never writes edits -- it only downloads them. So
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index cc8dcca..985d85b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -24,8 +24,11 @@
 import java.util.List;
 import java.lang.Math;
 
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -41,7 +44,8 @@
 /**
  * This class provides fetching a specified file from the NameNode.
  */
-class TransferFsImage {
+@InterfaceAudience.Private
+public class TransferFsImage {
   
   public final static String CONTENT_LENGTH = "Content-Length";
   public final static String MD5_HEADER = "X-MD5-Digest";
@@ -103,7 +107,7 @@
    * @param storage the storage directory to transfer the image from
    * @param txid the transaction ID of the image to be uploaded
    */
-  static void uploadImageFromStorage(String fsName,
+  public static void uploadImageFromStorage(String fsName,
       InetSocketAddress imageListenAddress,
       NNStorage storage, long txid) throws IOException {
     
@@ -111,7 +115,20 @@
         txid, imageListenAddress, storage);
     // this doesn't directly upload an image, but rather asks the NN
     // to connect back to the 2NN to download the specified image.
-    TransferFsImage.getFileClient(fsName, fileid, null, null, false);
+    try {
+      TransferFsImage.getFileClient(fsName, fileid, null, null, false);
+    } catch (HttpGetFailedException e) {
+      if (e.getResponseCode() == HttpServletResponse.SC_CONFLICT) {
+        // this is OK - this means that a previous attempt to upload
+        // this checkpoint succeeded even though we thought it failed.
+        LOG.info("Image upload with txid " + txid + 
+            " conflicted with a previous image upload to the " +
+            "same NameNode. Continuing...", e);
+        return;
+      } else {
+        throw e;
+      }
+    }
     LOG.info("Uploaded image with txid " + txid + " to namenode at " +
     		fsName);
   }
@@ -194,10 +211,11 @@
     HttpURLConnection connection = (HttpURLConnection) url.openConnection();
     
     if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
-      throw new IOException(
+      throw new HttpGetFailedException(
           "Image transfer servlet at " + url +
           " failed with status code " + connection.getResponseCode() +
-          "\nResponse message:\n" + connection.getResponseMessage());
+          "\nResponse message:\n" + connection.getResponseMessage(),
+          connection);
     }
     
     long advertisedSize;
@@ -289,5 +307,19 @@
     String header = connection.getHeaderField(MD5_HEADER);
     return (header != null) ? new MD5Hash(header) : null;
   }
+  
+  public static class HttpGetFailedException extends IOException {
+    private static final long serialVersionUID = 1L;
+    private final int responseCode;
+
+    HttpGetFailedException(String msg, HttpURLConnection connection) throws IOException {
+      super(msg);
+      this.responseCode = connection.getResponseCode();
+    }
+    
+    public int getResponseCode() {
+      return responseCode;
+    }
+  }
 
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java
index 9ac17fc..ca7e1d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/UnsupportedActionException.java
@@ -32,8 +32,7 @@
   /** for java.io.Serializable */
   private static final long serialVersionUID = 1L;
 
-  public UnsupportedActionException(String action) {
-    super("Action " + action + "() is not supported.");
+  public UnsupportedActionException(String msg) {
+    super(msg);
   }
-
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java
new file mode 100644
index 0000000..a61e134
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ActiveState.java
@@ -0,0 +1,76 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+
+/**
+ * Active state of the namenode. In this state, namenode provides the namenode
+ * service and handles operations of type {@link OperationCategory#WRITE} and
+ * {@link OperationCategory#READ}.
+ */
+@InterfaceAudience.Private
+public class ActiveState extends HAState {
+  public ActiveState() {
+    super(HAServiceState.ACTIVE);
+  }
+
+  @Override
+  public void checkOperation(HAContext context, OperationCategory op) {
+    return; // Other than journal all operations are allowed in active state
+  }
+  
+  @Override
+  public boolean shouldPopulateReplQueues() {
+    return true;
+  }
+  
+  @Override
+  public void setState(HAContext context, HAState s) throws ServiceFailedException {
+    if (s == NameNode.STANDBY_STATE) {
+      setStateInternal(context, s);
+      return;
+    }
+    super.setState(context, s);
+  }
+
+  @Override
+  public void enterState(HAContext context) throws ServiceFailedException {
+    try {
+      context.startActiveServices();
+    } catch (IOException e) {
+      throw new ServiceFailedException("Failed to start active services", e);
+    }
+  }
+
+  @Override
+  public void exitState(HAContext context) throws ServiceFailedException {
+    try {
+      context.stopActiveServices();
+    } catch (IOException e) {
+      throw new ServiceFailedException("Failed to stop active services", e);
+    }
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
new file mode 100644
index 0000000..a20880a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
@@ -0,0 +1,167 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+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.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A FailoverProxyProvider implementation which allows one to configure two URIs
+ * to connect to during fail-over. The first configured address is tried first,
+ * and on a fail-over event the other address is tried.
+ */
+public class ConfiguredFailoverProxyProvider<T> implements
+    FailoverProxyProvider<T> {
+  
+  private static final Log LOG =
+      LogFactory.getLog(ConfiguredFailoverProxyProvider.class);
+  
+  private final Configuration conf;
+  private final List<AddressRpcProxyPair<T>> proxies =
+      new ArrayList<AddressRpcProxyPair<T>>();
+  private final UserGroupInformation ugi;
+  private final Class<T> xface;
+  
+  private int currentProxyIndex = 0;
+
+  public ConfiguredFailoverProxyProvider(Configuration conf, URI uri,
+      Class<T> xface) {
+    Preconditions.checkArgument(
+        xface.isAssignableFrom(NamenodeProtocols.class),
+        "Interface class %s is not a valid NameNode protocol!");
+    this.xface = xface;
+    
+    this.conf = new Configuration(conf);
+    int maxRetries = this.conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY,
+        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
+        maxRetries);
+    
+    int maxRetriesOnSocketTimeouts = this.conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        DFSConfigKeys.DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT);
+    this.conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        maxRetriesOnSocketTimeouts);
+    
+    try {
+      ugi = UserGroupInformation.getCurrentUser();
+      
+      Map<String, Map<String, InetSocketAddress>> map = DFSUtil.getHaNnRpcAddresses(
+          conf);
+      Map<String, InetSocketAddress> addressesInNN = map.get(uri.getHost());
+      
+      if (addressesInNN == null || addressesInNN.size() == 0) {
+        throw new RuntimeException("Could not find any configured addresses " +
+            "for URI " + uri);
+      }
+      
+      for (InetSocketAddress address : addressesInNN.values()) {
+        proxies.add(new AddressRpcProxyPair<T>(address));
+        
+        // The client may have a delegation token set for the logical
+        // URI of the cluster. Clone this token to apply to each of the
+        // underlying IPC addresses so that the IPC code can find it.
+        HAUtil.cloneDelegationTokenForLogicalUri(ugi, uri, address);
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+    
+  @Override
+  public Class<T> getInterface() {
+    return xface;
+  }
+
+  /**
+   * Lazily initialize the RPC proxy object.
+   */
+  @SuppressWarnings("unchecked")
+  @Override
+  public synchronized T getProxy() {
+    AddressRpcProxyPair current = proxies.get(currentProxyIndex);
+    if (current.namenode == null) {
+      try {
+        current.namenode = NameNodeProxies.createNonHAProxy(conf,
+            current.address, xface, ugi, false).getProxy();
+      } catch (IOException e) {
+        LOG.error("Failed to create RPC proxy to NameNode", e);
+        throw new RuntimeException(e);
+      }
+    }
+    return (T)current.namenode;
+  }
+
+  @Override
+  public synchronized void performFailover(T currentProxy) {
+    currentProxyIndex = (currentProxyIndex + 1) % proxies.size();
+  }
+
+  /**
+   * A little pair object to store the address and connected RPC proxy object to
+   * an NN. Note that {@link AddressRpcProxyPair#namenode} may be null.
+   */
+  private static class AddressRpcProxyPair<T> {
+    public InetSocketAddress address;
+    public T namenode;
+    
+    public AddressRpcProxyPair(InetSocketAddress address) {
+      this.address = address;
+    }
+  }
+
+  /**
+   * Close all the proxy objects which have been opened over the lifetime of
+   * this proxy provider.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    for (AddressRpcProxyPair<T> proxy : proxies) {
+      if (proxy.namenode != null) {
+        if (proxy.namenode instanceof Closeable) {
+          ((Closeable)proxy.namenode).close();
+        } else {
+          RPC.stopProxy(proxy.namenode);
+        }
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
new file mode 100644
index 0000000..780bad7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -0,0 +1,339 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedAction;
+import java.util.Collection;
+
+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.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputException;
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.SecurityUtil;
+
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+
+/**
+ * EditLogTailer represents a thread which periodically reads from edits
+ * journals and applies the transactions contained within to a given
+ * FSNamesystem.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class EditLogTailer {
+  public static final Log LOG = LogFactory.getLog(EditLogTailer.class);
+  
+  private final EditLogTailerThread tailerThread;
+  
+  private final FSNamesystem namesystem;
+  private FSEditLog editLog;
+  
+  private volatile Runtime runtime = Runtime.getRuntime();
+
+  private InetSocketAddress activeAddr;
+  private NamenodeProtocol cachedActiveProxy = null;
+
+  /**
+   * The last transaction ID at which an edit log roll was initiated.
+   */
+  private long lastRollTriggerTxId = HdfsConstants.INVALID_TXID;
+  
+  /**
+   * The highest transaction ID loaded by the Standby.
+   */
+  private long lastLoadedTxnId = HdfsConstants.INVALID_TXID;
+
+  /**
+   * The last time we successfully loaded a non-zero number of edits from the
+   * shared directory.
+   */
+  private long lastLoadTimestamp;
+
+  /**
+   * How often the Standby should roll edit logs. Since the Standby only reads
+   * from finalized log segments, the Standby will only be as up-to-date as how
+   * often the logs are rolled.
+   */
+  private long logRollPeriodMs;
+
+  /**
+   * How often the Standby should check if there are new finalized segment(s)
+   * available to be read from.
+   */
+  private long sleepTimeMs;
+  
+  public EditLogTailer(FSNamesystem namesystem) {
+    this.tailerThread = new EditLogTailerThread();
+    this.namesystem = namesystem;
+    this.editLog = namesystem.getEditLog();
+    
+
+    Configuration conf = namesystem.getConf();
+    lastLoadTimestamp = now();
+
+    logRollPeriodMs = conf.getInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
+        DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT) * 1000;
+    if (logRollPeriodMs >= 0) {
+      this.activeAddr = getActiveNodeAddress();
+      Preconditions.checkArgument(activeAddr.getPort() > 0,
+          "Active NameNode must have an IPC port configured. " +
+          "Got address '%s'", activeAddr);
+      LOG.info("Will roll logs on active node at " + activeAddr + " every " +
+          (logRollPeriodMs / 1000) + " seconds.");
+    } else {
+      LOG.info("Not going to trigger log rolls on active node because " +
+          DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY + " is negative.");
+    }
+    
+    sleepTimeMs = conf.getInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY,
+        DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT) * 1000;
+    
+    LOG.debug("logRollPeriodMs=" + logRollPeriodMs +
+        " sleepTime=" + sleepTimeMs);
+  }
+  
+  private InetSocketAddress getActiveNodeAddress() {
+    Configuration conf = namesystem.getConf();
+    Configuration activeConf = HAUtil.getConfForOtherNode(conf);
+    return NameNode.getServiceAddress(activeConf, true);
+  }
+  
+  private NamenodeProtocol getActiveNodeProxy() throws IOException {
+    if (cachedActiveProxy == null) {
+      Configuration conf = namesystem.getConf();
+      NamenodeProtocolPB proxy = 
+        RPC.waitForProxy(NamenodeProtocolPB.class,
+            RPC.getProtocolVersion(NamenodeProtocolPB.class), activeAddr, conf);
+      cachedActiveProxy = new NamenodeProtocolTranslatorPB(proxy);
+    }
+    assert cachedActiveProxy != null;
+    return cachedActiveProxy;
+  }
+
+  public void start() {
+    tailerThread.start();
+  }
+  
+  public void stop() throws IOException {
+    tailerThread.setShouldRun(false);
+    tailerThread.interrupt();
+    try {
+      tailerThread.join();
+    } catch (InterruptedException e) {
+      LOG.warn("Edit log tailer thread exited with an exception");
+      throw new IOException(e);
+    }
+  }
+  
+  @VisibleForTesting
+  FSEditLog getEditLog() {
+    return editLog;
+  }
+  
+  @VisibleForTesting
+  void setEditLog(FSEditLog editLog) {
+    this.editLog = editLog;
+  }
+  
+  @VisibleForTesting
+  synchronized void setRuntime(Runtime runtime) {
+    this.runtime = runtime;
+  }
+  
+  public void catchupDuringFailover() throws IOException {
+    Preconditions.checkState(tailerThread == null ||
+        !tailerThread.isAlive(),
+        "Tailer thread should not be running once failover starts");
+    try {
+      doTailEdits();
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+  
+  private void doTailEdits() throws IOException, InterruptedException {
+    // Write lock needs to be interruptible here because the 
+    // transitionToActive RPC takes the write lock before calling
+    // tailer.stop() -- so if we're not interruptible, it will
+    // deadlock.
+    namesystem.writeLockInterruptibly();
+    try {
+      FSImage image = namesystem.getFSImage();
+
+      long lastTxnId = image.getLastAppliedTxId();
+      
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("lastTxnId: " + lastTxnId);
+      }
+      Collection<EditLogInputStream> streams;
+      try {
+        streams = editLog.selectInputStreams(lastTxnId + 1, 0, false);
+      } catch (IOException ioe) {
+        // This is acceptable. If we try to tail edits in the middle of an edits
+        // log roll, i.e. the last one has been finalized but the new inprogress
+        // edits file hasn't been started yet.
+        LOG.warn("Edits tailer failed to find any streams. Will try again " +
+            "later.", ioe);
+        return;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("edit streams to load from: " + streams.size());
+      }
+      
+      // Once we have streams to load, errors encountered are legitimate cause
+      // for concern, so we don't catch them here. Simple errors reading from
+      // disk are ignored.
+      long editsLoaded = 0;
+      try {
+        editsLoaded = image.loadEdits(streams, namesystem);
+      } catch (EditLogInputException elie) {
+        editsLoaded = elie.getNumEditsLoaded();
+        throw elie;
+      } finally {
+        if (editsLoaded > 0 || LOG.isDebugEnabled()) {
+          LOG.info(String.format("Loaded %d edits starting from txid %d ",
+              editsLoaded, lastTxnId));
+        }
+      }
+
+      if (editsLoaded > 0) {
+        lastLoadTimestamp = now();
+      }
+      lastLoadedTxnId = image.getLastAppliedTxId();
+    } finally {
+      namesystem.writeUnlock();
+    }
+  }
+
+  /**
+   * @return timestamp (in msec) of when we last loaded a non-zero number of edits.
+   */
+  public long getLastLoadTimestamp() {
+    return lastLoadTimestamp;
+  }
+
+  /**
+   * @return true if the configured log roll period has elapsed.
+   */
+  private boolean tooLongSinceLastLoad() {
+    return logRollPeriodMs >= 0 && 
+      (now() - lastLoadTimestamp) > logRollPeriodMs ;
+  }
+
+  /**
+   * Trigger the active node to roll its logs.
+   */
+  private void triggerActiveLogRoll() {
+    LOG.info("Triggering log roll on remote NameNode " + activeAddr);
+    try {
+      getActiveNodeProxy().rollEditLog();
+      lastRollTriggerTxId = lastLoadedTxnId;
+    } catch (IOException ioe) {
+      LOG.warn("Unable to trigger a roll of the active NN", ioe);
+    }
+  }
+
+  /**
+   * The thread which does the actual work of tailing edits journals and
+   * applying the transactions to the FSNS.
+   */
+  private class EditLogTailerThread extends Thread {
+    private volatile boolean shouldRun = true;
+    
+    private EditLogTailerThread() {
+      super("Edit log tailer");
+    }
+    
+    private void setShouldRun(boolean shouldRun) {
+      this.shouldRun = shouldRun;
+    }
+    
+    @Override
+    public void run() {
+      SecurityUtil.doAsLoginUserOrFatal(
+          new PrivilegedAction<Object>() {
+          @Override
+          public Object run() {
+            doWork();
+            return null;
+          }
+        });
+    }
+    
+    private void doWork() {
+      while (shouldRun) {
+        try {
+          // There's no point in triggering a log roll if the Standby hasn't
+          // read any more transactions since the last time a roll was
+          // triggered. 
+          if (tooLongSinceLastLoad() &&
+              lastRollTriggerTxId < lastLoadedTxnId) {
+            triggerActiveLogRoll();
+          }
+          /**
+           * Check again in case someone calls {@link EditLogTailer#stop} while
+           * we're triggering an edit log roll, since ipc.Client catches and
+           * ignores {@link InterruptedException} in a few places. This fixes
+           * the bug described in HDFS-2823.
+           */
+          if (!shouldRun) {
+            break;
+          }
+          doTailEdits();
+        } catch (EditLogInputException elie) {
+          LOG.warn("Error while reading edits from disk. Will try again.", elie);
+        } catch (InterruptedException ie) {
+          // interrupter should have already set shouldRun to false
+          continue;
+        } catch (Throwable t) {
+          LOG.error("Unknown error encountered while tailing edits. " +
+              "Shutting down standby NN.", t);
+          runtime.exit(1);
+        }
+
+        try {
+          Thread.sleep(sleepTimeMs);
+        } catch (InterruptedException e) {
+          LOG.warn("Edit log tailer interrupted", e);
+        }
+      }
+    }
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java
new file mode 100644
index 0000000..6b070b2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java
@@ -0,0 +1,61 @@
+package org.apache.hadoop.hdfs.server.namenode.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.ipc.StandbyException;
+
+/**
+ * Context that is to be used by {@link HAState} for getting/setting the
+ * current state and performing required operations.
+ */
+@InterfaceAudience.Private
+public interface HAContext {
+  /** Set the state of the context to given {@code state} */
+  public void setState(HAState state);
+  
+  /** Get the state from the context */
+  public HAState getState();
+  
+  /** Start the services required in active state */
+  public void startActiveServices() throws IOException;
+  
+  /** Stop the services when exiting active state */
+  public void stopActiveServices() throws IOException;
+  
+  /** Start the services required in standby state */
+  public void startStandbyServices() throws IOException;
+
+  /** Prepare to exit the standby state */
+  public void prepareToStopStandbyServices() throws ServiceFailedException;
+
+  /** Stop the services when exiting standby state */
+  public void stopStandbyServices() throws IOException;
+
+  /**
+   * Take a write-lock on the underlying namesystem
+   * so that no concurrent state transitions or edits
+   * can be made.
+   */
+  void writeLock();
+
+  /**
+   * Unlock the lock taken by {@link #writeLock()}
+   */
+  void writeUnlock();
+
+  /**
+   * Verify that the given operation category is allowed in the
+   * current state. This is to allow NN implementations (eg BackupNode)
+   * to override it with node-specific handling.
+   */
+  void checkOperation(OperationCategory op) throws StandbyException;
+
+  /**
+   * @return true if the node should allow stale reads (ie reads
+   * while the namespace is not up to date)
+   */
+  boolean allowStaleReads();
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java
new file mode 100644
index 0000000..20ea854
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java
@@ -0,0 +1,148 @@
+/**
+ * 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.namenode.ha;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.ipc.StandbyException;
+
+/**
+ * Namenode base state to implement state machine pattern.
+ */
+@InterfaceAudience.Private
+abstract public class HAState {
+  protected final HAServiceState state;
+
+  /**
+   * Constructor
+   * @param name Name of the state.
+   */
+  public HAState(HAServiceState state) {
+    this.state = state;
+  }
+
+  /**
+   * @return the generic service state
+   */
+  public HAServiceState getServiceState() {
+    return state;
+  }
+
+  /**
+   * Internal method to transition the state of a given namenode to a new state.
+   * @param nn Namenode
+   * @param s new state
+   * @throws ServiceFailedException on failure to transition to new state.
+   */
+  protected final void setStateInternal(final HAContext context, final HAState s)
+      throws ServiceFailedException {
+    prepareToExitState(context);
+    s.prepareToEnterState(context);
+    context.writeLock();
+    try {
+      exitState(context);
+      context.setState(s);
+      s.enterState(context);
+    } finally {
+      context.writeUnlock();
+    }
+  }
+
+  /**
+   * Method to be overridden by subclasses to prepare to enter a state.
+   * This method is called <em>without</em> the context being locked,
+   * and after {@link #prepareToExitState(HAContext)} has been called
+   * for the previous state, but before {@link #exitState(HAContext)}
+   * has been called for the previous state.
+   * @param context HA context
+   * @throws ServiceFailedException on precondition failure
+   */
+  public void prepareToEnterState(final HAContext context)
+      throws ServiceFailedException {}
+
+  /**
+   * Method to be overridden by subclasses to perform steps necessary for
+   * entering a state.
+   * @param context HA context
+   * @throws ServiceFailedException on failure to enter the state.
+   */
+  public abstract void enterState(final HAContext context)
+      throws ServiceFailedException;
+
+  /**
+   * Method to be overridden by subclasses to prepare to exit a state.
+   * This method is called <em>without</em> the context being locked.
+   * This is used by the standby state to cancel any checkpoints
+   * that are going on. It can also be used to check any preconditions
+   * for the state transition.
+   * 
+   * This method should not make any destructuve changes to the state
+   * (eg stopping threads) since {@link #prepareToEnterState(HAContext)}
+   * may subsequently cancel the state transition.
+   * @param context HA context
+   * @throws ServiceFailedException on precondition failure
+   */
+  public void prepareToExitState(final HAContext context)
+      throws ServiceFailedException {}
+
+  /**
+   * Method to be overridden by subclasses to perform steps necessary for
+   * exiting a state.
+   * @param context HA context
+   * @throws ServiceFailedException on failure to enter the state.
+   */
+  public abstract void exitState(final HAContext context)
+      throws ServiceFailedException;
+
+  /**
+   * Move from the existing state to a new state
+   * @param context HA context
+   * @param s new state
+   * @throws ServiceFailedException on failure to transition to new state.
+   */
+  public void setState(HAContext context, HAState s) throws ServiceFailedException {
+    if (this == s) { // Aleady in the new state
+      return;
+    }
+    throw new ServiceFailedException("Transtion from state " + this + " to "
+        + s + " is not allowed.");
+  }
+  
+  /**
+   * Check if an operation is supported in a given state.
+   * @param context HA context
+   * @param op Type of the operation.
+   * @throws UnsupportedActionException if a given type of operation is not
+   *           supported in this state.
+   */
+  public abstract void checkOperation(final HAContext context, final OperationCategory op)
+      throws StandbyException;
+
+  public abstract boolean shouldPopulateReplQueues();
+
+  /**
+   * @return String representation of the service state.
+   */
+  @Override
+  public String toString() {
+    return state.toString();
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
new file mode 100644
index 0000000..036dd43
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -0,0 +1,299 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedAction;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.server.namenode.CheckpointConf;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceCancelledException;
+import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Thread which runs inside the NN when it's in Standby state,
+ * periodically waking up to take a checkpoint of the namespace.
+ * When it takes a checkpoint, it saves it to its local
+ * storage and then uploads it to the remote NameNode.
+ */
+@InterfaceAudience.Private
+public class StandbyCheckpointer {
+  private static final Log LOG = LogFactory.getLog(StandbyCheckpointer.class);
+  private static final long PREVENT_AFTER_CANCEL_MS = 2*60*1000L;
+  private final CheckpointConf checkpointConf;
+  private final FSNamesystem namesystem;
+  private long lastCheckpointTime;
+  private final CheckpointerThread thread;
+  private String activeNNAddress;
+  private InetSocketAddress myNNAddress;
+  
+  // Keep track of how many checkpoints were canceled.
+  // This is for use in tests.
+  private static int canceledCount = 0;
+  
+  public StandbyCheckpointer(Configuration conf, FSNamesystem ns) {
+    this.namesystem = ns;
+    this.checkpointConf = new CheckpointConf(conf); 
+    this.thread = new CheckpointerThread();
+
+    setNameNodeAddresses(conf);
+  }
+
+  /**
+   * Determine the address of the NN we are checkpointing
+   * as well as our own HTTP address from the configuration.
+   */
+  private void setNameNodeAddresses(Configuration conf) {
+    // Look up our own address.
+    String myAddrString = getHttpAddress(conf);
+
+    // Look up the active node's address
+    Configuration confForActive = HAUtil.getConfForOtherNode(conf);
+    activeNNAddress = getHttpAddress(confForActive);
+    
+    // Sanity-check.
+    Preconditions.checkArgument(checkAddress(activeNNAddress),
+        "Bad address for active NN: %s", activeNNAddress);
+    Preconditions.checkArgument(checkAddress(myAddrString),
+        "Bad address for standby NN: %s", myAddrString);
+    myNNAddress = NetUtils.createSocketAddr(myAddrString);
+  }
+  
+  private String getHttpAddress(Configuration conf) {
+    String configuredAddr = DFSUtil.getInfoServer(null, conf, true);
+    
+    // Use the hostname from the RPC address as a default, in case
+    // the HTTP address is configured to 0.0.0.0.
+    String hostnameFromRpc = NameNode.getServiceAddress(
+        conf, true).getHostName();
+    try {
+      return DFSUtil.substituteForWildcardAddress(
+          configuredAddr, hostnameFromRpc);
+    } catch (IOException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+  
+  /**
+   * Ensure that the given address is valid and has a port
+   * specified.
+   */
+  private boolean checkAddress(String addrStr) {
+    InetSocketAddress addr = NetUtils.createSocketAddr(addrStr);
+    return addr.getPort() != 0 && !addr.getAddress().isAnyLocalAddress();
+  }
+
+  public void start() {
+    LOG.info("Starting standby checkpoint thread...\n" +
+        "Checkpointing active NN at " + activeNNAddress + "\n" +
+        "Serving checkpoints at " + myNNAddress);
+    thread.start();
+  }
+  
+  public void stop() throws IOException {
+    thread.setShouldRun(false);
+    thread.interrupt();
+    try {
+      thread.join();
+    } catch (InterruptedException e) {
+      LOG.warn("Edit log tailer thread exited with an exception");
+      throw new IOException(e);
+    }
+  }
+
+  private void doCheckpoint() throws InterruptedException, IOException {
+    long txid;
+    
+    namesystem.writeLockInterruptibly();
+    try {
+      assert namesystem.getEditLog().isOpenForRead() :
+        "Standby Checkpointer should only attempt a checkpoint when " +
+        "NN is in standby mode, but the edit logs are in an unexpected state";
+      
+      FSImage img = namesystem.getFSImage();
+      
+      long prevCheckpointTxId = img.getStorage().getMostRecentCheckpointTxId();
+      long thisCheckpointTxId = img.getLastAppliedOrWrittenTxId();
+      assert thisCheckpointTxId >= prevCheckpointTxId;
+      if (thisCheckpointTxId == prevCheckpointTxId) {
+        LOG.info("A checkpoint was triggered but the Standby Node has not " +
+            "received any transactions since the last checkpoint at txid " +
+            thisCheckpointTxId + ". Skipping...");
+        return;
+      }
+      
+      img.saveNamespace(namesystem);
+      txid = img.getStorage().getMostRecentCheckpointTxId();
+      assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +
+        thisCheckpointTxId + " but instead saved at txid=" + txid;
+    } finally {
+      namesystem.writeUnlock();
+    }
+    
+    // Upload the saved checkpoint back to the active
+    TransferFsImage.uploadImageFromStorage(
+        activeNNAddress, myNNAddress,
+        namesystem.getFSImage().getStorage(), txid);
+  }
+  
+  /**
+   * Cancel any checkpoint that's currently being made,
+   * and prevent any new checkpoints from starting for the next
+   * minute or so.
+   */
+  public void cancelAndPreventCheckpoints() throws ServiceFailedException {
+    try {
+      thread.preventCheckpointsFor(PREVENT_AFTER_CANCEL_MS);
+      // TODO(HA): there is a really narrow race here if we are just
+      // about to start a checkpoint - this won't cancel it!
+      namesystem.getFSImage().cancelSaveNamespace(
+          "About to exit standby state");
+    } catch (InterruptedException e) {
+      throw new ServiceFailedException(
+          "Interrupted while trying to cancel checkpoint");
+    }
+  }
+  
+  @VisibleForTesting
+  static int getCanceledCount() {
+    return canceledCount;
+  }
+
+  private long countUncheckpointedTxns() {
+    FSImage img = namesystem.getFSImage();
+    return img.getLastAppliedOrWrittenTxId() -
+      img.getStorage().getMostRecentCheckpointTxId();
+  }
+
+  private class CheckpointerThread extends Thread {
+    private volatile boolean shouldRun = true;
+    private volatile long preventCheckpointsUntil = 0;
+
+    private CheckpointerThread() {
+      super("Standby State Checkpointer");
+    }
+    
+    private void setShouldRun(boolean shouldRun) {
+      this.shouldRun = shouldRun;
+    }
+
+    @Override
+    public void run() {
+      // We have to make sure we're logged in as far as JAAS
+      // is concerned, in order to use kerberized SSL properly.
+      SecurityUtil.doAsLoginUserOrFatal(
+          new PrivilegedAction<Object>() {
+          @Override
+          public Object run() {
+            doWork();
+            return null;
+          }
+        });
+    }
+
+    /**
+     * Prevent checkpoints from occurring for some time period
+     * in the future. This is used when preparing to enter active
+     * mode. We need to not only cancel any concurrent checkpoint,
+     * but also prevent any checkpoints from racing to start just
+     * after the cancel call.
+     * 
+     * @param delayMs the number of MS for which checkpoints will be
+     * prevented
+     */
+    private void preventCheckpointsFor(long delayMs) {
+      preventCheckpointsUntil = now() + delayMs;
+    }
+
+    private void doWork() {
+      // Reset checkpoint time so that we don't always checkpoint
+      // on startup.
+      lastCheckpointTime = now();
+      while (shouldRun) {
+        try {
+          Thread.sleep(1000 * checkpointConf.getCheckPeriod());
+        } catch (InterruptedException ie) {
+        }
+        if (!shouldRun) {
+          break;
+        }
+        try {
+          // We may have lost our ticket since last checkpoint, log in again, just in case
+          if (UserGroupInformation.isSecurityEnabled()) {
+            UserGroupInformation.getCurrentUser().reloginFromKeytab();
+          }
+          
+          long now = now();
+          long uncheckpointed = countUncheckpointedTxns();
+          long secsSinceLast = (now - lastCheckpointTime)/1000;
+          
+          boolean needCheckpoint = false;
+          if (uncheckpointed >= checkpointConf.getTxnCount()) {
+            LOG.info("Triggering checkpoint because there have been " + 
+                uncheckpointed + " txns since the last checkpoint, which " +
+                "exceeds the configured threshold " +
+                checkpointConf.getTxnCount());
+            needCheckpoint = true;
+          } else if (secsSinceLast >= checkpointConf.getPeriod()) {
+            LOG.info("Triggering checkpoint because it has been " +
+                secsSinceLast + " seconds since the last checkpoint, which " +
+                "exceeds the configured interval " + checkpointConf.getPeriod());
+            needCheckpoint = true;
+          }
+          if (needCheckpoint && now < preventCheckpointsUntil) {
+            LOG.info("But skipping this checkpoint since we are about to failover!");
+            canceledCount++;
+          } else if (needCheckpoint) {
+            doCheckpoint();
+            lastCheckpointTime = now;
+          }
+        } catch (SaveNamespaceCancelledException ce) {
+          LOG.info("Checkpoint was cancelled: " + ce.getMessage());
+          canceledCount++;
+        } catch (InterruptedException ie) {
+          // Probably requested shutdown.
+          continue;
+        } catch (Throwable t) {
+          LOG.error("Exception in doCheckpoint", t);
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  String getActiveNNAddress() {
+    return activeNNAddress;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java
new file mode 100644
index 0000000..60e8371
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyState.java
@@ -0,0 +1,95 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.ipc.StandbyException;
+
+/**
+ * Namenode standby state. In this state the namenode acts as warm standby and
+ * keeps the following updated:
+ * <ul>
+ * <li>Namespace by getting the edits.</li>
+ * <li>Block location information by receiving block reports and blocks
+ * received from the datanodes.</li>
+ * </ul>
+ * 
+ * It does not handle read/write/checkpoint operations.
+ */
+@InterfaceAudience.Private
+public class StandbyState extends HAState {
+  public StandbyState() {
+    super(HAServiceState.STANDBY);
+  }
+
+  @Override
+  public void setState(HAContext context, HAState s) throws ServiceFailedException {
+    if (s == NameNode.ACTIVE_STATE) {
+      setStateInternal(context, s);
+      return;
+    }
+    super.setState(context, s);
+  }
+
+  @Override
+  public void enterState(HAContext context) throws ServiceFailedException {
+    try {
+      context.startStandbyServices();
+    } catch (IOException e) {
+      throw new ServiceFailedException("Failed to start standby services", e);
+    }
+  }
+
+  @Override
+  public void prepareToExitState(HAContext context) throws ServiceFailedException {
+    context.prepareToStopStandbyServices();
+  }
+
+  @Override
+  public void exitState(HAContext context) throws ServiceFailedException {
+    try {
+      context.stopStandbyServices();
+    } catch (IOException e) {
+      throw new ServiceFailedException("Failed to stop standby services", e);
+    }
+  }
+
+  @Override
+  public void checkOperation(HAContext context, OperationCategory op)
+      throws StandbyException {
+    if (op == OperationCategory.UNCHECKED ||
+        (op == OperationCategory.READ && context.allowStaleReads())) {
+      return;
+    }
+    String msg = "Operation category " + op + " is not supported in state "
+        + context.getState();
+    throw new StandbyException(msg);
+  }
+
+  @Override
+  public boolean shouldPopulateReplQueues() {
+    return false;
+  }
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index c9c7150..5669497 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -22,8 +22,8 @@
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.security.KerberosInfo;
 
@@ -93,7 +93,7 @@
    * sendHeartbeat() tells the NameNode that the DataNode is still
    * alive and well.  Includes some status info, too. 
    * It also gives the NameNode a chance to return 
-   * an array of "DatanodeCommand" objects.
+   * an array of "DatanodeCommand" objects in HeartbeatResponse.
    * A DatanodeCommand tells the DataNode to invalidate local block(s), 
    * or to copy them to other DataNodes, etc.
    * @param registration datanode registration information
@@ -103,7 +103,7 @@
    * @param failedVolumes number of failed volumes
    * @throws IOException on error
    */
-  public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
+  public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
                                        StorageReport[] reports,
                                        int xmitsInProgress,
                                        int xceiverCount,
@@ -118,7 +118,8 @@
    * @param registration
    * @param poolId - the block pool ID for the blocks
    * @param reports - report of blocks per storage
-   *     Each block is represented as 2 longs.
+   *     Each finalized block is represented as 3 longs. Each under-
+   *     construction replica is represented as 4 longs.
    *     This is done instead of Block[] to reduce memory used by block reports.
    *     
    * @return - the next command for DN to process.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
new file mode 100644
index 0000000..96f74a0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
@@ -0,0 +1,85 @@
+/**
+ * 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.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.ObjectWritable;
+import org.apache.hadoop.io.Writable;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+/**
+ * Response to {@link DatanodeProtocol#sendHeartbeat}
+ */
+public class HeartbeatResponse implements Writable {
+  /** Commands returned from the namenode to the datanode */
+  private DatanodeCommand[] commands;
+  
+  /** Information about the current HA-related state of the NN */
+  private NNHAStatusHeartbeat haStatus;
+  
+  public HeartbeatResponse() {
+    // Empty constructor required for Writable
+  }
+  
+  public HeartbeatResponse(DatanodeCommand[] cmds,
+      NNHAStatusHeartbeat haStatus) {
+    commands = cmds;
+    this.haStatus = haStatus;
+  }
+  
+  public DatanodeCommand[] getCommands() {
+    return commands;
+  }
+  
+  public NNHAStatusHeartbeat getNameNodeHaState() {
+    return haStatus;
+  }
+
+  ///////////////////////////////////////////
+  // Writable
+  ///////////////////////////////////////////
+  @Override
+  public void write(DataOutput out) throws IOException {
+    int length = commands == null ? 0 : commands.length;
+    out.writeInt(length);
+    for (int i = 0; i < length; i++) {
+      ObjectWritable.writeObject(out, commands[i], commands[i].getClass(),
+                                 null, true);
+    }
+    haStatus.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int length = in.readInt();
+    commands = new DatanodeCommand[length];
+    ObjectWritable objectWritable = new ObjectWritable();
+    for (int i = 0; i < length; i++) {
+      commands[i] = (DatanodeCommand) ObjectWritable.readObject(in,
+          objectWritable, null);
+    }
+    haStatus = new NNHAStatusHeartbeat();
+    haStatus.readFields(in);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
new file mode 100644
index 0000000..633aa85
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NNHAStatusHeartbeat.java
@@ -0,0 +1,73 @@
+/**
+ * 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.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class NNHAStatusHeartbeat implements Writable {
+
+  private State state;
+  private long txid = HdfsConstants.INVALID_TXID;
+  
+  public NNHAStatusHeartbeat() {
+  }
+  
+  public NNHAStatusHeartbeat(State state, long txid) {
+    this.state = state;
+    this.txid = txid;
+  }
+
+  public State getState() {
+    return state;
+  }
+  
+  public long getTxId() {
+    return txid;
+  }
+  
+  ///////////////////////////////////////////
+  // Writable
+  ///////////////////////////////////////////
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeEnum(out, state);
+    out.writeLong(txid);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    state = WritableUtils.readEnum(in, State.class);
+    txid = in.readLong();
+  }
+
+  @InterfaceAudience.Private
+  public enum State {
+    ACTIVE,
+    STANDBY;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
index 59b279c..a75308a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
@@ -101,10 +101,7 @@
    * call fails if the file system is in SafeMode.
    * @throws IOException
    * @return a unique token to identify this transaction.
-   * @deprecated 
-   *    See {@link org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode}
    */
-  @Deprecated
   public CheckpointSignature rollEditLog() throws IOException;
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
index 4de386f..e05b8fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.server.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.RefreshUserMappingsProtocol;
@@ -32,5 +33,6 @@
           NamenodeProtocol,
           RefreshAuthorizationPolicyProtocol,
           RefreshUserMappingsProtocol,
-          GetUserMappingsProtocol {
+          GetUserMappingsProtocol,
+          HAServiceProtocol {
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java
index 45014ad..bde5a5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ReceivedDeletedBlockInfo.java
@@ -25,22 +25,47 @@
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
 
 /**
- * A data structure to store Block and delHints together, used to send
- * received/deleted ACKs.
+ * A data structure to store the blocks in an incremental block report. 
  */
 public class ReceivedDeletedBlockInfo implements Writable {
   Block block;
+  BlockStatus status;
   String delHints;
 
-  public final static String TODELETE_HINT = "-";
+  public static enum BlockStatus {
+    RECEIVING_BLOCK(1),
+    RECEIVED_BLOCK(2),
+    DELETED_BLOCK(3);
+    
+    private final int code;
+    BlockStatus(int code) {
+      this.code = code;
+    }
+    
+    public int getCode() {
+      return code;
+    }
+    
+    public static BlockStatus fromCode(int code) {
+      for (BlockStatus bs : BlockStatus.values()) {
+        if (bs.code == code) {
+          return bs;
+        }
+      }
+      return null;
+    }
+  }
 
   public ReceivedDeletedBlockInfo() {
   }
 
-  public ReceivedDeletedBlockInfo(Block blk, String delHints) {
+  public ReceivedDeletedBlockInfo(
+      Block blk, BlockStatus status, String delHints) {
     this.block = blk;
+    this.status = status;
     this.delHints = delHints;
   }
 
@@ -60,13 +85,19 @@
     this.delHints = hints;
   }
 
+  public BlockStatus getStatus() {
+    return status;
+  }
+
   public boolean equals(Object o) {
     if (!(o instanceof ReceivedDeletedBlockInfo)) {
       return false;
     }
     ReceivedDeletedBlockInfo other = (ReceivedDeletedBlockInfo) o;
     return this.block.equals(other.getBlock())
-        && this.delHints.equals(other.delHints);
+        && this.status == other.status
+        && (this.delHints == other.delHints ||
+            this.delHints != null && this.delHints.equals(other.delHints));
   }
 
   public int hashCode() {
@@ -79,23 +110,30 @@
   }
 
   public boolean isDeletedBlock() {
-    return delHints.equals(TODELETE_HINT);
+    return status == BlockStatus.DELETED_BLOCK;
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     this.block.write(out);
-    Text.writeString(out, this.delHints);
+    WritableUtils.writeVInt(out, this.status.code);
+    if (this.status == BlockStatus.DELETED_BLOCK) {
+      Text.writeString(out, this.delHints);
+    }
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
     this.block = new Block();
     this.block.readFields(in);
-    this.delHints = Text.readString(in);
+    this.status = BlockStatus.fromCode(WritableUtils.readVInt(in));
+    if (this.status == BlockStatus.DELETED_BLOCK) {
+      this.delHints = Text.readString(in);
+    }
   }
 
   public String toString() {
-    return block.toString() + ", delHint: " + delHints;
+    return block.toString() + ", status: " + status +
+      ", delHint: " + delHints;
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 1025880..edbbb22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -38,19 +38,20 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 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.UserGroupInformation;
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -791,9 +792,9 @@
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
 
     // Create the client
-    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB refreshProtocol = 
-        new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
-        NameNode.getAddress(conf), getUGI(), conf);
+    RefreshAuthorizationPolicyProtocol refreshProtocol =
+        NameNodeProxies.createProxy(conf, FileSystem.getDefaultUri(conf),
+            RefreshAuthorizationPolicyProtocol.class).getProxy();
     
     // Refresh the authorization policy in-effect
     refreshProtocol.refreshServiceAcl();
@@ -817,9 +818,9 @@
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
  
     // Create the client
-    RefreshUserMappingsProtocolClientSideTranslatorPB refreshProtocol = 
-        new RefreshUserMappingsProtocolClientSideTranslatorPB(
-        NameNode.getAddress(conf), getUGI(), conf);
+    RefreshUserMappingsProtocol refreshProtocol =
+      NameNodeProxies.createProxy(conf, FileSystem.getDefaultUri(conf),
+          RefreshUserMappingsProtocol.class).getProxy();
 
     // Refresh the user-to-groups mappings
     refreshProtocol.refreshUserToGroupsMappings();
@@ -844,9 +845,9 @@
         conf.get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, ""));
 
     // Create the client
-    RefreshUserMappingsProtocolClientSideTranslatorPB refreshProtocol = 
-        new RefreshUserMappingsProtocolClientSideTranslatorPB(
-        NameNode.getAddress(conf), getUGI(), conf);
+    RefreshUserMappingsProtocol refreshProtocol =
+      NameNodeProxies.createProxy(conf, FileSystem.getDefaultUri(conf),
+          RefreshUserMappingsProtocol.class).getProxy();
 
     // Refresh the user-to-groups mappings
     refreshProtocol.refreshSuperUserGroupsConfiguration();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSHAAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSHAAdmin.java
new file mode 100644
index 0000000..13bde2a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSHAAdmin.java
@@ -0,0 +1,116 @@
+/**
+ * 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.tools;
+
+import java.io.PrintStream;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.HAAdmin;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Class to extend HAAdmin to do a little bit of HDFS-specific configuration.
+ */
+public class DFSHAAdmin extends HAAdmin {
+
+  private static final Log LOG = LogFactory.getLog(DFSHAAdmin.class);
+
+  private String nameserviceId;
+
+  protected void setErrOut(PrintStream errOut) {
+    this.errOut = errOut;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    if (conf != null) {
+      // Make a copy so we don't mutate it. Also use an HdfsConfiguration to
+      // force loading of hdfs-site.xml.
+      conf = new HdfsConfiguration(conf);
+      String nameNodePrincipal = conf.get(
+          DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY, "");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Using NN principal: " + nameNodePrincipal);
+      }
+
+      conf.set(CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
+          nameNodePrincipal);
+    }
+    super.setConf(conf);
+  }
+
+  /**
+   * Try to map the given namenode ID to its service address.
+   */
+  @Override
+  protected String getServiceAddr(String nnId) {
+    HdfsConfiguration conf = (HdfsConfiguration)getConf();
+    String serviceAddr = 
+      DFSUtil.getNamenodeServiceAddr(conf, nameserviceId, nnId);
+    if (serviceAddr == null) {
+      throw new IllegalArgumentException(
+          "Unable to determine service address for namenode '" + nnId + "'");
+    }
+    return serviceAddr;
+  }
+
+  @Override
+  protected String getUsageString() {
+    return "Usage: DFSHAAdmin [-ns <nameserviceId>]";
+  }
+
+  @Override
+  protected int runCmd(String[] argv) throws Exception {
+    if (argv.length < 1) {
+      printUsage(errOut);
+      return -1;
+    }
+
+    int i = 0;
+    String cmd = argv[i++];
+
+    if ("-ns".equals(cmd)) {
+      if (i == argv.length) {
+        errOut.println("Missing nameservice ID");
+        printUsage(errOut);
+        return -1;
+      }
+      nameserviceId = argv[i++];
+      if (i >= argv.length) {
+        errOut.println("Missing command");
+        printUsage(errOut);
+        return -1;
+      }
+      argv = Arrays.copyOfRange(argv, i, argv.length);
+    }
+    
+    return super.runCmd(argv);
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int res = ToolRunner.run(new DFSHAAdmin(), argv);
+    System.exit(res);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index bc98995..1a99fcb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -32,11 +32,13 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.Krb5AndCertsSslSocketConnector;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -204,8 +206,9 @@
    * Derive the namenode http address from the current file system,
    * either default or as set by "-fs" in the generic options.
    * @return Returns http address or null if failure.
+   * @throws IOException if we can't determine the active NN address
    */
-  private String getCurrentNamenodeAddress() {
+  private String getCurrentNamenodeAddress() throws IOException {
     //String nnAddress = null;
     Configuration conf = getConf();
 
@@ -222,16 +225,21 @@
       System.err.println("FileSystem is " + fs.getUri());
       return null;
     }
-    DistributedFileSystem dfs = (DistributedFileSystem) fs;
-
-    // Derive the nameservice ID from the filesystem URI.
-    // The URI may have been provided by a human, and the server name may be
-    // aliased, so compare InetSocketAddresses instead of URI strings, and
-    // test against both possible variants of RPC address.
-    InetSocketAddress namenode = 
-      NameNode.getAddress(dfs.getUri().getAuthority());
     
-    return DFSUtil.getInfoServer(namenode, conf, true);
+    // force client address resolution.
+    fs.exists(new Path("/"));
+    
+    // Derive the nameservice ID from the filesystem connection. The URI may
+    // have been provided by a human, the server name may be aliased, or there
+    // may be multiple possible actual addresses (e.g. in an HA setup) so
+    // compare InetSocketAddresses instead of URI strings, and test against both
+    // possible configurations of RPC address (DFS_NAMENODE_RPC_ADDRESS_KEY and
+    // DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY).
+    DistributedFileSystem dfs = (DistributedFileSystem) fs;
+    DFSClient dfsClient = dfs.getClient();
+    InetSocketAddress addr = RPC.getServerAddress(dfsClient.getNamenode());
+    
+    return DFSUtil.getInfoServer(addr, conf, true);
   }
 
   private int doWork(final String[] args) throws IOException {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
index 0c751773..e3a67ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
@@ -29,6 +29,7 @@
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -164,7 +165,7 @@
   static class NameNodesCommandHandler extends CommandHandler {
     @Override
     int doWorkInternal(GetConf tool) throws IOException {
-      tool.printList(DFSUtil.getNNServiceRpcAddresses(tool.getConf()));
+      tool.printMap(DFSUtil.getNNServiceRpcAddresses(tool.getConf()));
       return 0;
     }
   }
@@ -175,7 +176,7 @@
   static class BackupNodesCommandHandler extends CommandHandler {
     @Override
     public int doWorkInternal(GetConf tool) throws IOException {
-      tool.printList(DFSUtil.getBackupNodeAddresses(tool.getConf()));
+      tool.printMap(DFSUtil.getBackupNodeAddresses(tool.getConf()));
       return 0;
     }
   }
@@ -186,7 +187,7 @@
   static class SecondaryNameNodesCommandHandler extends CommandHandler {
     @Override
     public int doWorkInternal(GetConf tool) throws IOException {
-      tool.printList(DFSUtil.getSecondaryNameNodeAddresses(tool.getConf()));
+      tool.printMap(DFSUtil.getSecondaryNameNodeAddresses(tool.getConf()));
       return 0;
     }
   }
@@ -200,9 +201,11 @@
     @Override
     public int doWorkInternal(GetConf tool) throws IOException {
       Configuration config = tool.getConf();
-      List<InetSocketAddress> rpclist = DFSUtil.getNNServiceRpcAddresses(config);
-      if (rpclist != null) {
-        for (InetSocketAddress rpc : rpclist) {
+      List<ConfiguredNNAddress> cnnlist = DFSUtil.flattenAddressMap(
+          DFSUtil.getNNServiceRpcAddresses(config));
+      if (!cnnlist.isEmpty()) {
+        for (ConfiguredNNAddress cnn : cnnlist) {
+          InetSocketAddress rpc = cnn.getAddress();
           tool.printOut(rpc.getHostName()+":"+rpc.getPort());
         }
         return 0;
@@ -232,10 +235,13 @@
   void printOut(String message) {
     out.println(message);
   }
-
-  void printList(List<InetSocketAddress> list) {
+  
+  void printMap(Map<String, Map<String, InetSocketAddress>> map) {
     StringBuilder buffer = new StringBuilder();
-    for (InetSocketAddress address : list) {
+
+    List<ConfiguredNNAddress> cnns = DFSUtil.flattenAddressMap(map);
+    for (ConfiguredNNAddress cnn : cnns) {
+      InetSocketAddress address = cnn.getAddress();
       if (buffer.length() > 0) {
         buffer.append(" ");
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
index 5ad227d..51612be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetGroups.java
@@ -21,8 +21,11 @@
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -34,6 +37,7 @@
  * HDFS implementation of a tool for getting the groups which a given user
  * belongs to.
  */
+@InterfaceAudience.Private
 public class GetGroups extends GetGroupsBase {
 
   static{
@@ -41,11 +45,11 @@
   }
 
   
-  GetGroups(Configuration conf) {
+  public GetGroups(Configuration conf) {
     super(conf);
   }
 
-  GetGroups(Configuration conf, PrintStream out) {
+  public GetGroups(Configuration conf, PrintStream out) {
     super(conf, out);
   }
   
@@ -57,9 +61,8 @@
   
   @Override
   protected GetUserMappingsProtocol getUgmProtocol() throws IOException {
-    return new GetUserMappingsProtocolClientSideTranslatorPB(
-        NameNode.getAddress(getConf()), UserGroupInformation.getCurrentUser(),
-        getConf());
+    return NameNodeProxies.createProxy(getConf(), FileSystem.getDefaultUri(getConf()),
+        GetUserMappingsProtocol.class).getProxy();
   }
 
   public static void main(String[] argv) throws Exception {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java
index a010830..3a460e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsElement.java
@@ -48,6 +48,8 @@
       BLOCK_ID,
       BLOCK_NUM_BYTES,
       BLOCK_GENERATION_STAMP,
+      BLOCK_DELTA_NUM_BYTES, // delta-encoded relative to previous block
+      BLOCK_DELTA_GEN_STAMP, // delta-encoded relative to previous block
     PERMISSION_STATUS,
       FS_PERMISSIONS,
     CLIENT_NAME,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
index f1da4c6..d34bff9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
@@ -41,7 +41,7 @@
 class EditsLoaderCurrent implements EditsLoader {
 
   private static int[] supportedVersions = { -18, -19, -20, -21, -22, -23, -24,
-      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39};
+      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39, -40};
 
   private EditsVisitor v;
   private int editsVersion = 0;
@@ -150,6 +150,25 @@
     }
   }
 
+  private void visit_OP_UPDATE_BLOCKS() throws IOException {
+    visitTxId();
+    v.visitStringUTF8(EditsElement.PATH);
+    VIntToken numBlocksToken = v.visitVInt(EditsElement.NUMBLOCKS);
+    for (int i = 0; i < numBlocksToken.value; i++) {
+      v.visitEnclosingElement(EditsElement.BLOCK);
+
+      v.visitLong(EditsElement.BLOCK_ID);
+      if (i == 0) {
+        v.visitVLong(EditsElement.BLOCK_NUM_BYTES);
+        v.visitVLong(EditsElement.BLOCK_GENERATION_STAMP);
+      } else {
+        v.visitVLong(EditsElement.BLOCK_DELTA_NUM_BYTES);
+        v.visitVLong(EditsElement.BLOCK_DELTA_GEN_STAMP);
+      }
+      v.leaveEnclosingElement();
+    }
+  }
+  
   /**
    * Visit OP_RENAME_OLD
    */
@@ -521,6 +540,9 @@
       case OP_START_LOG_SEGMENT: // 24
         visit_OP_BEGIN_LOG_SEGMENT();
         break;
+      case OP_UPDATE_BLOCKS: // 25
+        visit_OP_UPDATE_BLOCKS();
+        break;
       default:
       {
         throw new IOException("Unknown op code " + editsOpCode);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
index 8960cbc..fdc9892 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
@@ -122,7 +122,8 @@
   protected final DateFormat dateFormat = 
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
-      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39};
+      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
+      -40};
   private int imageVersion = 0;
 
   /* (non-Javadoc)
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
index cd88963..8a0f992 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
@@ -30,6 +30,9 @@
 
   /** Acquire write lock. */
   public void writeLock();
+  
+  /** Acquire write lock, unless interrupted while waiting  */
+  void writeLockInterruptibly() throws InterruptedException;
 
   /** Release write lock. */
   public void writeUnlock();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 81ca74b..2a96544 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -189,10 +189,25 @@
 }
 
 /**
+ * state - State the NN is in when returning response to the DN
+ * txid - Highest transaction ID this NN has seen
+ */
+message NNHAStatusHeartbeatProto {
+  enum State {
+    ACTIVE = 0;
+    STANDBY = 1;
+  }
+  required State state = 1; 
+  required uint64 txid = 2;
+}
+
+/**
  * cmds - Commands from namenode to datanode.
+ * haStatus - Status (from an HA perspective) of the NN sending this response
  */
 message HeartbeatResponseProto {
   repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
+  required NNHAStatusHeartbeatProto haStatus = 2;
 }
 
 /**
@@ -226,12 +241,16 @@
 /**
  * Data structure to send received or deleted block information
  * from datanode to namenode.
- *
- * deleteHint set to "-" indicates block deletion.
- * other deleteHint indicates block addition.
  */
 message ReceivedDeletedBlockInfoProto {
+  enum BlockStatus {
+    RECEIVING = 1; // block being created
+    RECEIVED = 2; // block creation complete
+    DELETED = 3;
+  }
+
   required BlockProto block = 1;
+  required BlockStatus status = 3;
   optional string deleteHint = 2;
 }
 
@@ -350,7 +369,9 @@
   rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
 
   /**
-   * Report from datanode about recently received or deleted block
+   * Incremental block report from the DN. This contains info about recently
+   * received and deleted blocks, as well as when blocks start being
+   * received.
    */
   rpc blockReceivedAndDeleted(BlockReceivedAndDeletedRequestProto) 
       returns(BlockReceivedAndDeletedResponseProto);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 1ce090b..9ec25d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -33,9 +33,11 @@
 <property>
   <name>dfs.namenode.logging.level</name>
   <value>info</value>
-  <description>The logging level for dfs namenode. Other values are "dir"(trac
-e namespace mutations), "block"(trace block under/over replications and block
-creations/deletions), or "all".</description>
+  <description>
+    The logging level for dfs namenode. Other values are "dir" (trace
+    namespace mutations), "block" (trace block under/over replications
+    and block creations/deletions), or "all".
+  </description>
 </property>
 
 <property>
@@ -226,6 +228,18 @@
       directories, for redundancy. Default value is same as dfs.namenode.name.dir
   </description>
 </property>
+
+<property>
+  <name>dfs.namenode.shared.edits.dir</name>
+  <value></value>
+  <description>A directory on shared storage between the multiple namenodes
+  in an HA cluster. This directory will be written by the active and read
+  by the standby in order to keep the namespaces synchronized. This directory
+  does not need to be listed in dfs.namenode.edits.dir above. It should be
+  left empty in a non-HA cluster.
+  </description>
+</property>
+  
 <property>
   <name>dfs.web.ugi</name>
   <value>webuser,webgroup</value>
@@ -624,6 +638,19 @@
 </property>
 
 <property>
+  <name>dfs.namenode.num.extra.edits.retained</name>
+  <value>1000000</value>
+  <description>The number of extra transactions which should be retained
+  beyond what is minimally necessary for a NN restart. This can be useful for
+  audit purposes or for an HA setup where a remote Standby Node may have
+  been offline for some time and need to have a longer backlog of retained
+  edits in order to start again.
+  Typically each edit is on the order of a few hundred bytes, so the default
+  of 1 million edits should be on the order of hundreds of MBs or low GBs.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.delegation.key.update-interval</name>
   <value>86400000</value>
   <description>The update interval for master key for delegation tokens 
@@ -698,4 +725,118 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.client.failover.max.attempts</name>
+  <value>15</value>
+  <description>
+    Expert only. The number of client failover attempts that should be
+    made before the failover is considered failed.
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.failover.sleep.base.millis</name>
+  <value>500</value>
+  <description>
+    Expert only. The time to wait, in milliseconds, between failover
+    attempts increases exponentially as a function of the number of
+    attempts made so far, with a random factor of +/- 50%. This option
+    specifies the base value used in the failover calculation. The
+    first failover will retry immediately. The 2nd failover attempt
+    will delay at least dfs.client.failover.sleep.base.millis
+    milliseconds. And so on.
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.failover.sleep.max.millis</name>
+  <value>15000</value>
+  <description>
+    Expert only. The time to wait, in milliseconds, between failover
+    attempts increases exponentially as a function of the number of
+    attempts made so far, with a random factor of +/- 50%. This option
+    specifies the maximum value to wait between failovers. 
+    Specifically, the time between two failover attempts will not
+    exceed +/- 50% of dfs.client.failover.sleep.max.millis
+    milliseconds.
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.failover.connection.retries</name>
+  <value>0</value>
+  <description>
+    Expert only. Indicates the number of retries a failover IPC client
+    will make to establish a server connection.
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.failover.connection.retries.on.timeouts</name>
+  <value>0</value>
+  <description>
+    Expert only. The number of retry attempts a failover IPC client
+    will make on socket timeout when establishing a server connection.
+  </description>
+</property>
+
+<property>
+  <name>dfs.federation.nameservices</name>
+  <value></value>
+  <description>
+    Comma-separated list of nameservices.
+  </description>
+</property>
+
+<property>
+  <name>dfs.federation.nameservice.id</name>
+  <value></value>
+  <description>
+    The ID of this nameservice. If the nameservice ID is not
+    configured or more than one nameservice is configured for
+    dfs.federation.nameservices it is determined automatically by
+    matching the local node's address with the configured address.
+  </description>
+</property>
+
+<property>
+  <name>dfs.ha.namenodes.EXAMPLENAMESERVICE</name>
+  <value></value>
+  <description>
+    The prefix for a given nameservice, contains a comma-separated
+    list of namenodes for a given nameservice (eg EXAMPLENAMESERVICE).
+  </description>
+</property>
+
+<property>
+  <name>dfs.ha.namenode.id</name>
+  <value></value>
+  <description>
+    The ID of this namenode. If the namenode ID is not configured it
+    is determined automatically by matching the local node's address
+    with the configured address.
+  </description>
+</property>
+
+<property>
+  <name>dfs.ha.log-roll.period</name>
+  <value>120</value>
+  <description>
+    How often, in seconds, the StandbyNode should ask the active to
+    roll edit logs. Since the StandbyNode only reads from finalized
+    log segments, the StandbyNode will only be as up-to-date as how
+    often the logs are rolled. Note that failover triggers a log roll
+    so the StandbyNode will be up to date before it becomes active.
+  </description>
+</property>
+
+<property>
+  <name>dfs.ha.tail-edits.period</name>
+  <value>60</value>
+  <description>
+    How often, in seconds, the StandbyNode should check for new
+    finalized log segments in the shared edits log.
+  </description>
+</property>
+
 </configuration>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp
index a71f40f..a4906a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/corrupt_files.jsp
@@ -23,6 +23,7 @@
 	import="org.apache.hadoop.fs.FileStatus"
 	import="org.apache.hadoop.fs.FileUtil"
 	import="org.apache.hadoop.fs.Path"
+	import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
 	import="java.util.Collection"
 	import="java.util.Arrays" %>
 <%!//for java.io.Serializable
@@ -30,6 +31,8 @@
 <%
   NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
   FSNamesystem fsn = nn.getNamesystem();
+  HAServiceState nnHAState = nn.getServiceState();
+  boolean isActive = (nnHAState == HAServiceState.ACTIVE);
   String namenodeRole = nn.getRole().toString();
   String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":"
       + nn.getNameNodeAddress().getPort();
@@ -46,8 +49,10 @@
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
 <%=NamenodeJspHelper.getVersionTable(fsn)%>
 <br>
-<b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b>
-<br>
+<% if (isActive) { %> 
+  <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b>
+  <br>
+<% } %> 
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
 <br>
 <b><a href=/dfshealth.jsp> Go back to DFS home</a></b>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
index ecce30a..81e595d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp
@@ -20,6 +20,7 @@
 <%@ page
   contentType="text/html; charset=UTF-8"
   import="org.apache.hadoop.util.ServletUtil"
+  import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
 %>
 <%!
   //for java.io.Serializable
@@ -29,7 +30,10 @@
   final NamenodeJspHelper.HealthJsp healthjsp  = new NamenodeJspHelper.HealthJsp();
   NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
   FSNamesystem fsn = nn.getNamesystem();
+  HAServiceState nnHAState = nn.getServiceState();
+  boolean isActive = (nnHAState == HAServiceState.ACTIVE);
   String namenodeRole = nn.getRole().toString();
+  String namenodeState = nnHAState.toString();
   String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameNodeAddress().getPort();
 %>
 
@@ -40,10 +44,12 @@
 <title>Hadoop <%=namenodeRole%>&nbsp;<%=namenodeLabel%></title>
     
 <body>
-<h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
+<h1><%=namenodeRole%> '<%=namenodeLabel%>' (<%=namenodeState%>)</h1>
 <%= NamenodeJspHelper.getVersionTable(fsn) %>
 <br />
-<b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
+<% if (isActive) { %> 
+  <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
+<% } %> 
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
 
 <hr>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp
index 886fbea..35deb05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfsnodelist.jsp
@@ -20,6 +20,7 @@
 <%@ page
   contentType="text/html; charset=UTF-8"
   import="org.apache.hadoop.util.ServletUtil"
+  import="org.apache.hadoop.ha.HAServiceProtocol.HAServiceState"
 %>
 <%!
   //for java.io.Serializable
@@ -30,6 +31,8 @@
 NameNode nn = NameNodeHttpServer.getNameNodeFromContext(application);
 String namenodeRole = nn.getRole().toString();
 FSNamesystem fsn = nn.getNamesystem();
+HAServiceState nnHAState = nn.getServiceState();
+boolean isActive = (nnHAState == HAServiceState.ACTIVE);
 String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameNodeAddress().getPort();
 %>
 
@@ -43,7 +46,9 @@
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
 <%= NamenodeJspHelper.getVersionTable(fsn) %>
 <br />
-<b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
+<% if (isActive) { %> 
+  <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
+<% } %> 
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b><br>
 <b><a href=/dfshealth.jsp> Go back to DFS home</a></b>
 <hr>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java
index 17608ac..1d5def6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java
@@ -23,6 +23,7 @@
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -47,9 +48,11 @@
   @BeforeClass
   public static void setUp() throws IOException {
     Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
-    NameNodeAdapter.getDtSecretManager(cluster.getNamesystem()).startThreads();
+
   }
 
   @AfterClass
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
index 83115ed..7ad56c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
@@ -27,8 +27,9 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -51,12 +52,15 @@
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
+    CONF.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    
     cluster =
-        new MiniDFSCluster.Builder(CONF).numNameNodes(2).numDataNodes(2)
+        new MiniDFSCluster.Builder(CONF).nnTopology(
+                MiniDFSNNTopology.simpleFederatedTopology(2))
+            .numDataNodes(2)
             .build();
     cluster.waitClusterUp();
-    NameNodeAdapter.getDtSecretManager(cluster.getNamesystem(0)).startThreads();
-    NameNodeAdapter.getDtSecretManager(cluster.getNamesystem(1)).startThreads();
     
     fHdfs = cluster.getFileSystem(0);
     fHdfs2 = cluster.getFileSystem(1);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java
index 4a60556..0e94b4e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsHdfs.java
@@ -26,9 +26,9 @@
 
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
@@ -51,9 +51,11 @@
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
     SupportsBlocks = true;
+    CONF.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     cluster.waitClusterUp();
-    NameNodeAdapter.getDtSecretManager(cluster.getNamesystem()).startThreads();
     fc = FileContext.getFileContext(cluster.getURI(0), CONF);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
index 50a34a8..f286481 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
@@ -109,12 +109,18 @@
     out.write(bytes);
   }
   
-  static void check(FileSystem fs, Path p, long length) throws IOException {
+  public static void check(FileSystem fs, Path p, long length) throws IOException {
     int i = -1;
     try {
       final FileStatus status = fs.getFileStatus(p);
-      TestCase.assertEquals(length, status.getLen());
-      InputStream in = fs.open(p);
+      FSDataInputStream in = fs.open(p);
+      if (in.getWrappedStream() instanceof DFSInputStream) {
+        long len = ((DFSInputStream)in.getWrappedStream()).getFileLength();
+        TestCase.assertEquals(length, len);
+      } else {
+        TestCase.assertEquals(length, status.getLen());
+      }
+      
       for(i++; i < length; i++) {
         TestCase.assertEquals((byte)i, (byte)in.read());  
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 409dd37..7854f95 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.junit.Assert.assertEquals;
 
 import java.io.BufferedOutputStream;
@@ -38,9 +40,11 @@
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -52,6 +56,7 @@
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
+import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -74,6 +79,8 @@
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
+import com.google.common.base.Joiner;
+
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
   
@@ -681,4 +688,21 @@
 
     return BlockOpResponseProto.parseDelimitedFrom(in);
   }
+  
+  public static void setFederatedConfiguration(MiniDFSCluster cluster,
+      Configuration conf) {
+    Set<String> nameservices = new HashSet<String>();
+    for (NameNodeInfo info : cluster.getNameNodeInfos()) {
+      assert info.nameserviceId != null;
+      nameservices.add(info.nameserviceId);
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
+          info.nameserviceId), DFSUtil.createUri(HdfsConstants.HDFS_URI_SCHEME,
+              info.nameNode.getNameNodeAddress()).toString());
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+          info.nameserviceId), DFSUtil.createUri(HdfsConstants.HDFS_URI_SCHEME,
+              info.nameNode.getNameNodeAddress()).toString());
+    }
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, Joiner.on(",")
+        .join(nameservices));
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index c3cc6bb..8888bec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -32,6 +32,7 @@
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -41,15 +42,25 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocolHelper;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface;
@@ -60,6 +71,7 @@
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
@@ -69,6 +81,11 @@
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+
 /**
  * This class creates a single-process DFS cluster for junit testing.
  * The data directories for non-simulated DFS are under the testing directory.
@@ -94,7 +111,6 @@
     private int nameNodePort = 0;
     private int nameNodeHttpPort = 0;
     private final Configuration conf;
-    private int numNameNodes = 1;
     private int numDataNodes = 1;
     private boolean format = true;
     private boolean manageNameDfsDirs = true;
@@ -106,22 +122,13 @@
     private String clusterId = null;
     private boolean waitSafeMode = true;
     private boolean setupHostsFile = false;
-    private boolean federation = false;
+    private MiniDFSNNTopology nnTopology = null;
     
     public Builder(Configuration conf) {
       this.conf = conf;
     }
     
     /**
-     * default false - non federated cluster
-     * @param val
-     * @return Builder object
-     */
-    public Builder federation (boolean val){
-      this.federation = val;
-      return this;
-    }
-    /**
      * Default: 0
      */
     public Builder nameNodePort(int val) {
@@ -140,14 +147,6 @@
     /**
      * Default: 1
      */
-    public Builder numNameNodes(int val) {
-      this.numNameNodes = val;
-      return this;
-    }
-
-    /**
-     * Default: 1
-     */
     public Builder numDataNodes(int val) {
       this.numDataNodes = val;
       return this;
@@ -235,6 +234,16 @@
     }
     
     /**
+     * Default: a single namenode.
+     * See {@link MiniDFSNNTopology#simpleFederatedTopology(int)} to set up
+     * federated nameservices
+     */
+    public Builder nnTopology(MiniDFSNNTopology topology) {
+      this.nnTopology = topology;
+      return this;
+    }
+    
+    /**
      * Construct the actual MiniDFSCluster
      */
     public MiniDFSCluster build() throws IOException {
@@ -246,15 +255,17 @@
    * Used by builder to create and return an instance of MiniDFSCluster
    */
   private MiniDFSCluster(Builder builder) throws IOException {
-    LOG.info("starting cluster with " + builder.numNameNodes + " namenodes.");
-    nameNodes = new NameNodeInfo[builder.numNameNodes];
-    // try to determine if in federation mode
-    if(builder.numNameNodes > 1)
-      builder.federation = true;
+    if (builder.nnTopology == null) {
+      // If no topology is specified, build a single NN. 
+      builder.nnTopology = MiniDFSNNTopology.simpleSingleNN(
+          builder.nameNodePort, builder.nameNodeHttpPort);
+    }
+    
+    LOG.info("starting cluster with " + 
+        builder.nnTopology.countNameNodes() + " namenodes.");
+    nameNodes = new NameNodeInfo[builder.nnTopology.countNameNodes()];
       
-    initMiniDFSCluster(builder.nameNodePort,
-                       builder.nameNodeHttpPort,
-                       builder.conf,
+    initMiniDFSCluster(builder.conf,
                        builder.numDataNodes,
                        builder.format,
                        builder.manageNameDfsDirs,
@@ -266,7 +277,7 @@
                        builder.clusterId,
                        builder.waitSafeMode,
                        builder.setupHostsFile,
-                       builder.federation);
+                       builder.nnTopology);
   }
   
   public class DataNodeProperties {
@@ -288,8 +299,16 @@
                          new ArrayList<DataNodeProperties>();
   private File base_dir;
   private File data_dir;
-  private boolean federation = false; 
   private boolean waitSafeMode = true;
+  private boolean federation;
+  
+  /**
+   * A unique instance identifier for the cluster. This
+   * is used to disambiguate HA filesystems in the case where
+   * multiple MiniDFSClusters are used in the same test suite. 
+   */
+  private int instanceId;
+  private static int instanceCount = 0;
   
   /**
    * Stores the information related to a namenode in the cluster
@@ -297,8 +316,13 @@
   static class NameNodeInfo {
     final NameNode nameNode;
     final Configuration conf;
-    NameNodeInfo(NameNode nn, Configuration conf) {
+    final String nameserviceId;
+    final String nnId;
+    NameNodeInfo(NameNode nn, String nameserviceId, String nnId,
+        Configuration conf) {
       this.nameNode = nn;
+      this.nameserviceId = nameserviceId;
+      this.nnId = nnId;
       this.conf = conf;
     }
   }
@@ -309,6 +333,9 @@
    */
   public MiniDFSCluster() {
     nameNodes = new NameNodeInfo[0]; // No namenode in the cluster
+    synchronized (MiniDFSCluster.class) {
+      instanceId = instanceCount++;
+    }
   }
   
   /**
@@ -480,22 +507,27 @@
                         String[] racks, String hosts[],
                         long[] simulatedCapacities) throws IOException {
     this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
-    initMiniDFSCluster(nameNodePort, 0, conf, numDataNodes, format,
+    initMiniDFSCluster(conf, numDataNodes, format,
         manageNameDfsDirs, manageDataDfsDirs, operation, racks, hosts,
-        simulatedCapacities, null, true, false, false);
+        simulatedCapacities, null, true, false,
+        MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0));
   }
 
-  private void initMiniDFSCluster(int nameNodePort, int nameNodeHttpPort,
+  private void initMiniDFSCluster(
       Configuration conf,
       int numDataNodes, boolean format, boolean manageNameDfsDirs,
       boolean manageDataDfsDirs, StartupOption operation, String[] racks,
       String[] hosts, long[] simulatedCapacities, String clusterId,
-      boolean waitSafeMode, boolean setupHostsFile, boolean federation) 
+      boolean waitSafeMode, boolean setupHostsFile,
+      MiniDFSNNTopology nnTopology)
   throws IOException {
+    synchronized (MiniDFSCluster.class) {
+      instanceId = instanceCount++;
+    }
+
     this.conf = conf;
     base_dir = new File(determineDfsBaseDir());
     data_dir = new File(base_dir, "data");
-    this.federation = federation;
     this.waitSafeMode = waitSafeMode;
     
     int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
@@ -505,28 +537,25 @@
     conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
                    StaticMapping.class, DNSToSwitchMapping.class);
     
-    Collection<String> nameserviceIds = DFSUtil.getNameServiceIds(conf);
-    if(nameserviceIds.size() > 1)  
-      federation = true;
-  
-    if (!federation) {
-      conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
-      conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
-          + nameNodeHttpPort);
-      NameNode nn = createNameNode(0, conf, numDataNodes, manageNameDfsDirs,
-          format, operation, clusterId);
-      nameNodes[0] = new NameNodeInfo(nn, conf);
-      FileSystem.setDefaultUri(conf, getURI(0));
-    } else {
-      if (nameserviceIds.isEmpty()) {
-        for (int i = 0; i < nameNodes.length; i++) {
-          nameserviceIds.add(NAMESERVICE_ID_PREFIX + i);
-        }
-      }
-      initFederationConf(conf, nameserviceIds, numDataNodes, nameNodePort);
-      createFederationNamenodes(conf, nameserviceIds, manageNameDfsDirs, format,
-          operation, clusterId);
+    // In an HA cluster, in order for the StandbyNode to perform checkpoints,
+    // it needs to know the HTTP port of the Active. So, if ephemeral ports
+    // are chosen, disable checkpoints for the test.
+    if (!nnTopology.allHttpPortsSpecified() &&
+        nnTopology.isHA()) {
+      LOG.info("MiniDFSCluster disabling checkpointing in the Standby node " +
+          "since no HTTP ports have been specified.");
+      conf.setBoolean(DFS_HA_STANDBY_CHECKPOINTS_KEY, false);
     }
+    if (!nnTopology.allIpcPortsSpecified() &&
+        nnTopology.isHA()) {
+      LOG.info("MiniDFSCluster disabling log-roll triggering in the "
+          + "Standby node since no IPC ports have been specified.");
+      conf.setInt(DFS_HA_LOGROLL_PERIOD_KEY, -1);
+    }
+    
+    federation = nnTopology.isFederated();
+    createNameNodesAndSetConf(
+        nnTopology, manageNameDfsDirs, format, operation, clusterId, conf);
     
     if (format) {
       if (data_dir.exists() && !FileUtil.fullyDelete(data_dir)) {
@@ -542,51 +571,130 @@
     ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
   }
   
-  /** Initialize configuration for federated cluster */
-  private static void initFederationConf(Configuration conf,
-      Collection<String> nameserviceIds, int numDataNodes, int nnPort) {
-    String nameserviceIdList = "";
-    for (String nameserviceId : nameserviceIds) {
-      // Create comma separated list of nameserviceIds
-      if (nameserviceIdList.length() > 0) {
-        nameserviceIdList += ",";
+  private void createNameNodesAndSetConf(MiniDFSNNTopology nnTopology,
+      boolean manageNameDfsDirs, boolean format, StartupOption operation,
+      String clusterId,
+      Configuration conf) throws IOException {
+    Preconditions.checkArgument(nnTopology.countNameNodes() > 0,
+        "empty NN topology: no namenodes specified!");
+
+    if (!federation && nnTopology.countNameNodes() == 1) {
+      NNConf onlyNN = nnTopology.getOnlyNameNode();
+      // we only had one NN, set DEFAULT_NAME for it
+      conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + onlyNN.getIpcPort());
+    }
+    
+    List<String> allNsIds = Lists.newArrayList();
+    for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
+      if (nameservice.getId() != null) {
+        allNsIds.add(nameservice.getId());
       }
-      nameserviceIdList += nameserviceId;
-      initFederatedNamenodeAddress(conf, nameserviceId, nnPort);
-      nnPort = nnPort == 0 ? 0 : nnPort + 2;
     }
-    conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
-  }
-
-  /* For federated namenode initialize the address:port */
-  private static void initFederatedNamenodeAddress(Configuration conf,
-      String nameserviceId, int nnPort) {
-    // Set nameserviceId specific key
-    String key = DFSUtil.getNameServiceIdKey(
-        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
-    conf.set(key, "127.0.0.1:0");
-
-    key = DFSUtil.getNameServiceIdKey(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
-    conf.set(key, "127.0.0.1:" + nnPort);
-  }
-  
-  private void createFederationNamenodes(Configuration conf,
-      Collection<String> nameserviceIds, boolean manageNameDfsDirs,
-      boolean format, StartupOption operation, String clusterId)
-      throws IOException {
-    // Create namenodes in the cluster
+    if (!allNsIds.isEmpty()) {
+      conf.set(DFS_FEDERATION_NAMESERVICES, Joiner.on(",").join(allNsIds));
+    }
+    
     int nnCounter = 0;
-    for (String nameserviceId : nameserviceIds) {
-      createFederatedNameNode(nnCounter++, conf, numDataNodes, manageNameDfsDirs,
-          format, operation, clusterId, nameserviceId);
+    for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
+      String nsId = nameservice.getId();
+      
+      Preconditions.checkArgument(
+          !federation || nsId != null,
+          "if there is more than one NS, they must have names");
+
+      // First set up the configuration which all of the NNs
+      // need to have - have to do this a priori before starting
+      // *any* of the NNs, so they know to come up in standby.
+      List<String> nnIds = Lists.newArrayList();
+      // Iterate over the NNs in this nameservice
+      for (NNConf nn : nameservice.getNNs()) {
+        nnIds.add(nn.getNnId());
+
+        initNameNodeAddress(conf, nameservice.getId(), nn);
+      }
+
+      // If HA is enabled on this nameservice, enumerate all the namenodes
+      // in the configuration. Also need to set a shared edits dir
+      if (nnIds.size() > 1) {
+        conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, nameservice.getId()),
+            Joiner.on(",").join(nnIds));
+        if (manageNameDfsDirs) {
+          URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter+nnIds.size()-1); 
+          conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
+        }
+      }
+
+      // Now format first NN and copy the storage directory from that node to the others.
+      int i = 0;
+      Collection<URI> prevNNDirs = null;
+      int nnCounterForFormat = nnCounter;
+      for (NNConf nn : nameservice.getNNs()) {
+        initNameNodeConf(conf, nsId, nn.getNnId(), manageNameDfsDirs,
+            nnCounterForFormat);
+        Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
+        if (format) {
+          for (URI nameDirUri : namespaceDirs) {
+            File nameDir = new File(nameDirUri);
+            if (nameDir.exists() && !FileUtil.fullyDelete(nameDir)) {
+              throw new IOException("Could not fully delete " + nameDir);
+            }
+          }
+        }
+        
+        boolean formatThisOne = format;
+        if (format && i++ > 0) {
+          // Don't format the second NN in an HA setup - that
+          // would result in it having a different clusterID,
+          // block pool ID, etc. Instead, copy the name dirs
+          // from the first one.
+          formatThisOne = false;
+          assert (null != prevNNDirs);
+          copyNameDirs(prevNNDirs, namespaceDirs, conf);
+        }
+        
+        nnCounterForFormat++;
+        if (formatThisOne) {
+          DFSTestUtil.formatNameNode(conf);
+        }
+        prevNNDirs = namespaceDirs;
+      }
+
+      // Start all Namenodes
+      for (NNConf nn : nameservice.getNNs()) {
+        initNameNodeConf(conf, nsId, nn.getNnId(), manageNameDfsDirs, nnCounter);
+        createNameNode(nnCounter++, conf, numDataNodes, false, operation,
+            clusterId, nsId, nn.getNnId());
+      }
+      
     }
+
   }
   
-  private NameNode createNameNode(int nnIndex, Configuration conf,
-      int numDataNodes, boolean manageNameDfsDirs, boolean format,
-      StartupOption operation, String clusterId)
+  public URI getSharedEditsDir(int minNN, int maxNN) throws IOException {
+    return formatSharedEditsDir(base_dir, minNN, maxNN);
+  }
+  
+  public static URI formatSharedEditsDir(File baseDir, int minNN, int maxNN)
       throws IOException {
+    return fileAsURI(new File(baseDir, "shared-edits-" +
+        minNN + "-through-" + maxNN));
+  }
+  
+  public NameNodeInfo[] getNameNodeInfos() {
+    return this.nameNodes;
+  }
+
+  private void initNameNodeConf(Configuration conf,
+      String nameserviceId, String nnId,
+      boolean manageNameDfsDirs, int nnIndex)
+      throws IOException {
+    if (nameserviceId != null) {
+      conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
+    }
+    if (nnId != null) {
+      conf.set(DFS_HA_NAMENODE_ID_KEY, nnId);
+    }
+    
     if (manageNameDfsDirs) {
       conf.set(DFS_NAMENODE_NAME_DIR_KEY,
           fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
@@ -595,7 +703,50 @@
           fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
           fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
     }
-    
+  }
+
+  private void copyNameDirs(Collection<URI> srcDirs, Collection<URI> dstDirs,
+      Configuration dstConf) throws IOException {
+    URI srcDir = Lists.newArrayList(srcDirs).get(0);
+    FileSystem dstFS = FileSystem.getLocal(dstConf).getRaw();
+    for (URI dstDir : dstDirs) {
+      Preconditions.checkArgument(!dstDir.equals(srcDir));
+      File dstDirF = new File(dstDir);
+      if (dstDirF.exists()) {
+        Files.deleteRecursively(dstDirF);
+      }
+      LOG.info("Copying namedir from primary node dir "
+          + srcDir + " to " + dstDir);
+      FileUtil.copy(
+          new File(srcDir),
+          dstFS, new Path(dstDir), false, dstConf);
+    }
+  }
+
+  /**
+   * Initialize the address and port for this NameNode. In the
+   * non-federated case, the nameservice and namenode ID may be
+   * null.
+   */
+  private static void initNameNodeAddress(Configuration conf,
+      String nameserviceId, NNConf nnConf) {
+    // Set NN-specific specific key
+    String key = DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId,
+        nnConf.getNnId());
+    conf.set(key, "127.0.0.1:" + nnConf.getHttpPort());
+
+    key = DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId,
+        nnConf.getNnId());
+    conf.set(key, "127.0.0.1:" + nnConf.getIpcPort());
+  }
+  
+  private void createNameNode(int nnIndex, Configuration conf,
+      int numDataNodes, boolean format, StartupOption operation,
+      String clusterId, String nameserviceId,
+      String nnId)
+      throws IOException {
     // Format and clean out DataNode directories
     if (format) {
       DFSTestUtil.formatNameNode(conf);
@@ -609,25 +760,20 @@
                      operation == StartupOption.FORMAT ||
                      operation == StartupOption.REGULAR) ?
       new String[] {} : new String[] {operation.getName()};
-    return NameNode.createNameNode(args, conf);
-  }
-  
-  private void createFederatedNameNode(int nnIndex, Configuration conf,
-      int numDataNodes, boolean manageNameDfsDirs, boolean format,
-      StartupOption operation, String clusterId, String nameserviceId)
-      throws IOException {
-    conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
-    NameNode nn = createNameNode(nnIndex, conf, numDataNodes, manageNameDfsDirs,
-        format, operation, clusterId);
-    conf.set(DFSUtil.getNameServiceIdKey(
-        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NetUtils
+    NameNode nn =  NameNode.createNameNode(args, conf);
+    
+    // After the NN has started, set back the bound ports into
+    // the conf
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId, nnId), NetUtils
         .getHostPortString(nn.getNameNodeAddress()));
-    conf.set(DFSUtil.getNameServiceIdKey(
-        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NetUtils
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId, nnId), NetUtils
         .getHostPortString(nn.getHttpAddress()));
-    DFSUtil.setGenericConf(conf, nameserviceId, 
+    DFSUtil.setGenericConf(conf, nameserviceId, nnId,
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
+    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
+        new Configuration(conf));
   }
 
   /**
@@ -652,6 +798,10 @@
     }
     return uri;
   }
+  
+  public int getInstanceId() {
+    return instanceId;
+  }
 
   /**
    * @return Configuration of for the given namenode
@@ -1005,7 +1155,14 @@
    */
   public NamenodeProtocols getNameNodeRpc() {
     checkSingleNameNode();
-    return getNameNode(0).getRpcServer();
+    return getNameNodeRpc(0);
+  }
+  
+  /**
+   * Get an instance of the NameNode's RPC handler.
+   */
+  public NamenodeProtocols getNameNodeRpc(int nnIndex) {
+    return getNameNode(nnIndex).getRpcServer();
   }
   
   /**
@@ -1075,6 +1232,7 @@
     LOG.info("Shutting down the Mini HDFS Cluster");
     shutdownDataNodes();
     for (NameNodeInfo nnInfo : nameNodes) {
+      if (nnInfo == null) continue;
       NameNode nameNode = nnInfo.nameNode;
       if (nameNode != null) {
         nameNode.stop();
@@ -1116,7 +1274,16 @@
       nn.stop();
       nn.join();
       Configuration conf = nameNodes[nnIndex].conf;
-      nameNodes[nnIndex] = new NameNodeInfo(null, conf);
+      nameNodes[nnIndex] = new NameNodeInfo(null, null, null, conf);
+    }
+  }
+  
+  /**
+   * Restart all namenodes.
+   */
+  public synchronized void restartNameNodes() throws IOException {
+    for (int i = 0; i < nameNodes.length; i++) {
+      restartNameNode(i);
     }
   }
   
@@ -1150,10 +1317,12 @@
    */
   public synchronized void restartNameNode(int nnIndex, boolean waitActive)
       throws IOException {
+    String nameserviceId = nameNodes[nnIndex].nameserviceId;
+    String nnId = nameNodes[nnIndex].nnId;
     Configuration conf = nameNodes[nnIndex].conf;
     shutdownNameNode(nnIndex);
     NameNode nn = NameNode.createNameNode(new String[] {}, conf);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, conf);
+    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId, conf);
     if (waitActive) {
       waitClusterUp();
       LOG.info("Restarted the namenode");
@@ -1345,17 +1514,11 @@
       return false;
     }
     long[] sizes;
-    try {
-      sizes = nameNode.getRpcServer().getStats();
-    } catch (IOException ioe) {
-      // This method above should never throw.
-      // It only throws IOE since it is exposed via RPC
-      throw (AssertionError)(new AssertionError("Unexpected IOE thrown: "
-          + StringUtils.stringifyException(ioe)).initCause(ioe));
-    }
+    sizes = NameNodeAdapter.getStats(nameNode.getNamesystem());
     boolean isUp = false;
     synchronized (this) {
-      isUp = ((!nameNode.isInSafeMode() || !waitSafeMode) && sizes[0] != 0);
+      isUp = ((!nameNode.isInSafeMode() || !waitSafeMode) &&
+          sizes[ClientProtocol.GET_STATS_CAPACITY_IDX] != 0);
     }
     return isUp;
   }
@@ -1459,9 +1622,48 @@
   /**
    * Get the directories where the namenode stores its edits.
    */
-  public Collection<URI> getNameEditsDirs(int nnIndex) {
+  public Collection<URI> getNameEditsDirs(int nnIndex) throws IOException {
     return FSNamesystem.getNamespaceEditsDirs(nameNodes[nnIndex].conf);
   }
+  
+  private HAServiceProtocol getHaServiceClient(int nnIndex) throws IOException {
+    InetSocketAddress addr = nameNodes[nnIndex].nameNode.getServiceRpcAddress();
+    return new HAServiceProtocolClientSideTranslatorPB(addr, conf);
+  }
+  
+  public void transitionToActive(int nnIndex) throws IOException,
+      ServiceFailedException {
+    HAServiceProtocolHelper.transitionToActive(getHaServiceClient(nnIndex));
+  }
+  
+  public void transitionToStandby(int nnIndex) throws IOException,
+      ServiceFailedException {
+    HAServiceProtocolHelper.transitionToStandby(getHaServiceClient(nnIndex));
+  }
+  
+  
+  public void triggerBlockReports()
+      throws IOException {
+    for (DataNode dn : getDataNodes()) {
+      DataNodeAdapter.triggerBlockReport(dn);
+    }
+  }
+
+
+  public void triggerDeletionReports()
+      throws IOException {
+    for (DataNode dn : getDataNodes()) {
+      DataNodeAdapter.triggerDeletionReport(dn);
+    }
+  }
+
+  public void triggerHeartbeats()
+      throws IOException {
+    for (DataNode dn : getDataNodes()) {
+      DataNodeAdapter.triggerHeartbeat(dn);
+    }
+  }
+
 
   /** Wait until the given namenode gets registration from all the datanodes */
   public void waitActive(int nnIndex) throws IOException {
@@ -1469,6 +1671,7 @@
       return;
     }
     InetSocketAddress addr = nameNodes[nnIndex].nameNode.getServiceRpcAddress();
+    assert addr.getPort() != 0;
     DFSClient client = new DFSClient(addr, conf);
 
     // ensure all datanodes have registered and sent heartbeat to the namenode
@@ -1512,9 +1715,9 @@
     // If a datanode failed to start, then do not wait
     for (DataNodeProperties dn : dataNodes) {
       // the datanode thread communicating with the namenode should be alive
-      if (!dn.datanode.isBPServiceAlive(addr)) {
-        LOG.warn("BPOfferService failed to start in datanode " + dn.datanode
-            + " for namenode at " + addr);
+      if (!dn.datanode.isConnectedToNN(addr)) {
+        LOG.warn("BPOfferService in datanode " + dn.datanode
+            + " failed to connect to namenode at " + addr);
         return false;
       }
     }
@@ -1653,6 +1856,10 @@
   public void setLeasePeriod(long soft, long hard) {
     NameNodeAdapter.setLeasePeriod(getNamesystem(), soft, hard);
   }
+  
+  public void setWaitSafeMode(boolean wait) {
+    this.waitSafeMode = wait;
+  }
 
   /**
    * Returns the current set of datanodes
@@ -1869,7 +2076,7 @@
       throws IOException {
     if(!federation)
       throw new IOException("cannot add namenode to non-federated cluster");
-    
+
     int nnIndex = nameNodes.length;
     int numNameNodes = nameNodes.length + 1;
     NameNodeInfo[] newlist = new NameNodeInfo[numNameNodes];
@@ -1880,10 +2087,13 @@
     String nameserviceIds = conf.get(DFS_FEDERATION_NAMESERVICES);
     nameserviceIds += "," + nameserviceId;
     conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIds);
-    
-    initFederatedNamenodeAddress(conf, nameserviceId, namenodePort);
-    createFederatedNameNode(nnIndex, conf, numDataNodes, true, true, null,
-        null, nameserviceId);
+  
+    String nnId = null;
+    initNameNodeAddress(conf, nameserviceId,
+        new NNConf(nnId).setIpcPort(namenodePort));
+    initNameNodeConf(conf, nameserviceId, nnId, true, nnIndex);
+    createNameNode(nnIndex, conf, numDataNodes, true, null, null,
+        nameserviceId, nnId);
 
     // Refresh datanodes with the newly started namenode
     for (DataNodeProperties dn : dataNodes) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
new file mode 100644
index 0000000..4dfbfd8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
@@ -0,0 +1,227 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ * This class is used to specify the setup of namenodes when instantiating
+ * a MiniDFSCluster. It consists of a set of nameservices, each of which
+ * may have one or more namenodes (in the case of HA)
+ */
+@InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "Hive", "MapReduce", "Pig"})
+@InterfaceStability.Unstable
+public class MiniDFSNNTopology {
+  private final List<NSConf> nameservices = Lists.newArrayList();
+  private boolean federation;
+
+  public MiniDFSNNTopology() {
+  }
+
+  /**
+   * Set up a simple non-federated non-HA NN.
+   */
+  public static MiniDFSNNTopology simpleSingleNN(
+      int nameNodePort, int nameNodeHttpPort) {
+    return new MiniDFSNNTopology()
+      .addNameservice(new MiniDFSNNTopology.NSConf(null)
+        .addNN(new MiniDFSNNTopology.NNConf(null)
+          .setHttpPort(nameNodeHttpPort)
+          .setIpcPort(nameNodePort)));
+  }
+  
+
+  /**
+   * Set up an HA topology with a single HA nameservice.
+   */
+  public static MiniDFSNNTopology simpleHATopology() {
+    return new MiniDFSNNTopology()
+      .addNameservice(new MiniDFSNNTopology.NSConf("minidfs-ns")
+        .addNN(new MiniDFSNNTopology.NNConf("nn1"))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2")));
+  }
+
+  /**
+   * Set up federated cluster with the given number of nameservices, each
+   * of which has only a single NameNode.
+   */
+  public static MiniDFSNNTopology simpleFederatedTopology(
+      int numNameservices) {
+    MiniDFSNNTopology topology = new MiniDFSNNTopology();
+    for (int i = 1; i <= numNameservices; i++) {
+      topology.addNameservice(new MiniDFSNNTopology.NSConf("ns" + i)
+        .addNN(new MiniDFSNNTopology.NNConf(null)));
+    }
+    topology.setFederation(true);
+    return topology;
+  }
+
+  /**
+   * Set up federated cluster with the given number of nameservices, each
+   * of which has two NameNodes.
+   */
+  public static MiniDFSNNTopology simpleHAFederatedTopology(
+      int numNameservices) {
+    MiniDFSNNTopology topology = new MiniDFSNNTopology();
+    for (int i = 0; i < numNameservices; i++) {
+      topology.addNameservice(new MiniDFSNNTopology.NSConf("ns" + i)
+        .addNN(new MiniDFSNNTopology.NNConf("nn0"))
+        .addNN(new MiniDFSNNTopology.NNConf("nn1")));
+    }
+    topology.setFederation(true);
+    return topology;
+  }
+
+  public MiniDFSNNTopology setFederation(boolean federation) {
+    this.federation = federation;
+    return this;
+  }
+
+  public MiniDFSNNTopology addNameservice(NSConf nameservice) {
+    Preconditions.checkArgument(!nameservice.getNNs().isEmpty(),
+        "Must have at least one NN in a nameservice");
+    this.nameservices.add(nameservice);
+    return this;
+  }
+
+  public int countNameNodes() {
+    int count = 0;
+    for (NSConf ns : nameservices) {
+      count += ns.nns.size();
+    }
+    return count;
+  }
+  
+  public NNConf getOnlyNameNode() {
+    Preconditions.checkState(countNameNodes() == 1,
+        "must have exactly one NN!");
+    return nameservices.get(0).getNNs().get(0);
+  }
+
+  public boolean isFederated() {
+    return nameservices.size() > 1 || federation;
+  }
+  
+  /**
+   * @return true if at least one of the nameservices
+   * in the topology has HA enabled.
+   */
+  public boolean isHA() {
+    for (NSConf ns : nameservices) {
+      if (ns.getNNs().size() > 1) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * @return true if all of the NNs in the cluster have their HTTP
+   * port specified to be non-ephemeral.
+   */
+  public boolean allHttpPortsSpecified() {
+    for (NSConf ns : nameservices) {
+      for (NNConf nn : ns.getNNs()) {
+        if (nn.getHttpPort() == 0) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+  
+  /**
+   * @return true if all of the NNs in the cluster have their IPC
+   * port specified to be non-ephemeral.
+   */
+  public boolean allIpcPortsSpecified() {
+    for (NSConf ns : nameservices) {
+      for (NNConf nn : ns.getNNs()) {
+        if (nn.getIpcPort() == 0) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  public List<NSConf> getNameservices() {
+    return nameservices;
+  }
+  
+  public static class NSConf {
+    private final String id;
+    private final List<NNConf> nns = Lists.newArrayList();
+    
+    public NSConf(String id) {
+      this.id = id;
+    }
+    
+    public NSConf addNN(NNConf nn) {
+      this.nns.add(nn);
+      return this;
+    }
+
+    public String getId() {
+      return id;
+    }
+
+    public List<NNConf> getNNs() {
+      return nns;
+    }
+  }
+  
+  public static class NNConf {
+    private String nnId;
+    private int httpPort;
+    private int ipcPort;
+    
+    public NNConf(String nnId) {
+      this.nnId = nnId;
+    }
+
+    String getNnId() {
+      return nnId;
+    }
+
+    int getIpcPort() {
+      return ipcPort;
+    }
+    
+    int getHttpPort() {
+      return httpPort;
+    }
+
+    public NNConf setHttpPort(int httpPort) {
+      this.httpPort = httpPort;
+      return this;
+    }
+
+    public NNConf setIpcPort(int ipcPort) {
+      this.ipcPort = ipcPort;
+      return this;
+    }
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
new file mode 100644
index 0000000..a88e8a7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.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.hdfs;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestDFSClientFailover {
+  
+  private static final Log LOG = LogFactory.getLog(TestDFSClientFailover.class);
+  
+  private static final Path TEST_FILE = new Path("/tmp/failover-test-file");
+  private static final int FILE_LENGTH_TO_VERIFY = 100;
+  
+  private Configuration conf = new Configuration();
+  private MiniDFSCluster cluster;
+  
+  @Before
+  public void setUpCluster() throws IOException {
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .build();
+    cluster.transitionToActive(0);
+    cluster.waitActive();
+  }
+  
+  @After
+  public void tearDownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  /**
+   * Make sure that client failover works when an active NN dies and the standby
+   * takes over.
+   */
+  @Test
+  public void testDfsClientFailover() throws IOException, URISyntaxException {
+    FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+    
+    DFSTestUtil.createFile(fs, TEST_FILE,
+        FILE_LENGTH_TO_VERIFY, (short)1, 1L);
+    
+    assertEquals(fs.getFileStatus(TEST_FILE).getLen(), FILE_LENGTH_TO_VERIFY);
+    cluster.shutdownNameNode(0);
+    cluster.transitionToActive(1);
+    assertEquals(fs.getFileStatus(TEST_FILE).getLen(), FILE_LENGTH_TO_VERIFY);
+    
+    // Check that it functions even if the URL becomes canonicalized
+    // to include a port number.
+    Path withPort = new Path("hdfs://" +
+        HATestUtil.getLogicalHostname(cluster) + ":" +
+        NameNode.DEFAULT_PORT + "/" + TEST_FILE.toUri().getPath());
+    FileSystem fs2 = withPort.getFileSystem(fs.getConf());
+    assertTrue(fs2.exists(withPort));
+
+    fs.close();
+  }
+  
+  /**
+   * Regression test for HDFS-2683.
+   */
+  @Test
+  public void testLogicalUriShouldNotHavePorts() {
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + ".foo",
+        ConfiguredFailoverProxyProvider.class.getName());
+    Path p = new Path("hdfs://foo:12345/");
+    try {
+      p.getFileSystem(conf).exists(p);
+      fail("Did not fail with fake FS");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "does not use port information", ioe);
+    }
+  }
+
+  /**
+   * Make sure that a helpful error message is shown if a proxy provider is
+   * configured for a given URI, but no actual addresses are configured for that
+   * URI.
+   */
+  @Test
+  public void testFailureWithMisconfiguredHaNNs() throws Exception {
+    String logicalHost = "misconfigured-ha-uri";
+    Configuration conf = new Configuration();
+    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHost,
+        ConfiguredFailoverProxyProvider.class.getName());
+    
+    URI uri = new URI("hdfs://" + logicalHost + "/test");
+    try {
+      FileSystem.get(uri, conf).exists(new Path("/test"));
+      fail("Successfully got proxy provider for misconfigured FS");
+    } catch (IOException ioe) {
+      LOG.info("got expected exception", ioe);
+      assertTrue("expected exception did not contain helpful message",
+          StringUtils.stringifyException(ioe).contains(
+          "Could not find any configured addresses for URI " + uri));
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
index a308c23..ad3e6d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
@@ -34,6 +34,7 @@
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 
+import static org.apache.hadoop.test.GenericTestUtils.assertExists;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -51,7 +52,7 @@
 */
 public class TestDFSUpgrade {
  
-  private static final int EXPECTED_TXID = 17;
+  private static final int EXPECTED_TXID = 49;
   private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
   private Configuration conf;
   private int testCounter = 0;
@@ -80,16 +81,16 @@
           Joiner.on("  \n").join(new File(baseDir, "current").list()));
       LOG.info("==================");
       
-      assertTrue(new File(baseDir,"current").isDirectory());
-      assertTrue(new File(baseDir,"current/VERSION").isFile());
-      assertTrue(new File(baseDir,"current/" 
-                          + getInProgressEditsFileName(imageTxId + 1)).isFile());
-      assertTrue(new File(baseDir,"current/" 
-                          + getImageFileName(imageTxId)).isFile());
-      assertTrue(new File(baseDir,"current/seen_txid").isFile());
+      assertExists(new File(baseDir,"current"));
+      assertExists(new File(baseDir,"current/VERSION"));
+      assertExists(new File(baseDir,"current/" 
+                          + getInProgressEditsFileName(imageTxId + 1)));
+      assertExists(new File(baseDir,"current/" 
+                          + getImageFileName(imageTxId)));
+      assertExists(new File(baseDir,"current/seen_txid"));
       
       File previous = new File(baseDir, "previous");
-      assertTrue(previous.isDirectory());
+      assertExists(previous);
       assertEquals(UpgradeUtilities.checksumContents(NAME_NODE, previous),
           UpgradeUtilities.checksumMasterNameNodeContents());
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index 3b93aeb..ef8f850 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -18,16 +18,20 @@
 
 package org.apache.hadoop.hdfs;
 
+import org.junit.Before;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -39,9 +43,20 @@
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 
 public class TestDFSUtil {
+  
+  /**
+   * Reset to default UGI settings since some tests change them.
+   */
+  @Before
+  public void resetUGI() {
+    UserGroupInformation.setConfiguration(new Configuration());
+  }
+  
   /**
    * Test conversion of LocatedBlock to BlockLocation
    */
@@ -86,7 +101,7 @@
   private Configuration setupAddress(String key) {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFS_FEDERATION_NAMESERVICES, "nn1");
-    conf.set(DFSUtil.getNameServiceIdKey(key, "nn1"), "localhost:9000");
+    conf.set(DFSUtil.addKeySuffixes(key, "nn1"), "localhost:9000");
     return conf;
   }
 
@@ -102,7 +117,7 @@
   }
   
   /**
-   * Test {@link DFSUtil#getNameNodeNameServiceId(Configuration)} to ensure
+   * Test {@link DFSUtil#getNamenodeNameServiceId(Configuration)} to ensure
    * nameserviceId for namenode is determined based on matching the address with
    * local node's address
    */
@@ -135,7 +150,7 @@
   }
 
   /**
-   * Test {@link DFSUtil#getNameServiceId(Configuration, String))} to ensure
+   * Test {@link DFSUtil#getNamenodeNameServiceId(Configuration)} to ensure
    * exception is thrown when multiple rpc addresses match the local node's
    * address
    */
@@ -143,9 +158,9 @@
   public void testGetNameServiceIdException() {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFS_FEDERATION_NAMESERVICES, "nn1,nn2");
-    conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
         "localhost:9000");
-    conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
         "localhost:9001");
     DFSUtil.getNamenodeNameServiceId(conf);
     fail("Expected exception is not thrown");
@@ -178,19 +193,24 @@
     final String NN1_ADDRESS = "localhost:9000";
     final String NN2_ADDRESS = "localhost:9001";
     final String NN3_ADDRESS = "localhost:9002";
-    conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn1"),
         NN1_ADDRESS);
-    conf.set(DFSUtil.getNameServiceIdKey(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, "nn2"),
         NN2_ADDRESS);
 
-    Collection<InetSocketAddress> nnAddresses = DFSUtil
+    Map<String, Map<String, InetSocketAddress>> nnMap = DFSUtil
         .getNNServiceRpcAddresses(conf);
-    assertEquals(2, nnAddresses.size());
-    Iterator<InetSocketAddress> iterator = nnAddresses.iterator();
-    InetSocketAddress addr = iterator.next();
+    assertEquals(2, nnMap.size());
+    
+    Map<String, InetSocketAddress> nn1Map = nnMap.get("nn1");
+    assertEquals(1, nn1Map.size());
+    InetSocketAddress addr = nn1Map.get(null);
     assertEquals("localhost", addr.getHostName());
     assertEquals(9000, addr.getPort());
-    addr = iterator.next();
+    
+    Map<String, InetSocketAddress> nn2Map = nnMap.get("nn2");
+    assertEquals(1, nn2Map.size());
+    addr = nn2Map.get(null);
     assertEquals("localhost", addr.getHostName());
     assertEquals(9001, addr.getPort());
 
@@ -198,6 +218,10 @@
     checkNameServiceId(conf, NN1_ADDRESS, "nn1");
     checkNameServiceId(conf, NN2_ADDRESS, "nn2");
     checkNameServiceId(conf, NN3_ADDRESS, null);
+
+    // HA is not enabled in a purely federated config
+    assertFalse(HAUtil.isHAEnabled(conf, "nn1"));
+    assertFalse(HAUtil.isHAEnabled(conf, "nn2"));
   }
 
   public void checkNameServiceId(Configuration conf, String addr,
@@ -216,9 +240,14 @@
     conf.set(FS_DEFAULT_NAME_KEY, hdfs_default);
     // If DFS_FEDERATION_NAMESERVICES is not set, verify that
     // default namenode address is returned.
-    List<InetSocketAddress> addrList = DFSUtil.getNNServiceRpcAddresses(conf);
-    assertEquals(1, addrList.size());
-    assertEquals(9999, addrList.get(0).getPort());
+    Map<String, Map<String, InetSocketAddress>> addrMap =
+      DFSUtil.getNNServiceRpcAddresses(conf);
+    assertEquals(1, addrMap.size());
+    
+    Map<String, InetSocketAddress> defaultNsMap = addrMap.get(null);
+    assertEquals(1, defaultNsMap.size());
+    
+    assertEquals(9999, defaultNsMap.get(null).getPort());
   }
   
   /**
@@ -226,20 +255,51 @@
    * copied to generic keys when the namenode starts.
    */
   @Test
-  public void testConfModification() throws IOException {
+  public void testConfModificationFederationOnly() {
     final HdfsConfiguration conf = new HdfsConfiguration();
-    conf.set(DFS_FEDERATION_NAMESERVICES, "nn1");
-    conf.set(DFS_FEDERATION_NAMESERVICE_ID, "nn1");
-    final String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
+    String nsId = "ns1";
+    
+    conf.set(DFS_FEDERATION_NAMESERVICES, nsId);
+    conf.set(DFS_FEDERATION_NAMESERVICE_ID, nsId);
 
     // Set the nameservice specific keys with nameserviceId in the config key
     for (String key : NameNode.NAMESERVICE_SPECIFIC_KEYS) {
       // Note: value is same as the key
-      conf.set(DFSUtil.getNameServiceIdKey(key, nameserviceId), key);
+      conf.set(DFSUtil.addKeySuffixes(key, nsId), key);
     }
 
     // Initialize generic keys from specific keys
-    NameNode.initializeGenericKeys(conf, nameserviceId);
+    NameNode.initializeGenericKeys(conf, nsId, null);
+
+    // Retrieve the keys without nameserviceId and Ensure generic keys are set
+    // to the correct value
+    for (String key : NameNode.NAMESERVICE_SPECIFIC_KEYS) {
+      assertEquals(key, conf.get(key));
+    }
+  }
+  
+  /**
+   * Test to ensure nameservice specific keys in the configuration are
+   * copied to generic keys when the namenode starts.
+   */
+  @Test
+  public void testConfModificationFederationAndHa() {
+    final HdfsConfiguration conf = new HdfsConfiguration();
+    String nsId = "ns1";
+    String nnId = "nn1";
+    
+    conf.set(DFS_FEDERATION_NAMESERVICES, nsId);
+    conf.set(DFS_FEDERATION_NAMESERVICE_ID, nsId);
+    conf.set(DFS_HA_NAMENODES_KEY_PREFIX + "." + nsId, nnId);
+
+    // Set the nameservice specific keys with nameserviceId in the config key
+    for (String key : NameNode.NAMESERVICE_SPECIFIC_KEYS) {
+      // Note: value is same as the key
+      conf.set(DFSUtil.addKeySuffixes(key, nsId, nnId), key);
+    }
+
+    // Initialize generic keys from specific keys
+    NameNode.initializeGenericKeys(conf, nsId, nnId);
 
     // Retrieve the keys without nameserviceId and Ensure generic keys are set
     // to the correct value
@@ -249,6 +309,39 @@
   }
 
   /**
+   * Regression test for HDFS-2934.
+   */
+  @Test
+  public void testSomeConfsNNSpecificSomeNSSpecific() {
+    final HdfsConfiguration conf = new HdfsConfiguration();
+
+    String key = DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
+    conf.set(key, "global-default");
+    conf.set(key + ".ns1", "ns1-override");
+    conf.set(key + ".ns1.nn1", "nn1-override");
+
+    // A namenode in another nameservice should get the global default.
+    Configuration newConf = new Configuration(conf);
+    NameNode.initializeGenericKeys(newConf, "ns2", "nn1");
+    assertEquals("global-default", newConf.get(key));
+    
+    // A namenode in another non-HA nameservice should get global default.
+    newConf = new Configuration(conf);
+    NameNode.initializeGenericKeys(newConf, "ns2", null);
+    assertEquals("global-default", newConf.get(key));    
+    
+    // A namenode in the same nameservice should get the ns setting
+    newConf = new Configuration(conf);
+    NameNode.initializeGenericKeys(newConf, "ns1", "nn2");
+    assertEquals("ns1-override", newConf.get(key));    
+
+    // The nn with the nn-specific setting should get its own override
+    newConf = new Configuration(conf);
+    NameNode.initializeGenericKeys(newConf, "ns1", "nn1");
+    assertEquals("nn1-override", newConf.get(key));    
+  }
+  
+  /**
    * Tests for empty configuration, an exception is thrown from
    * {@link DFSUtil#getNNServiceRpcAddresses(Configuration)}
    * {@link DFSUtil#getBackupNodeAddresses(Configuration)}
@@ -258,21 +351,30 @@
   public void testEmptyConf() {
     HdfsConfiguration conf = new HdfsConfiguration(false);
     try {
-      DFSUtil.getNNServiceRpcAddresses(conf);
-      fail("Expected IOException is not thrown");
+      Map<String, Map<String, InetSocketAddress>> map =
+          DFSUtil.getNNServiceRpcAddresses(conf);
+      fail("Expected IOException is not thrown, result was: " +
+          DFSUtil.addressMapToString(map));
     } catch (IOException expected) {
+      /** Expected */
     }
 
     try {
-      DFSUtil.getBackupNodeAddresses(conf);
-      fail("Expected IOException is not thrown");
+      Map<String, Map<String, InetSocketAddress>> map =
+        DFSUtil.getBackupNodeAddresses(conf);
+      fail("Expected IOException is not thrown, result was: " +
+          DFSUtil.addressMapToString(map));
     } catch (IOException expected) {
+      /** Expected */
     }
 
     try {
-      DFSUtil.getSecondaryNameNodeAddresses(conf);
-      fail("Expected IOException is not thrown");
+      Map<String, Map<String, InetSocketAddress>> map =
+        DFSUtil.getSecondaryNameNodeAddresses(conf);
+      fail("Expected IOException is not thrown, result was: " +
+          DFSUtil.addressMapToString(map));
     } catch (IOException expected) {
+      /** Expected */
     }
   }
 
@@ -286,5 +388,144 @@
     String httpport = DFSUtil.getInfoServer(null, conf, false);
     assertEquals("0.0.0.0:50070", httpport);
   }
+  
+  @Test
+  public void testHANameNodesWithFederation() throws URISyntaxException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    
+    final String NS1_NN1_HOST = "ns1-nn1.example.com:8020";
+    final String NS1_NN2_HOST = "ns1-nn2.example.com:8020";
+    final String NS2_NN1_HOST = "ns2-nn1.example.com:8020";
+    final String NS2_NN2_HOST = "ns2-nn2.example.com:8020";
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1");
+    
+    // Two nameservices, each with two NNs.
+    conf.set(DFS_FEDERATION_NAMESERVICES, "ns1,ns2");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),
+        "ns1-nn1,ns1-nn2");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns2"),
+        "ns2-nn1,ns2-nn2");
+    conf.set(DFSUtil.addKeySuffixes(
+          DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn1"),
+        NS1_NN1_HOST);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "ns1-nn2"),
+        NS1_NN2_HOST);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn1"),
+        NS2_NN1_HOST);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns2", "ns2-nn2"),
+        NS2_NN2_HOST);
+    
+    Map<String, Map<String, InetSocketAddress>> map =
+      DFSUtil.getHaNnRpcAddresses(conf);
 
-}
\ No newline at end of file
+    assertTrue(HAUtil.isHAEnabled(conf, "ns1"));
+    assertTrue(HAUtil.isHAEnabled(conf, "ns2"));
+    assertFalse(HAUtil.isHAEnabled(conf, "ns3"));
+    
+    assertEquals(NS1_NN1_HOST, map.get("ns1").get("ns1-nn1").toString());
+    assertEquals(NS1_NN2_HOST, map.get("ns1").get("ns1-nn2").toString());
+    assertEquals(NS2_NN1_HOST, map.get("ns2").get("ns2-nn1").toString());
+    assertEquals(NS2_NN2_HOST, map.get("ns2").get("ns2-nn2").toString());
+    
+    assertEquals(NS1_NN1_HOST, 
+        DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn1"));
+    assertEquals(NS1_NN2_HOST, 
+        DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn2"));
+    assertEquals(NS2_NN1_HOST, 
+        DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn1"));
+
+    // No nameservice was given and we can't determine which service addr
+    // to use as two nameservices could share a namenode ID.
+    assertEquals(null, DFSUtil.getNamenodeServiceAddr(conf, null, "ns1-nn1"));
+
+    // Ditto for nameservice IDs, if multiple are defined
+    assertEquals(null, DFSUtil.getNamenodeNameServiceId(conf));
+    assertEquals(null, DFSUtil.getSecondaryNameServiceId(conf));
+    
+    Collection<URI> uris = DFSUtil.getNameServiceUris(conf, DFS_NAMENODE_RPC_ADDRESS_KEY);
+    assertEquals(2, uris.size());
+    assertTrue(uris.contains(new URI("hdfs://ns1")));
+    assertTrue(uris.contains(new URI("hdfs://ns2")));
+  }
+
+  @Test
+  public void getNameNodeServiceAddr() throws IOException {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    
+    // One nameservice with two NNs
+    final String NS1_NN1_HOST = "ns1-nn1.example.com:8020";
+    final String NS1_NN1_HOST_SVC = "ns1-nn2.example.com:8021";
+    final String NS1_NN2_HOST = "ns1-nn1.example.com:8020";
+    final String NS1_NN2_HOST_SVC = "ns1-nn2.example.com:8021";
+   
+    conf.set(DFS_FEDERATION_NAMESERVICES, "ns1");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2"); 
+
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn2"), NS1_NN2_HOST);
+
+    // The rpc address is used if no service address is defined
+    assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, null, "nn1"));
+    assertEquals(NS1_NN2_HOST, DFSUtil.getNamenodeServiceAddr(conf, null, "nn2"));
+
+    // A nameservice is specified explicitly
+    assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1"));
+    assertEquals(null, DFSUtil.getNamenodeServiceAddr(conf, "invalid", "nn1"));
+    
+    // The service addrs are used when they are defined
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST_SVC);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns1", "nn2"), NS1_NN2_HOST_SVC);
+
+    assertEquals(NS1_NN1_HOST_SVC, DFSUtil.getNamenodeServiceAddr(conf, null, "nn1"));
+    assertEquals(NS1_NN2_HOST_SVC, DFSUtil.getNamenodeServiceAddr(conf, null, "nn2"));
+
+    // We can determine the nameservice ID, there's only one listed
+    assertEquals("ns1", DFSUtil.getNamenodeNameServiceId(conf));
+    assertEquals("ns1", DFSUtil.getSecondaryNameServiceId(conf));
+  }
+
+  @Test
+  public void testSubstituteForWildcardAddress() throws IOException {
+    assertEquals("foo:12345",
+        DFSUtil.substituteForWildcardAddress("0.0.0.0:12345", "foo"));
+    assertEquals("127.0.0.1:12345",
+        DFSUtil.substituteForWildcardAddress("127.0.0.1:12345", "foo"));
+  }
+  
+  @Test
+  public void testGetNNUris() throws Exception {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    
+    final String NS1_NN1_HOST = "ns1-nn1.example.com:8020";
+    final String NS1_NN2_HOST = "ns1-nn1.example.com:8020";
+    final String NS2_NN_HOST  = "ns2-nn.example.com:8020";
+    final String NN_HOST      = "nn.example.com:8020";
+    
+    conf.set(DFS_FEDERATION_NAMESERVICES, "ns1,ns2");
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2");
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn1"), NS1_NN1_HOST);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn2"), NS1_NN2_HOST);
+    
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "ns2"),
+        NS2_NN_HOST);
+    
+    conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "hdfs://" + NN_HOST);
+    
+    Collection<URI> uris = DFSUtil.getNameServiceUris(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, 
+        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
+    
+    assertEquals(3, uris.size());
+    assertTrue(uris.contains(new URI("hdfs://ns1")));
+    assertTrue(uris.contains(new URI("hdfs://" + NS2_NN_HOST)));
+    assertTrue(uris.contains(new URI("hdfs://" + NN_HOST)));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
index 089ab4d..af0bf6a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
@@ -302,7 +302,7 @@
         testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L,
             "Cannot create a RBW block", true);
         // test PIPELINE_SETUP_APPEND on an existing block
-        newGS = newBlock.getGenerationStamp() + 1;
+        newGS = firstBlock.getGenerationStamp() + 1;
         testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_APPEND,
             newGS, "Cannot append to a RBW replica", true);
         // test PIPELINE_SETUP_APPEND on an existing block
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index faf7efd..6997ebc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -279,7 +279,8 @@
    * @throws IOException */
   private void startCluster(int numNameNodes, int numDatanodes,
       Configuration conf) throws IOException {
-    cluster = new MiniDFSCluster.Builder(conf).numNameNodes(numNameNodes)
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(numNameNodes))
         .numDataNodes(numDatanodes).build();
     cluster.waitActive();
     for (int i = 0; i < numNameNodes; i++) {
@@ -507,7 +508,8 @@
       InterruptedException {
     conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
     int numDatanodes = 1;
-    cluster = new MiniDFSCluster.Builder(conf).numNameNodes(numNameNodes)
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(numNameNodes))
         .numDataNodes(numDatanodes).setupHostsFile(true).build();
     cluster.waitActive();
     
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
index 033478f..e10eab8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
@@ -90,7 +90,7 @@
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       FileSystem fs = cluster.getFileSystem();
       File editLog =
-        new File(FSImageTestUtil.getNameNodeCurrentDirs(cluster).get(0),
+        new File(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0).get(0),
             NNStorage.getInProgressEditsFileName(1));
       EnumMap<FSEditLogOpCodes, Holder<Integer>> counts;
       
@@ -98,18 +98,31 @@
       writeAndAppend(fs, p1, BLOCK_SIZE, BLOCK_SIZE);
 
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
+      // OP_ADD to create file
+      // OP_UPDATE_BLOCKS for first block
+      // OP_CLOSE to close file
+      // OP_ADD to reopen file
+      // OP_UPDATE_BLOCKS for second block
+      // OP_CLOSE to close file
       assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
+      assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
       assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
 
       Path p2 = new Path("/not-block-boundaries");
       writeAndAppend(fs, p2, BLOCK_SIZE/2, BLOCK_SIZE);
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
-      // We get *3* OP_ADDS from this test rather than two. The first
-      // OP_ADD comes from re-opening the file to establish the lease,
-      // the second comes from the updatePipeline call when the block
-      // itself has its generation stamp incremented
-      assertEquals(5, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
-      assertEquals(4, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
+      // OP_ADD to create file
+      // OP_UPDATE_BLOCKS for first block
+      // OP_CLOSE to close file
+      // OP_ADD to re-establish the lease
+      // OP_UPDATE_BLOCKS from the updatePipeline call (increments genstamp of last block)
+      // OP_UPDATE_BLOCKS at the start of the second block
+      // OP_CLOSE to close file
+      // Total: 2 OP_ADDs, 3 OP_UPDATE_BLOCKS, and 2 OP_CLOSEs in addition
+      //        to the ones above
+      assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
+      assertEquals(2+3, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
+      assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
       
       cluster.restartNameNode();
       
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
index af9d05c..d3df0c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
@@ -146,8 +146,14 @@
       // report corrupted block by the third datanode
       DatanodeRegistration dnR = 
         DataNodeTestUtils.getDNRegistrationForBP(dataNode, blk.getBlockPoolId());
-      cluster.getNamesystem().getBlockManager().findAndMarkBlockAsCorrupt(
-          blk, new DatanodeInfo(dnR), "TEST");
+      FSNamesystem ns = cluster.getNamesystem();
+      ns.writeLock();
+      try {
+        cluster.getNamesystem().getBlockManager().findAndMarkBlockAsCorrupt(
+            blk, new DatanodeInfo(dnR), "TEST");
+      } finally {
+        ns.writeUnlock();
+      }
       
       // open the file
       fs.open(FILE_PATH);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index 8693885..b0878d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -25,7 +25,6 @@
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -34,8 +33,6 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
-
 import junit.framework.TestCase;
 /**
  * This class tests if block replacement request to data nodes work correctly.
@@ -97,8 +94,8 @@
       // get RPC client to namenode
       InetSocketAddress addr = new InetSocketAddress("localhost",
           cluster.getNameNodePort());
-      NamenodeProtocol namenode = new NamenodeProtocolTranslatorPB(addr, CONF,
-          UserGroupInformation.getCurrentUser());
+      NamenodeProtocol namenode = NameNodeProxies.createProxy(CONF,
+          NameNode.getUri(addr), NamenodeProtocol.class).getProxy();
 
       // get blocks of size fileLen from dataNodes[0]
       BlockWithLocations[] locs;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java
index 0d8174e..3e90665 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestIsMethodSupported.java
@@ -22,6 +22,7 @@
 
 import junit.framework.Assert;
 
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
@@ -31,8 +32,13 @@
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
+import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -49,7 +55,7 @@
   
   @BeforeClass
   public static void setUp() throws Exception {
-    cluster = (new MiniDFSCluster.Builder(conf)).numNameNodes(1)
+    cluster = (new MiniDFSCluster.Builder(conf))
         .numDataNodes(1).build();
     nnAddress = cluster.getNameNode().getNameNodeAddress();
     dnAddress = new InetSocketAddress(cluster.getDataNodes().get(0)
@@ -66,8 +72,9 @@
   @Test
   public void testNamenodeProtocol() throws IOException {
     NamenodeProtocolTranslatorPB translator =
-        new NamenodeProtocolTranslatorPB(nnAddress, conf,
-            UserGroupInformation.getCurrentUser());
+        (NamenodeProtocolTranslatorPB) NameNodeProxies.createNonHAProxy(conf,
+            nnAddress, NamenodeProtocol.class, UserGroupInformation.getCurrentUser(),
+            true).getProxy();
     boolean exists = translator.isMethodSupported("rollEditLog");
     Assert.assertTrue(exists);
     exists = translator.isMethodSupported("bogusMethod");
@@ -99,15 +106,17 @@
   @Test
   public void testClientNamenodeProtocol() throws IOException {
     ClientNamenodeProtocolTranslatorPB translator = 
-        new ClientNamenodeProtocolTranslatorPB(nnAddress, conf, 
-            UserGroupInformation.getCurrentUser()); 
+        (ClientNamenodeProtocolTranslatorPB) NameNodeProxies.createNonHAProxy(
+            conf, nnAddress, ClientProtocol.class,
+            UserGroupInformation.getCurrentUser(), true).getProxy();
     Assert.assertTrue(translator.isMethodSupported("mkdirs"));
   }
   
   @Test
   public void tesJournalProtocol() throws IOException {
-    JournalProtocolTranslatorPB translator = 
-        new JournalProtocolTranslatorPB(nnAddress, conf);
+    JournalProtocolTranslatorPB translator = (JournalProtocolTranslatorPB)
+        NameNodeProxies.createNonHAProxy(conf, nnAddress, JournalProtocol.class,
+            UserGroupInformation.getCurrentUser(), true).getProxy();
     //Nameode doesn't implement JournalProtocol
     Assert.assertFalse(translator.isMethodSupported("startLogSegment"));
   }
@@ -130,24 +139,30 @@
   @Test
   public void testGetUserMappingsProtocol() throws IOException {
     GetUserMappingsProtocolClientSideTranslatorPB translator = 
-        new GetUserMappingsProtocolClientSideTranslatorPB(
-            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+        (GetUserMappingsProtocolClientSideTranslatorPB)
+        NameNodeProxies.createNonHAProxy(conf, nnAddress,
+            GetUserMappingsProtocol.class, UserGroupInformation.getCurrentUser(),
+            true).getProxy();
     Assert.assertTrue(translator.isMethodSupported("getGroupsForUser"));
   }
   
   @Test
   public void testRefreshAuthorizationPolicyProtocol() throws IOException {
-    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB translator =
-        new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
-            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+    RefreshAuthorizationPolicyProtocolClientSideTranslatorPB translator = 
+      (RefreshAuthorizationPolicyProtocolClientSideTranslatorPB)
+      NameNodeProxies.createNonHAProxy(conf, nnAddress,
+          RefreshAuthorizationPolicyProtocol.class,
+          UserGroupInformation.getCurrentUser(), true).getProxy();
     Assert.assertTrue(translator.isMethodSupported("refreshServiceAcl"));
   }
   
   @Test
   public void testRefreshUserMappingsProtocol() throws IOException {
     RefreshUserMappingsProtocolClientSideTranslatorPB translator =
-        new RefreshUserMappingsProtocolClientSideTranslatorPB(
-            nnAddress, UserGroupInformation.getCurrentUser(), conf);
+        (RefreshUserMappingsProtocolClientSideTranslatorPB)
+        NameNodeProxies.createNonHAProxy(conf, nnAddress,
+            RefreshUserMappingsProtocol.class,
+            UserGroupInformation.getCurrentUser(), true).getProxy();
     Assert.assertTrue(
         translator.isMethodSupported("refreshUserToGroupsMappings"));
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
index 4e31523..0eec0d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
@@ -20,6 +20,7 @@
 
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,11 +38,13 @@
   private static final String CLUSTER_1 = "cluster1";
   private static final String CLUSTER_2 = "cluster2";
   private static final String CLUSTER_3 = "cluster3";
+  private static final String CLUSTER_4 = "cluster4";
   protected String testDataPath;
   protected File testDataDir;
   @Before
   public void setUp() {
-    testDataPath = System.getProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA);
+    testDataPath = System.getProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
+        "build/test/data");
     testDataDir = new File(new File(testDataPath).getParentFile(),
                            "miniclusters");
 
@@ -103,5 +106,21 @@
     }
   }
 
-
+  @Test(timeout=100000)
+  public void testIsClusterUpAfterShutdown() throws Throwable {
+    Configuration conf = new HdfsConfiguration();
+    File testDataCluster4 = new File(testDataPath, CLUSTER_4);
+    String c4Path = testDataCluster4.getAbsolutePath();
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, c4Path);
+    MiniDFSCluster cluster4 = new MiniDFSCluster.Builder(conf).build();
+    try {
+      DistributedFileSystem dfs = (DistributedFileSystem) cluster4.getFileSystem();
+      dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER);
+      cluster4.shutdown();
+    } finally {
+      while(cluster4.isClusterUp()){
+        Thread.sleep(1000);
+      }  
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
new file mode 100644
index 0000000..cb98929
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
@@ -0,0 +1,353 @@
+/**
+ * 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 org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import static org.junit.Assert.*;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * A JUnit test for checking if restarting DFS preserves the
+ * blocks that are part of an unclosed file.
+ */
+public class TestPersistBlocks {
+  static {
+    ((Log4JLogger)FSImage.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
+  }
+  
+  private static final int BLOCK_SIZE = 4096;
+  private static final int NUM_BLOCKS = 5;
+
+  private static final String FILE_NAME = "/data";
+  private static final Path FILE_PATH = new Path(FILE_NAME);
+  
+  static final byte[] DATA_BEFORE_RESTART = new byte[BLOCK_SIZE * NUM_BLOCKS];
+  static final byte[] DATA_AFTER_RESTART = new byte[BLOCK_SIZE * NUM_BLOCKS];
+  
+  private static final String HADOOP_1_0_MULTIBLOCK_TGZ =
+    "hadoop-1.0-multiblock-file.tgz";
+  static {
+    Random rand = new Random();
+    rand.nextBytes(DATA_BEFORE_RESTART);
+    rand.nextBytes(DATA_AFTER_RESTART);
+  }
+  
+  /** check if DFS remains in proper condition after a restart */
+  @Test
+  public void testRestartDfs() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    // Turn off persistent IPC, so that the DFSClient can survive NN restart
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+        0);
+    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
+    MiniDFSCluster cluster = null;
+
+    long len = 0;
+    FSDataOutputStream stream;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      FileSystem fs = cluster.getFileSystem();
+      // Creating a file with 4096 blockSize to write multiple blocks
+      stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
+      stream.write(DATA_BEFORE_RESTART);
+      stream.hflush();
+      
+      // Wait for at least a few blocks to get through
+      while (len <= BLOCK_SIZE) {
+        FileStatus status = fs.getFileStatus(FILE_PATH);
+        len = status.getLen();
+        Thread.sleep(100);
+      }
+      
+      // explicitly do NOT close the file.
+      cluster.restartNameNode();
+      
+      // Check that the file has no less bytes than before the restart
+      // This would mean that blocks were successfully persisted to the log
+      FileStatus status = fs.getFileStatus(FILE_PATH);
+      assertTrue("Length too short: " + status.getLen(),
+          status.getLen() >= len);
+      
+      // And keep writing (ensures that leases are also persisted correctly)
+      stream.write(DATA_AFTER_RESTART);
+      stream.close();
+      
+      // Verify that the data showed up, both from before and after the restart.
+      FSDataInputStream readStream = fs.open(FILE_PATH);
+      try {
+        byte[] verifyBuf = new byte[DATA_BEFORE_RESTART.length];
+        IOUtils.readFully(readStream, verifyBuf, 0, verifyBuf.length);
+        assertArrayEquals(DATA_BEFORE_RESTART, verifyBuf);
+        
+        IOUtils.readFully(readStream, verifyBuf, 0, verifyBuf.length);
+        assertArrayEquals(DATA_AFTER_RESTART, verifyBuf);
+      } finally {
+        IOUtils.closeStream(readStream);
+      }
+    } finally {
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
+  
+  @Test
+  public void testRestartDfsWithAbandonedBlock() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    // Turn off persistent IPC, so that the DFSClient can survive NN restart
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+        0);
+    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
+    MiniDFSCluster cluster = null;
+
+    long len = 0;
+    FSDataOutputStream stream;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      FileSystem fs = cluster.getFileSystem();
+      // Creating a file with 4096 blockSize to write multiple blocks
+      stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
+      stream.write(DATA_BEFORE_RESTART);
+      stream.hflush();
+      
+      // Wait for all of the blocks to get through
+      while (len < BLOCK_SIZE * (NUM_BLOCKS - 1)) {
+        FileStatus status = fs.getFileStatus(FILE_PATH);
+        len = status.getLen();
+        Thread.sleep(100);
+      }
+      
+      // Abandon the last block
+      DFSClient dfsclient = DFSClientAdapter.getDFSClient((DistributedFileSystem)fs);
+      LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(
+          FILE_NAME, 0, BLOCK_SIZE * NUM_BLOCKS);
+      assertEquals(NUM_BLOCKS, blocks.getLocatedBlocks().size());
+      LocatedBlock b = blocks.getLastLocatedBlock();
+      dfsclient.getNamenode().abandonBlock(b.getBlock(), FILE_NAME,
+          dfsclient.clientName);
+      
+      // explicitly do NOT close the file.
+      cluster.restartNameNode();
+      
+      // Check that the file has no less bytes than before the restart
+      // This would mean that blocks were successfully persisted to the log
+      FileStatus status = fs.getFileStatus(FILE_PATH);
+      assertTrue("Length incorrect: " + status.getLen(),
+          status.getLen() != len - BLOCK_SIZE);
+
+      // Verify the data showed up from before restart, sans abandoned block.
+      FSDataInputStream readStream = fs.open(FILE_PATH);
+      try {
+        byte[] verifyBuf = new byte[DATA_BEFORE_RESTART.length - BLOCK_SIZE];
+        IOUtils.readFully(readStream, verifyBuf, 0, verifyBuf.length);
+        byte[] expectedBuf = new byte[DATA_BEFORE_RESTART.length - BLOCK_SIZE];
+        System.arraycopy(DATA_BEFORE_RESTART, 0,
+            expectedBuf, 0, expectedBuf.length);
+        assertArrayEquals(expectedBuf, verifyBuf);
+      } finally {
+        IOUtils.closeStream(readStream);
+      }
+    } finally {
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
+  
+  @Test
+  public void testRestartWithPartialBlockHflushed() throws IOException {
+    final Configuration conf = new HdfsConfiguration();
+    // Turn off persistent IPC, so that the DFSClient can survive NN restart
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+        0);
+    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
+    MiniDFSCluster cluster = null;
+
+    FSDataOutputStream stream;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      FileSystem fs = cluster.getFileSystem();
+      NameNode.getAddress(conf).getPort();
+      // Creating a file with 4096 blockSize to write multiple blocks
+      stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
+      stream.write(DATA_BEFORE_RESTART);
+      stream.write((byte)1);
+      stream.hflush();
+      
+      // explicitly do NOT close the file before restarting the NN.
+      cluster.restartNameNode();
+      
+      // this will fail if the final block of the file is prematurely COMPLETEd
+      stream.write((byte)2);
+      stream.hflush();
+      stream.close();
+      
+      assertEquals(DATA_BEFORE_RESTART.length + 2,
+          fs.getFileStatus(FILE_PATH).getLen());
+      
+      FSDataInputStream readStream = fs.open(FILE_PATH);
+      try {
+        byte[] verifyBuf = new byte[DATA_BEFORE_RESTART.length + 2];
+        IOUtils.readFully(readStream, verifyBuf, 0, verifyBuf.length);
+        byte[] expectedBuf = new byte[DATA_BEFORE_RESTART.length + 2];
+        System.arraycopy(DATA_BEFORE_RESTART, 0, expectedBuf, 0,
+            DATA_BEFORE_RESTART.length);
+        System.arraycopy(new byte[]{1, 2}, 0, expectedBuf,
+            DATA_BEFORE_RESTART.length, 2);
+        assertArrayEquals(expectedBuf, verifyBuf);
+      } finally {
+        IOUtils.closeStream(readStream);
+      }
+    } finally {
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
+  
+  @Test
+  public void testRestartWithAppend() throws IOException {
+    final Configuration conf = new HdfsConfiguration();
+    // Turn off persistent IPC, so that the DFSClient can survive NN restart
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
+        0);
+    conf.setBoolean(DFSConfigKeys.DFS_PERSIST_BLOCKS_KEY, true);
+    MiniDFSCluster cluster = null;
+
+    FSDataOutputStream stream;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      FileSystem fs = cluster.getFileSystem();
+      NameNode.getAddress(conf).getPort();
+      // Creating a file with 4096 blockSize to write multiple blocks
+      stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);
+      stream.write(DATA_BEFORE_RESTART, 0, DATA_BEFORE_RESTART.length / 2);
+      stream.close();
+      stream = fs.append(FILE_PATH, BLOCK_SIZE);
+      stream.write(DATA_BEFORE_RESTART, DATA_BEFORE_RESTART.length / 2,
+          DATA_BEFORE_RESTART.length / 2);
+      stream.close();
+      
+      assertEquals(DATA_BEFORE_RESTART.length,
+          fs.getFileStatus(FILE_PATH).getLen());
+      
+      cluster.restartNameNode();
+      
+      assertEquals(DATA_BEFORE_RESTART.length,
+          fs.getFileStatus(FILE_PATH).getLen());
+      
+      FSDataInputStream readStream = fs.open(FILE_PATH);
+      try {
+        byte[] verifyBuf = new byte[DATA_BEFORE_RESTART.length];
+        IOUtils.readFully(readStream, verifyBuf, 0, verifyBuf.length);
+        assertArrayEquals(DATA_BEFORE_RESTART, verifyBuf);
+      } finally {
+        IOUtils.closeStream(readStream);
+      }
+    } finally {
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
+  
+  /**
+   * Earlier versions of HDFS didn't persist block allocation to the edit log.
+   * This makes sure that we can still load an edit log when the OP_CLOSE
+   * is the opcode which adds all of the blocks. This is a regression
+   * test for HDFS-2773.
+   * This test uses a tarred pseudo-distributed cluster from Hadoop 1.0
+   * which has a multi-block file. This is similar to the tests in
+   * {@link TestDFSUpgradeFromImage} but none of those images include
+   * a multi-block file.
+   */
+  @Test
+  public void testEarlierVersionEditLog() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+        
+    String tarFile = System.getProperty("test.cache.data", "build/test/cache")
+      + "/" + HADOOP_1_0_MULTIBLOCK_TGZ;
+    String testDir = System.getProperty("test.build.data", "build/test/data");
+    File dfsDir = new File(testDir, "image-1.0");
+    if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) {
+      throw new IOException("Could not delete dfs directory '" + dfsDir + "'");
+    }
+    FileUtil.unTar(new File(tarFile), new File(testDir));
+
+    File nameDir = new File(dfsDir, "name");
+    GenericTestUtils.assertExists(nameDir);
+    File dataDir = new File(dfsDir, "data");
+    GenericTestUtils.assertExists(dataDir);
+    
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDir.getAbsolutePath());
+    
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+      .format(false)
+      .manageDataDfsDirs(false)
+      .manageNameDfsDirs(false)
+      .numDataNodes(1)
+      .startupOption(StartupOption.UPGRADE)
+      .build();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      Path testPath = new Path("/user/todd/4blocks");
+      // Read it without caring about the actual data within - we just need
+      // to make sure that the block states and locations are OK.
+      DFSTestUtil.readFile(fs, testPath);
+      
+      // Ensure that we can append to it - if the blocks were in some funny
+      // state we'd get some kind of issue here. 
+      FSDataOutputStream stm = fs.append(testPath);
+      try {
+        stm.write(1);
+      } finally {
+        IOUtils.closeStream(stm);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
index a488b0a..e211d20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
@@ -75,7 +75,8 @@
   private void checkFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     Configuration conf = fileSys.getConf();
-    ClientProtocol namenode = DFSUtil.createNamenode(conf);
+    ClientProtocol namenode = NameNodeProxies.createProxy(conf, fileSys.getUri(),
+        ClientProtocol.class).getProxy();
       
     waitForBlockReplication(name.toString(), namenode, 
                             Math.min(numDatanodes, repl), -1);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
index 4d18e98..c2aaf06 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
@@ -20,6 +20,8 @@
 
 
 
+import static org.junit.Assert.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
@@ -32,12 +34,16 @@
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -64,6 +70,7 @@
     config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
     config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
+    config.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     config.set("hadoop.security.auth_to_local",
         "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
     FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0");
@@ -71,7 +78,6 @@
     cluster.waitActive();
     dtSecretManager = NameNodeAdapter.getDtSecretManager(
         cluster.getNamesystem());
-    dtSecretManager.startThreads();
   }
 
   @After
@@ -269,5 +275,51 @@
       }
     });
   }
- 
+  
+  /**
+   * Test that the delegation token secret manager only runs when the
+   * NN is out of safe mode. This is because the secret manager
+   * has to log to the edit log, which should not be written in
+   * safe mode. Regression test for HDFS-2579.
+   */
+  @Test
+  public void testDTManagerInSafeMode() throws Exception {
+    cluster.startDataNodes(config, 1, true, StartupOption.REGULAR, null);
+    FileSystem fs = cluster.getFileSystem();
+    for (int i = 0; i < 5; i++) {
+      DFSTestUtil.createFile(fs, new Path("/test-" + i), 100, (short)1, 1L);
+    }
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY, 500); 
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 30000);
+    cluster.setWaitSafeMode(false);
+    cluster.restartNameNode();
+    NameNode nn = cluster.getNameNode();
+    assertTrue(nn.isInSafeMode());
+    DelegationTokenSecretManager sm =
+      NameNodeAdapter.getDtSecretManager(nn.getNamesystem());
+    assertFalse("Secret manager should not run in safe mode", sm.isRunning());
+    
+    NameNodeAdapter.leaveSafeMode(nn, false);
+    assertTrue("Secret manager should start when safe mode is exited",
+        sm.isRunning());
+    
+    LOG.info("========= entering safemode again");
+    
+    NameNodeAdapter.enterSafeMode(nn, false);
+    assertFalse("Secret manager should stop again when safe mode " +
+        "is manually entered", sm.isRunning());
+    
+    // Set the cluster to leave safemode quickly on its own.
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 0);
+    cluster.setWaitSafeMode(true);
+    cluster.restartNameNode();
+    nn = cluster.getNameNode();
+    sm = NameNodeAdapter.getDtSecretManager(nn.getNamesystem());
+
+    assertFalse(nn.isInSafeMode());
+    assertTrue(sm.isRunning());
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
index cdad31c..6837f65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
@@ -48,7 +48,6 @@
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
@@ -114,11 +113,12 @@
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
     config.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER),
         "group1");
+    config.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     configureSuperUserIPAddresses(config, REAL_USER);
     FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0");
     cluster = new MiniDFSCluster.Builder(config).build();
     cluster.waitActive();
-    NameNodeAdapter.getDtSecretManager(cluster.getNamesystem()).startThreads();
     ProxyUsers.refreshSuperUserGroupsConfiguration(config);
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
index 61953c8..01725b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
@@ -373,7 +373,7 @@
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 512);
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numNameNodes(1)
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(1).build();
     cluster.waitActive();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index eb56746..81b03a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -18,9 +18,10 @@
 package org.apache.hadoop.hdfs.server.balancer;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
@@ -37,28 +38,28 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 /**
  * This class tests if a balancer schedules tasks correctly.
  */
 public class TestBalancer extends TestCase {
   private static final Log LOG = LogFactory.getLog(
-  "org.apache.hadoop.hdfs.TestReplication");
+  "org.apache.hadoop.hdfs.TestBalancer");
   
-  final private static long CAPACITY = 500L;
-  final private static String RACK0 = "/rack0";
-  final private static String RACK1 = "/rack1";
-  final private static String RACK2 = "/rack2";
-  final static private String fileName = "/tmp.txt";
-  final static private Path filePath = new Path(fileName);
+  final static long CAPACITY = 500L;
+  final static String RACK0 = "/rack0";
+  final static String RACK1 = "/rack1";
+  final static String RACK2 = "/rack2";
+  final private static String fileName = "/tmp.txt";
+  final static Path filePath = new Path(fileName);
   private MiniDFSCluster cluster;
 
   ClientProtocol client;
@@ -82,9 +83,10 @@
   }
 
   /* create a file with a length of <code>fileLen</code> */
-  private void createFile(long fileLen, short replicationFactor)
+  static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen,
+      short replicationFactor, int nnIndex)
   throws IOException {
-    FileSystem fs = cluster.getFileSystem();
+    FileSystem fs = cluster.getFileSystem(nnIndex);
     DFSTestUtil.createFile(fs, filePath, fileLen, 
         replicationFactor, r.nextLong());
     DFSTestUtil.waitReplication(fs, filePath, replicationFactor);
@@ -99,11 +101,12 @@
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numNodes).build();
     try {
       cluster.waitActive();
-      client = DFSUtil.createNamenode(conf);
+      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
 
       short replicationFactor = (short)(numNodes-1);
       long fileLen = size/replicationFactor;
-      createFile(fileLen, replicationFactor);
+      createFile(cluster , filePath, fileLen, replicationFactor, 0);
 
       List<LocatedBlock> locatedBlocks = client.
       getBlockLocations(fileName, 0, fileLen).getLocatedBlocks();
@@ -193,7 +196,8 @@
                                               .simulatedCapacities(capacities)
                                               .build();
     cluster.waitActive();
-    client = DFSUtil.createNamenode(conf);
+    client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
+        ClientProtocol.class).getProxy();
 
     for(int i = 0; i < blocksDN.length; i++)
       cluster.injectBlocks(i, Arrays.asList(blocksDN[i]));
@@ -211,7 +215,8 @@
    * @throws IOException - if getStats() fails
    * @throws TimeoutException
    */
-  private void waitForHeartBeat(long expectedUsedSpace, long expectedTotalSpace)
+  static void waitForHeartBeat(long expectedUsedSpace,
+      long expectedTotalSpace, ClientProtocol client, MiniDFSCluster cluster)
   throws IOException, TimeoutException {
     long timeout = TIMEOUT;
     long failtime = (timeout <= 0L) ? Long.MAX_VALUE
@@ -248,7 +253,8 @@
    * @throws IOException
    * @throws TimeoutException
    */
-  private void waitForBalancer(long totalUsedSpace, long totalCapacity) 
+  static void waitForBalancer(long totalUsedSpace, long totalCapacity,
+      ClientProtocol client, MiniDFSCluster cluster)
   throws IOException, TimeoutException {
     long timeout = TIMEOUT;
     long failtime = (timeout <= 0L) ? Long.MAX_VALUE
@@ -305,13 +311,15 @@
                                 .build();
     try {
       cluster.waitActive();
-      client = DFSUtil.createNamenode(conf);
+      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
 
       long totalCapacity = sum(capacities);
       
       // fill up the cluster to be 30% full
       long totalUsedSpace = totalCapacity*3/10;
-      createFile(totalUsedSpace/numOfDatanodes, (short)numOfDatanodes);
+      createFile(cluster, filePath, totalUsedSpace / numOfDatanodes,
+          (short) numOfDatanodes, 0);
       // start up an empty node with the same capacity and on the same rack
       cluster.startDataNodes(conf, 1, true, null,
           new String[]{newRack}, new long[]{newCapacity});
@@ -327,17 +335,16 @@
 
   private void runBalancer(Configuration conf,
       long totalUsedSpace, long totalCapacity) throws Exception {
-    waitForHeartBeat(totalUsedSpace, totalCapacity);
+    waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
 
     // start rebalancing
-    final List<InetSocketAddress> namenodes =new ArrayList<InetSocketAddress>();
-    namenodes.add(NameNode.getServiceAddress(conf, true));
+    Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
     final int r = Balancer.run(namenodes, Balancer.Parameters.DEFALUT, conf);
     assertEquals(Balancer.ReturnStatus.SUCCESS.code, r);
 
-    waitForHeartBeat(totalUsedSpace, totalCapacity);
+    waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
     LOG.info("Rebalancing with default ctor.");
-    waitForBalancer(totalUsedSpace, totalCapacity);
+    waitForBalancer(totalUsedSpace, totalCapacity, client, cluster);
   }
   
   /** one-node cluster test*/
@@ -396,13 +403,15 @@
                                 .build();
     try {
       cluster.waitActive();
-      client = DFSUtil.createNamenode(conf);
+      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
 
       long totalCapacity = sum(capacities);
 
       // fill up the cluster to be 30% full
       long totalUsedSpace = totalCapacity * 3 / 10;
-      createFile(totalUsedSpace / numOfDatanodes, (short) numOfDatanodes);
+      createFile(cluster, filePath, totalUsedSpace / numOfDatanodes,
+          (short) numOfDatanodes, 0);
       // start up an empty node with the same capacity and on the same rack
       cluster.startDataNodes(conf, 1, true, null, new String[] { newRack },
           new long[] { newCapacity });
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
new file mode 100644
index 0000000..9d13a2b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
@@ -0,0 +1,108 @@
+/**
+ * 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.balancer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.net.URI;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.junit.Test;
+
+/**
+ * Test balancer with HA NameNodes
+ */
+public class TestBalancerWithHANameNodes {
+  private MiniDFSCluster cluster;
+  ClientProtocol client;
+
+  static {
+    Balancer.setBlockMoveWaitTime(1000L);
+  }
+
+  /**
+   * Test a cluster with even distribution, then a new empty node is added to
+   * the cluster. Test start a cluster with specified number of nodes, and fills
+   * it to be 30% full (with a single file replicated identically to all
+   * datanodes); It then adds one new empty node and starts balancing.
+   */
+  @Test(timeout = 60000)
+  public void testBalancerWithHANameNodes() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    TestBalancer.initConf(conf);
+    long newNodeCapacity = TestBalancer.CAPACITY; // new node's capacity
+    String newNodeRack = TestBalancer.RACK2; // new node's rack
+    // array of racks for original nodes in cluster
+    String[] racks = new String[] { TestBalancer.RACK0, TestBalancer.RACK1 };
+    // array of capacities of original nodes in cluster
+    long[] capacities = new long[] { TestBalancer.CAPACITY,
+        TestBalancer.CAPACITY };
+    assertEquals(capacities.length, racks.length);
+    int numOfDatanodes = capacities.length;
+    NNConf nn1Conf = new MiniDFSNNTopology.NNConf("nn1");
+    nn1Conf.setIpcPort(NameNode.DEFAULT_PORT);
+    Configuration copiedConf = new Configuration(conf);
+    cluster = new MiniDFSCluster.Builder(copiedConf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(capacities.length)
+        .racks(racks)
+        .simulatedCapacities(capacities)
+        .build();
+    HATestUtil.setFailoverConfigurations(cluster, conf);
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(1);
+      Thread.sleep(500);
+      client = NameNodeProxies.createProxy(conf, FileSystem.getDefaultUri(conf),
+          ClientProtocol.class).getProxy();
+      long totalCapacity = TestBalancer.sum(capacities);
+      // fill up the cluster to be 30% full
+      long totalUsedSpace = totalCapacity * 3 / 10;
+      TestBalancer.createFile(cluster, TestBalancer.filePath, totalUsedSpace
+          / numOfDatanodes, (short) numOfDatanodes, 1);
+
+      // start up an empty node with the same capacity and on the same rack
+      cluster.startDataNodes(conf, 1, true, null, new String[] { newNodeRack },
+          new long[] { newNodeCapacity });
+      totalCapacity += newNodeCapacity;
+      TestBalancer.waitForHeartBeat(totalUsedSpace, totalCapacity, client,
+          cluster);
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      assertEquals(1, namenodes.size());
+      assertTrue(namenodes.contains(HATestUtil.getLogicalUri(cluster)));
+      final int r = Balancer.run(namenodes, Balancer.Parameters.DEFALUT, conf);
+      assertEquals(Balancer.ReturnStatus.SUCCESS.code, r);
+      TestBalancer.waitForBalancer(totalUsedSpace, totalCapacity, client,
+          cluster);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
index 6d06da4..b130e02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithMultipleNameNodes.java
@@ -18,8 +18,9 @@
 package org.apache.hadoop.hdfs.server.balancer;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
+import java.net.URI;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 
@@ -34,12 +35,13 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -155,7 +157,7 @@
     LOG.info("BALANCER 1");
 
     // start rebalancing
-    final List<InetSocketAddress> namenodes = DFSUtil.getNNServiceRpcAddresses(s.conf);
+    final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(s.conf);
     final int r = Balancer.run(namenodes, Balancer.Parameters.DEFALUT, s.conf);
     Assert.assertEquals(Balancer.ReturnStatus.SUCCESS.code, r);
 
@@ -249,8 +251,9 @@
     final ExtendedBlock[][] blocks;
     {
       LOG.info("UNEVEN 1");
-      final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-          .numNameNodes(nNameNodes)
+      final MiniDFSCluster cluster = new MiniDFSCluster
+          .Builder(new Configuration(conf))
+          .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
           .numDataNodes(nDataNodes)
           .racks(racks)
           .simulatedCapacities(capacities)
@@ -258,6 +261,7 @@
       LOG.info("UNEVEN 2");
       try {
         cluster.waitActive();
+        DFSTestUtil.setFederatedConfiguration(cluster, conf);
         LOG.info("UNEVEN 3");
         final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
         blocks = generateBlocks(s, usedSpacePerNN);
@@ -271,7 +275,7 @@
     {
       LOG.info("UNEVEN 10");
       final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-          .numNameNodes(nNameNodes)
+          .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
           .numDataNodes(nDataNodes)
           .racks(racks)
           .simulatedCapacities(capacities)
@@ -324,13 +328,15 @@
     Assert.assertEquals(nDataNodes, racks.length);
 
     LOG.info("RUN_TEST -1");
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .numNameNodes(nNameNodes)
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(new Configuration(conf))
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
         .numDataNodes(nDataNodes)
         .racks(racks)
         .simulatedCapacities(capacities)
         .build();
     LOG.info("RUN_TEST 0");
+    DFSTestUtil.setFederatedConfiguration(cluster, conf);
 
     try {
       cluster.waitActive();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 8c9b4b3..1ec7511 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -24,8 +24,11 @@
 import java.util.Set;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.Daemon;
+import org.junit.Assert;
 
 import com.google.common.base.Preconditions;
 
@@ -124,6 +127,58 @@
     return blockManager.computeDatanodeWork();
   }
   
+  public static int computeInvalidationWork(BlockManager bm) {
+    return bm.computeInvalidateWork(Integer.MAX_VALUE);
+  }
+  
+  /**
+   * Compute all the replication and invalidation work for the
+   * given BlockManager.
+   * 
+   * This differs from the above functions in that it computes
+   * replication work for all DNs rather than a particular subset,
+   * regardless of invalidation/replication limit configurations.
+   * 
+   * NB: you may want to set
+   * {@link DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY} to
+   * a high value to ensure that all work is calculated.
+   */
+  public static int computeAllPendingWork(BlockManager bm)
+    throws IOException {
+    int work = computeInvalidationWork(bm);
+    work += bm.computeReplicationWork(Integer.MAX_VALUE);
+    return work;
+  }
+
+  /**
+   * Ensure that the given NameNode marks the specified DataNode as
+   * entirely dead/expired.
+   * @param nn the NameNode to manipulate
+   * @param dnName the name of the DataNode
+   */
+  public static void noticeDeadDatanode(NameNode nn, String dnName) {
+    FSNamesystem namesystem = nn.getNamesystem();
+    namesystem.writeLock();
+    try {
+      DatanodeManager dnm = namesystem.getBlockManager().getDatanodeManager();
+      HeartbeatManager hbm = dnm.getHeartbeatManager();
+      DatanodeDescriptor[] dnds = hbm.getDatanodes();
+      DatanodeDescriptor theDND = null;
+      for (DatanodeDescriptor dnd : dnds) {
+        if (dnd.getName().equals(dnName)) {
+          theDND = dnd;
+        }
+      }
+      Assert.assertNotNull("Could not find DN with name: " + dnName, theDND);
+      
+      synchronized (hbm) {
+        theDND.setLastUpdate(0);
+        hbm.heartbeatCheck();
+      }
+    } finally {
+      namesystem.writeUnlock();
+    }
+  }
   
   /**
    * Change whether the block placement policy will prefer the writer's
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
index c18a5c04..2d7a122 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
@@ -41,7 +41,7 @@
 public class TestHeartbeatHandling extends TestCase {
   /**
    * Test if
-   * {@link FSNamesystem#handleHeartbeat(DatanodeRegistration, long, long, long, long, int, int)}
+   * {@link FSNamesystem#handleHeartbeat}
    * can pick up replication and/or invalidate requests and observes the max
    * limit
    */
@@ -75,7 +75,8 @@
             dd.addBlockToBeReplicated(
                 new Block(i, 0, GenerationStamp.FIRST_VALID_STAMP), ONE_TARGET);
           }
-          DatanodeCommand[]cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem);
+          DatanodeCommand[] cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd,
+              namesystem).getCommands();
           assertEquals(1, cmds.length);
           assertEquals(DatanodeProtocol.DNA_TRANSFER, cmds[0].getAction());
           assertEquals(MAX_REPLICATE_LIMIT, ((BlockCommand)cmds[0]).getBlocks().length);
@@ -85,27 +86,31 @@
             blockList.add(new Block(i, 0, GenerationStamp.FIRST_VALID_STAMP));
           }
           dd.addBlocksToBeInvalidated(blockList);
-          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem);
+          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem)
+              .getCommands();
           assertEquals(2, cmds.length);
           assertEquals(DatanodeProtocol.DNA_TRANSFER, cmds[0].getAction());
           assertEquals(MAX_REPLICATE_LIMIT, ((BlockCommand)cmds[0]).getBlocks().length);
           assertEquals(DatanodeProtocol.DNA_INVALIDATE, cmds[1].getAction());
           assertEquals(MAX_INVALIDATE_LIMIT, ((BlockCommand)cmds[1]).getBlocks().length);
           
-          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem);
+          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem)
+              .getCommands();
           assertEquals(2, cmds.length);
           assertEquals(DatanodeProtocol.DNA_TRANSFER, cmds[0].getAction());
           assertEquals(REMAINING_BLOCKS, ((BlockCommand)cmds[0]).getBlocks().length);
           assertEquals(DatanodeProtocol.DNA_INVALIDATE, cmds[1].getAction());
           assertEquals(MAX_INVALIDATE_LIMIT, ((BlockCommand)cmds[1]).getBlocks().length);
           
-          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem);
+          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem)
+              .getCommands();
           assertEquals(1, cmds.length);
           assertEquals(DatanodeProtocol.DNA_INVALIDATE, cmds[0].getAction());
           assertEquals(REMAINING_BLOCKS, ((BlockCommand)cmds[0]).getBlocks().length);
 
-          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem);
-          assertEquals(null, cmds);
+          cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem)
+              .getCommands();
+          assertEquals(0, cmds.length);
         }
       } finally {
         namesystem.writeUnlock();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index 986ca13..d47f110 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -81,15 +81,8 @@
       DataNodeProperties dnprop = cluster.stopDataNode(datanode.getName());
       
       // make sure that NN detects that the datanode is down
-      try {
-        namesystem.writeLock();
-        synchronized (hm) {
-          datanode.setLastUpdate(0); // mark it dead
-          hm.heartbeatCheck();
-        }
-      } finally {
-        namesystem.writeUnlock();
-      }
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(), datanode.getName());
       
       // the block will be replicated
       DFSTestUtil.waitReplication(fs, FILE_PATH, REPLICATION_FACTOR);
@@ -121,16 +114,8 @@
       // bring down non excessive datanode
       dnprop = cluster.stopDataNode(nonExcessDN.getName());
       // make sure that NN detects that the datanode is down
-      
-      try {
-        namesystem.writeLock();
-        synchronized(hm) {
-          nonExcessDN.setLastUpdate(0); // mark it dead
-          hm.heartbeatCheck();
-        }
-      } finally {
-        namesystem.writeUnlock();
-      }
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(), nonExcessDN.getName());
 
       // The block should be replicated
       initializeTimeout(TIMEOUT);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java
new file mode 100644
index 0000000..16977bb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java
@@ -0,0 +1,68 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.util.Queue;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+
+
+public class TestPendingDataNodeMessages {
+  PendingDataNodeMessages msgs = new PendingDataNodeMessages();
+  
+  private final Block block1Gs1 = new Block(1, 0, 1);
+  private final Block block1Gs2 = new Block(1, 0, 2);
+  private final Block block1Gs2DifferentInstance =
+    new Block(1, 0, 2);
+  private final Block block2Gs1 = new Block(2, 0, 1);
+  
+  private final DatanodeDescriptor fakeDN = new DatanodeDescriptor(
+      new DatanodeID("fake"));
+  
+  @Test
+  public void testQueues() {
+    msgs.enqueueReportedBlock(fakeDN, block1Gs1, ReplicaState.FINALIZED);
+    msgs.enqueueReportedBlock(fakeDN, block1Gs2, ReplicaState.FINALIZED);
+
+    assertEquals(2, msgs.count());
+    
+    // Nothing queued yet for block 2
+    assertNull(msgs.takeBlockQueue(block2Gs1));
+    assertEquals(2, msgs.count());
+    
+    Queue<ReportedBlockInfo> q =
+      msgs.takeBlockQueue(block1Gs2DifferentInstance);
+    assertEquals(
+        "ReportedBlockInfo [block=blk_1_1, dn=fake, reportedState=FINALIZED]," +
+        "ReportedBlockInfo [block=blk_1_2, dn=fake, reportedState=FINALIZED]",
+        Joiner.on(",").join(q));
+    assertEquals(0, msgs.count());
+    
+    // Should be null if we pull again
+    assertNull(msgs.takeBlockQueue(block1Gs1));
+    assertEquals(0, msgs.count());
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java
index fb015a2..6ab878c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeAdapter.java
@@ -50,6 +50,29 @@
       boolean heartbeatsDisabledForTests) {
     dn.setHeartbeatsDisabledForTests(heartbeatsDisabledForTests);
   }
+
+  public static void triggerDeletionReport(DataNode dn) throws IOException {
+    for (BPOfferService bpos : dn.getAllBpOs()) {
+      bpos.triggerDeletionReportForTests();
+    }
+  }
+
+  public static void triggerHeartbeat(DataNode dn) throws IOException {
+    for (BPOfferService bpos : dn.getAllBpOs()) {
+      bpos.triggerHeartbeatForTests();
+    }
+  }
+  
+  public static void triggerBlockReport(DataNode dn) throws IOException {
+    for (BPOfferService bpos : dn.getAllBpOs()) {
+      bpos.triggerBlockReportForTests();
+    }
+  }
+
+  public static long getPendingAsyncDeletions(DataNode dn) {
+    FSDataset fsd = (FSDataset)dn.getFSDataset();
+    return fsd.asyncDiskService.countPendingDeletions();
+  }
   
   /**
    * Insert a Mockito spy object between the given DataNode and
@@ -69,10 +92,20 @@
     }
     Preconditions.checkArgument(bpos != null,
         "No such bpid: %s", bpid);
+    
+    BPServiceActor bpsa = null;
+    for (BPServiceActor thisBpsa : bpos.getBPServiceActors()) {
+      if (thisBpsa.getNNSocketAddress().equals(nn.getServiceRpcAddress())) {
+        bpsa = thisBpsa;
+        break;
+      }
+    }
+    Preconditions.checkArgument(bpsa != null,
+      "No service actor to NN at %s", nn.getServiceRpcAddress());
 
-    DatanodeProtocolClientSideTranslatorPB origNN = bpos.getBpNamenode();
+    DatanodeProtocolClientSideTranslatorPB origNN = bpsa.getNameNodeProxy();
     DatanodeProtocolClientSideTranslatorPB spy = Mockito.spy(origNN);
-    bpos.setBpNamenode(spy);
+    bpsa.setNameNode(spy);
     return spy;
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
new file mode 100644
index 0000000..41e7c8b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -0,0 +1,373 @@
+/**
+ * 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.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
+import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+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.NNHAStatusHeartbeat;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat.State;
+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.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class TestBPOfferService {
+
+  private static final String FAKE_BPID = "fake bpid";
+  private static final String FAKE_CLUSTERID = "fake cluster";
+  protected static final Log LOG = LogFactory.getLog(
+      TestBPOfferService.class);
+  private static final ExtendedBlock FAKE_BLOCK =
+    new ExtendedBlock(FAKE_BPID, 12345L);
+
+  static {
+    ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  private DatanodeProtocolClientSideTranslatorPB mockNN1;
+  private DatanodeProtocolClientSideTranslatorPB mockNN2;
+  private NNHAStatusHeartbeat[] mockHaStatuses = new NNHAStatusHeartbeat[2];
+  private int heartbeatCounts[] = new int[2];
+  private DataNode mockDn;
+  private FSDatasetInterface mockFSDataset;
+  
+  @Before
+  public void setupMocks() throws Exception {
+    mockNN1 = setupNNMock(0);
+    mockNN2 = setupNNMock(1);
+
+    // Set up a mock DN with the bare-bones configuration
+    // objects, etc.
+    mockDn = Mockito.mock(DataNode.class);
+    Mockito.doReturn(true).when(mockDn).shouldRun();
+    Configuration conf = new Configuration();
+    Mockito.doReturn(conf).when(mockDn).getConf();
+    Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
+    Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))
+    .when(mockDn).getMetrics();
+
+    // Set up a simulated dataset with our fake BP
+    mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, null, conf));
+    mockFSDataset.addBlockPool(FAKE_BPID, conf);
+
+    // Wire the dataset to the DN.
+    Mockito.doReturn(mockFSDataset).when(mockDn).getFSDataset();
+  }
+
+  /**
+   * Set up a mock NN with the bare minimum for a DN to register to it.
+   */
+  private DatanodeProtocolClientSideTranslatorPB setupNNMock(int nnIdx)
+      throws Exception {
+    DatanodeProtocolClientSideTranslatorPB mock =
+        Mockito.mock(DatanodeProtocolClientSideTranslatorPB.class);
+    Mockito.doReturn(
+        new NamespaceInfo(1, FAKE_CLUSTERID, FAKE_BPID,
+            0, HdfsConstants.LAYOUT_VERSION))
+      .when(mock).versionRequest();
+    
+    Mockito.doReturn(new DatanodeRegistration("fake-node"))
+      .when(mock).registerDatanode(Mockito.any(DatanodeRegistration.class),
+          Mockito.any(DatanodeStorage[].class));
+    
+    Mockito.doAnswer(new HeartbeatAnswer(nnIdx))
+      .when(mock).sendHeartbeat(
+          Mockito.any(DatanodeRegistration.class),
+          Mockito.any(StorageReport[].class),
+          Mockito.anyInt(),
+          Mockito.anyInt(),
+          Mockito.anyInt());
+    mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(State.STANDBY, 0);
+    return mock;
+  }
+  
+  /**
+   * Mock answer for heartbeats which returns an empty set of commands
+   * and the HA status for the chosen NN from the
+   * {@link TestBPOfferService#mockHaStatuses} array.
+   */
+  private class HeartbeatAnswer implements Answer<HeartbeatResponse> {
+    private final int nnIdx;
+
+    public HeartbeatAnswer(int nnIdx) {
+      this.nnIdx = nnIdx;
+    }
+
+    @Override
+    public HeartbeatResponse answer(InvocationOnMock invocation) throws Throwable {
+      heartbeatCounts[nnIdx]++;
+      return new HeartbeatResponse(new DatanodeCommand[0],
+          mockHaStatuses[nnIdx]);
+    }
+  }
+
+
+  /**
+   * Test that the BPOS can register to talk to two different NNs,
+   * sends block reports to both, etc.
+   */
+  @Test
+  public void testBasicFunctionality() throws Exception {
+    BPOfferService bpos = setupBPOSForNNs(mockNN1, mockNN2);
+    bpos.start();
+    try {
+      waitForInitialization(bpos);
+      
+      // The DN should have register to both NNs.
+      Mockito.verify(mockNN1).registerDatanode(Mockito.any(DatanodeRegistration.class),
+          Mockito.any(DatanodeStorage[].class));
+      Mockito.verify(mockNN2).registerDatanode(Mockito.any(DatanodeRegistration.class),
+          Mockito.any(DatanodeStorage[].class));
+      
+      // Should get block reports from both NNs
+      waitForBlockReport(mockNN1);
+      waitForBlockReport(mockNN2);
+
+      // When we receive a block, it should report it to both NNs
+      bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "");
+
+      ReceivedDeletedBlockInfo[] ret = waitForBlockReceived(FAKE_BLOCK, mockNN1);
+      assertEquals(1, ret.length);
+      assertEquals(FAKE_BLOCK.getLocalBlock(), ret[0].getBlock());
+      
+      ret = waitForBlockReceived(FAKE_BLOCK, mockNN2);
+      assertEquals(1, ret.length);
+      assertEquals(FAKE_BLOCK.getLocalBlock(), ret[0].getBlock());
+
+    } finally {
+      bpos.stop();
+    }
+  }
+
+  /**
+   * Test that DNA_INVALIDATE commands from the standby are ignored.
+   */
+  @Test
+  public void testIgnoreDeletionsFromNonActive() throws Exception {
+    BPOfferService bpos = setupBPOSForNNs(mockNN1, mockNN2);
+
+    // Ask to invalidate FAKE_BLOCK when block report hits the
+    // standby
+    Mockito.doReturn(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
+        FAKE_BPID, new Block[] { FAKE_BLOCK.getLocalBlock() }))
+        .when(mockNN2).blockReport(
+            Mockito.<DatanodeRegistration>anyObject(),  
+            Mockito.eq(FAKE_BPID),
+            Mockito.<StorageBlockReport[]>anyObject());
+
+    bpos.start();
+    try {
+      waitForInitialization(bpos);
+      
+      // Should get block reports from both NNs
+      waitForBlockReport(mockNN1);
+      waitForBlockReport(mockNN2);
+
+    } finally {
+      bpos.stop();
+    }
+    
+    // Should ignore the delete command from the standby
+    Mockito.verify(mockFSDataset, Mockito.never())
+      .invalidate(Mockito.eq(FAKE_BPID),
+          (Block[]) Mockito.anyObject());
+  }
+
+  /**
+   * Ensure that, if the two NNs configured for a block pool
+   * have different block pool IDs, they will refuse to both
+   * register.
+   */
+  @Test
+  public void testNNsFromDifferentClusters() throws Exception {
+    Mockito.doReturn(
+        new NamespaceInfo(1, "fake foreign cluster", FAKE_BPID,
+            0, HdfsConstants.LAYOUT_VERSION))
+      .when(mockNN1).versionRequest();
+        
+    BPOfferService bpos = setupBPOSForNNs(mockNN1, mockNN2);
+    bpos.start();
+    try {
+      waitForOneToFail(bpos);
+    } finally {
+      bpos.stop();
+    }
+  }
+  
+  /**
+   * Test that the DataNode determines the active NameNode correctly
+   * based on the HA-related information in heartbeat responses.
+   * See HDFS-2627.
+   */
+  @Test
+  public void testPickActiveNameNode() throws Exception {
+    BPOfferService bpos = setupBPOSForNNs(mockNN1, mockNN2);
+    bpos.start();
+    try {
+      waitForInitialization(bpos);
+      
+      // Should start with neither NN as active.
+      assertNull(bpos.getActiveNN());
+
+      // Have NN1 claim active at txid 1
+      mockHaStatuses[0] = new NNHAStatusHeartbeat(State.ACTIVE, 1);
+      bpos.triggerHeartbeatForTests();
+      assertSame(mockNN1, bpos.getActiveNN());
+
+      // NN2 claims active at a higher txid
+      mockHaStatuses[1] = new NNHAStatusHeartbeat(State.ACTIVE, 2);
+      bpos.triggerHeartbeatForTests();
+      assertSame(mockNN2, bpos.getActiveNN());
+      
+      // Even after another heartbeat from the first NN, it should
+      // think NN2 is active, since it claimed a higher txid
+      bpos.triggerHeartbeatForTests();
+      assertSame(mockNN2, bpos.getActiveNN());
+      
+      // Even if NN2 goes to standby, DN shouldn't reset to talking to NN1,
+      // because NN1's txid is lower than the last active txid. Instead,
+      // it should consider neither active.
+      mockHaStatuses[1] = new NNHAStatusHeartbeat(State.STANDBY, 2);
+      bpos.triggerHeartbeatForTests();
+      assertNull(bpos.getActiveNN());
+      
+      // Now if NN1 goes back to a higher txid, it should be considered active
+      mockHaStatuses[0] = new NNHAStatusHeartbeat(State.ACTIVE, 3);
+      bpos.triggerHeartbeatForTests();
+      assertSame(mockNN1, bpos.getActiveNN());
+
+    } finally {
+      bpos.stop();
+    }
+  }
+
+  private void waitForOneToFail(final BPOfferService bpos)
+      throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return bpos.countNameNodes() == 1;
+      }
+    }, 100, 10000);
+  }
+
+  /**
+   * Create a BPOfferService which registers with and heartbeats with the
+   * specified namenode proxy objects.
+   * @throws IOException 
+   */
+  private BPOfferService setupBPOSForNNs(
+      DatanodeProtocolClientSideTranslatorPB ... nns) throws IOException {
+    // Set up some fake InetAddresses, then override the connectToNN
+    // function to return the corresponding proxies.
+
+    final Map<InetSocketAddress, DatanodeProtocolClientSideTranslatorPB> nnMap = Maps.newLinkedHashMap();
+    for (int port = 0; port < nns.length; port++) {
+      nnMap.put(new InetSocketAddress(port), nns[port]);
+      Mockito.doReturn(nns[port]).when(mockDn).connectToNN(
+          Mockito.eq(new InetSocketAddress(port)));
+    }
+
+    return new BPOfferService(Lists.newArrayList(nnMap.keySet()), mockDn);
+  }
+
+  private void waitForInitialization(final BPOfferService bpos)
+      throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return bpos.isAlive() && bpos.isInitialized();
+      }
+    }, 100, 10000);
+  }
+  
+  private void waitForBlockReport(final DatanodeProtocolClientSideTranslatorPB mockNN)
+      throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          Mockito.verify(mockNN).blockReport(
+              Mockito.<DatanodeRegistration>anyObject(),  
+              Mockito.eq(FAKE_BPID),
+              Mockito.<StorageBlockReport[]>anyObject());
+          return true;
+        } catch (Throwable t) {
+          LOG.info("waiting on block report: " + t.getMessage());
+          return false;
+        }
+      }
+    }, 500, 10000);
+  }
+  
+  private ReceivedDeletedBlockInfo[] waitForBlockReceived(
+      ExtendedBlock fakeBlock,
+      DatanodeProtocolClientSideTranslatorPB mockNN) throws Exception {
+    final ArgumentCaptor<StorageReceivedDeletedBlocks[]> captor =
+      ArgumentCaptor.forClass(StorageReceivedDeletedBlocks[].class);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+
+      @Override
+      public Boolean get() {
+        try {
+          Mockito.verify(mockNN1).blockReceivedAndDeleted(
+            Mockito.<DatanodeRegistration>anyObject(),
+            Mockito.eq(FAKE_BPID),
+            captor.capture());
+          return true;
+        } catch (Throwable t) {
+          return false;
+        }
+      }
+    }, 100, 10000);
+    return captor.getValue()[0].getBlocks();
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java
new file mode 100644
index 0000000..c0301ac
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockPoolManager.java
@@ -0,0 +1,138 @@
+/**
+ * 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.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+
+public class TestBlockPoolManager {
+  private Log LOG = LogFactory.getLog(TestBlockPoolManager.class);
+  private DataNode mockDN = Mockito.mock(DataNode.class);
+  private BlockPoolManager bpm;
+  private StringBuilder log = new StringBuilder();
+  private int mockIdx = 1;
+  
+  @Before
+  public void setupBPM() {
+    bpm = new BlockPoolManager(mockDN){
+
+      @Override
+      protected BPOfferService createBPOS(List<InetSocketAddress> nnAddrs) {
+        final int idx = mockIdx++;
+        doLog("create #" + idx);
+        final BPOfferService bpos = Mockito.mock(BPOfferService.class);
+        Mockito.doReturn("Mock BPOS #" + idx).when(bpos).toString();
+        // Log refreshes
+        try {
+          Mockito.doAnswer(
+              new Answer<Void>() {
+                @Override
+                public Void answer(InvocationOnMock invocation) throws Throwable {
+                  doLog("refresh #" + idx);
+                  return null;
+                }
+              }).when(bpos).refreshNNList(
+                  Mockito.<ArrayList<InetSocketAddress>>any());
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        // Log stops
+        Mockito.doAnswer(
+            new Answer<Void>() {
+              @Override
+              public Void answer(InvocationOnMock invocation) throws Throwable {
+                doLog("stop #" + idx);
+                bpm.remove(bpos);
+                return null;
+              }
+            }).when(bpos).stop();
+        return bpos;
+      }
+    };
+  }
+  
+  private void doLog(String string) {
+    synchronized(log) {
+      LOG.info(string);
+      log.append(string).append("\n");
+    }
+  }
+
+  @Test
+  public void testSimpleSingleNS() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY,
+        "hdfs://mock1:8020");
+    bpm.refreshNamenodes(conf);
+    assertEquals("create #1\n", log.toString());
+  }
+
+  @Test
+  public void testFederationRefresh() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES,
+        "ns1,ns2");
+    addNN(conf, "ns1", "mock1:8020");
+    addNN(conf, "ns2", "mock1:8020");
+    bpm.refreshNamenodes(conf);
+    assertEquals(
+        "create #1\n" +
+        "create #2\n", log.toString());
+    log.setLength(0);
+
+    // Remove the first NS
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES,
+        "ns1");
+    bpm.refreshNamenodes(conf);
+    assertEquals(
+        "stop #1\n" +
+        "refresh #2\n", log.toString());
+    log.setLength(0);
+    
+    // Add back an NS -- this creates a new BPOS since the old
+    // one for ns2 should have been previously retired
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES,
+        "ns1,ns2");
+    bpm.refreshNamenodes(conf);
+    assertEquals(
+        "create #3\n" +
+        "refresh #2\n", log.toString());
+  }
+
+  private static void addNN(Configuration conf, String ns, String addr) {
+    String key = DFSUtil.addKeySuffixes(
+        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, ns);
+    conf.set(key, addr);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index cb42441..59a61cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -42,10 +42,13 @@
 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.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Daemon;
@@ -137,7 +140,9 @@
             Mockito.anyInt(),
             Mockito.anyInt(),
             Mockito.anyInt()))
-        .thenReturn(new DatanodeCommand[0]);
+        .thenReturn(new HeartbeatResponse(
+            new DatanodeCommand[0],
+            new NNHAStatusHeartbeat(State.ACTIVE, 1)));
 
     dn = new DataNode(conf, dirs, null) {
       @Override
@@ -147,14 +152,8 @@
         return namenode;
       }
     };
-    dn.runDatanodeDaemon();
-    while (!dn.isDatanodeFullyStarted()) {
-      try {
-        Thread.sleep(50);
-      } catch (InterruptedException e) {
-        fail("Interrupted starting DN");
-      }
-    }
+    // Trigger a heartbeat so that it acknowledges the NN as active.
+    dn.getAllBpOs()[0].triggerHeartbeatForTests();
   }
 
   /**
@@ -462,7 +461,7 @@
         initReplicaRecovery(any(RecoveringBlock.class));
     Daemon d = spyDN.recoverBlocks(initRecoveringBlocks());
     d.join();
-    DatanodeProtocol dnP = dn.getBPNamenode(POOL_ID);
+    DatanodeProtocol dnP = dn.getActiveNamenodeForBP(POOL_ID);
     verify(dnP).commitBlockSynchronization(
         block, RECOVERY_ID, 0, true, true, DatanodeID.EMPTY_ARRAY);
   }
@@ -519,7 +518,7 @@
     } catch (IOException e) {
       e.getMessage().startsWith("Cannot recover ");
     }
-    DatanodeProtocol namenode = dn.getBPNamenode(POOL_ID);
+    DatanodeProtocol namenode = dn.getActiveNamenodeForBP(POOL_ID);
     verify(namenode, never()).commitBlockSynchronization(
         any(ExtendedBlock.class), anyLong(), anyLong(), anyBoolean(),
         anyBoolean(), any(DatanodeID[].class));
@@ -548,7 +547,7 @@
       } catch (IOException e) {
         e.getMessage().startsWith("Cannot recover ");
       }
-      DatanodeProtocol namenode = dn.getBPNamenode(POOL_ID);
+      DatanodeProtocol namenode = dn.getActiveNamenodeForBP(POOL_ID);
       verify(namenode, never()).commitBlockSynchronization(
           any(ExtendedBlock.class), anyLong(), anyLong(), anyBoolean(),
           anyBoolean(), any(DatanodeID[].class));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
index b7a1017..0faa5b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeExit.java
@@ -28,6 +28,7 @@
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -36,7 +37,6 @@
  * Tests if DataNode process exits if all Block Pool services exit. 
  */
 public class TestDataNodeExit {
-  private static int BASEPORT = 9923;
   private static long WAIT_TIME_IN_MILLIS = 10;
   Configuration conf;
   MiniDFSCluster cluster = null;
@@ -46,8 +46,9 @@
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 100);
-    cluster = new MiniDFSCluster.Builder(conf).numNameNodes(3)
-        .nameNodePort(BASEPORT).build();
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
+      .build();
     for (int i = 0; i < 3; i++) {
       cluster.waitActive(i);
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
index 4a84ce8..20a16c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
@@ -23,6 +23,8 @@
 import static org.junit.Assert.assertNotSame;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
@@ -30,6 +32,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -55,8 +58,9 @@
    */
   @Test
   public void test2NNRegistration() throws IOException {
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numNameNodes(2)
-        .nameNodePort(9928).build();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .build();
     try {
       cluster.waitActive();
       NameNode nn1 = cluster.getNameNode(0);
@@ -90,23 +94,22 @@
       assertEquals("number of volumes is wrong", 2, volInfos.size());
 
       for (BPOfferService bpos : dn.getAllBpOs()) {
-        LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration.name + "; sid="
-            + bpos.bpRegistration.storageID + "; nna=" + bpos.nnAddr);
+        LOG.info("BP: " + bpos);
       }
 
       BPOfferService bpos1 = dn.getAllBpOs()[0];
       BPOfferService bpos2 = dn.getAllBpOs()[1];
 
       // The order of bpos is not guaranteed, so fix the order
-      if (bpos1.nnAddr.equals(nn2.getNameNodeAddress())) {
+      if (getNNSocketAddress(bpos1).equals(nn2.getNameNodeAddress())) {
         BPOfferService tmp = bpos1;
         bpos1 = bpos2;
         bpos2 = tmp;
       }
 
-      assertEquals("wrong nn address", bpos1.nnAddr,
+      assertEquals("wrong nn address", getNNSocketAddress(bpos1),
           nn1.getNameNodeAddress());
-      assertEquals("wrong nn address", bpos2.nnAddr,
+      assertEquals("wrong nn address", getNNSocketAddress(bpos2),
           nn2.getNameNodeAddress());
       assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1);
       assertEquals("wrong bpid", bpos2.getBlockPoolId(), bpid2);
@@ -120,6 +123,12 @@
       cluster.shutdown();
     }
   }
+  
+  private static InetSocketAddress getNNSocketAddress(BPOfferService bpos) {
+    List<BPServiceActor> actors = bpos.getBPServiceActors();
+    assertEquals(1, actors.size());
+    return actors.get(0).getNNSocketAddress();
+  }
 
   /**
    * starts single nn and single dn and verifies registration and handshake
@@ -153,15 +162,16 @@
 
       for (BPOfferService bpos : dn.getAllBpOs()) {
         LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration.name + "; sid="
-            + bpos.bpRegistration.storageID + "; nna=" + bpos.nnAddr);
+            + bpos.bpRegistration.storageID + "; nna=" +
+            getNNSocketAddress(bpos));
       }
 
       // try block report
       BPOfferService bpos1 = dn.getAllBpOs()[0];
-      bpos1.lastBlockReport = 0;
-      bpos1.blockReport();
+      bpos1.triggerBlockReportForTests();
 
-      assertEquals("wrong nn address", bpos1.nnAddr,
+      assertEquals("wrong nn address",
+          getNNSocketAddress(bpos1),
           nn1.getNameNodeAddress());
       assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1);
       assertEquals("wrong cid", dn.getClusterId(), cid1);
@@ -179,8 +189,9 @@
   
   @Test
   public void testClusterIdMismatch() throws IOException {
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numNameNodes(2).
-    nameNodePort(9928).build();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .build();
     try {
       cluster.waitActive();
 
@@ -215,25 +226,27 @@
 
     Configuration conf = new HdfsConfiguration();
     // start Federated cluster and add a node.
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numNameNodes(2).
-    nameNodePort(9928).build();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+      .build();
     Assert.assertNotNull(cluster);
     Assert.assertEquals("(1)Should be 2 namenodes", 2, cluster.getNumNameNodes());
     
     // add a node
-    cluster.addNameNode(conf, 9929);
+    cluster.addNameNode(conf, 0);
     Assert.assertEquals("(1)Should be 3 namenodes", 3, cluster.getNumNameNodes());
     cluster.shutdown();
         
     // 2. start with Federation flag set
     conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf).federation(true).
-    nameNodePort(9928).build();
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(1))
+      .build();
     Assert.assertNotNull(cluster);
     Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes());
     
     // add a node
-    cluster.addNameNode(conf, 9929);   
+    cluster.addNameNode(conf, 0);
     Assert.assertEquals("(2)Should be 2 namenodes", 2, cluster.getNumNameNodes());
     cluster.shutdown();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
index ca9b3dc..dbbaedd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
@@ -42,7 +42,10 @@
     DataNode mockDN = mock(DataNode.class);
     Mockito.doReturn(true).when(mockDN).shouldRun();
     
-    BPOfferService bpos = new BPOfferService(INVALID_ADDR, mockDN);
+    BPOfferService mockBPOS = Mockito.mock(BPOfferService.class);
+    Mockito.doReturn(mockDN).when(mockBPOS).getDataNode();
+    
+    BPServiceActor actor = new BPServiceActor(INVALID_ADDR, mockBPOS);
 
     NamespaceInfo fakeNSInfo = mock(NamespaceInfo.class);
     when(fakeNSInfo.getBuildVersion()).thenReturn("NSBuildVersion");
@@ -50,10 +53,9 @@
         mock(DatanodeProtocolClientSideTranslatorPB.class);
     when(fakeDNProt.versionRequest()).thenReturn(fakeNSInfo);
 
-    bpos.setNameNode( fakeDNProt );
-    bpos.bpNSInfo = fakeNSInfo;
+    actor.setNameNode( fakeDNProt );
     try {   
-      bpos.retrieveNamespaceInfo();
+      actor.retrieveNamespaceInfo();
       fail("register() did not throw exception! " +
            "Expected: IncorrectVersionException");
     } catch (IncorrectVersionException ie) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java
index 0b0ca7b..2ff075c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java
@@ -31,6 +31,7 @@
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.junit.Test;
 
@@ -47,8 +48,9 @@
     try {
       conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES,
           "namesServerId1,namesServerId2");
-      cluster = new MiniDFSCluster.Builder(conf).federation(true).numNameNodes(
-          2).numDataNodes(2).build();
+      cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .numDataNodes(2).build();
 
       cluster.waitActive();
 
@@ -155,8 +157,9 @@
     try {
       conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES,
           "namesServerId1,namesServerId2");
-      cluster = new MiniDFSCluster.Builder(conf).federation(true).numNameNodes(
-          2).numDataNodes(1).build();
+      cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .numDataNodes(1).build();
 
       cluster.waitActive();
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMulitipleNNDataBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMulitipleNNDataBlockScanner.java
index 6862628..a21cab5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMulitipleNNDataBlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestMulitipleNNDataBlockScanner.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.junit.Test;
 
 
@@ -41,12 +42,13 @@
   String bpids[] = new String[3];
   FileSystem fs[] = new FileSystem[3];
   
-  public void setUp(int port) throws IOException {
+  public void setUp() throws IOException {
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 100);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 100);
-    cluster = new MiniDFSCluster.Builder(conf).numNameNodes(3).nameNodePort(
-        port).build();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(3))
+        .build();
     for (int i = 0; i < 3; i++) {
       cluster.waitActive(i);
     }
@@ -65,7 +67,7 @@
   
   @Test
   public void testDataBlockScanner() throws IOException, InterruptedException {
-    setUp(9923);
+    setUp();
     try {
       DataNode dn = cluster.getDataNodes().get(0);
       for (int i = 0; i < 3; i++) {
@@ -89,9 +91,10 @@
   @Test
   public void testBlockScannerAfterRefresh() throws IOException,
       InterruptedException {
-    setUp(9933);
+    setUp();
     try {
-      Configuration conf = new HdfsConfiguration(cluster.getConfiguration(0));
+      Configuration dnConf = cluster.getDataNodes().get(0).getConf();
+      Configuration conf = new HdfsConfiguration(dnConf);
       StringBuilder namenodesBuilder = new StringBuilder();
 
       String bpidToShutdown = cluster.getNamesystem(2).getBlockPoolId();
@@ -140,7 +143,7 @@
   @Test
   public void testBlockScannerAfterRestart() throws IOException,
       InterruptedException {
-    setUp(9943);
+    setUp();
     try {
       cluster.restartDataNode(0);
       cluster.waitActive();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
index 150f1178..2d6f2103 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
@@ -22,12 +22,18 @@
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NSConf;
 import org.junit.Test;
 
+import com.google.common.base.Joiner;
+import com.google.common.collect.Sets;
+
 /**
  * Tests datanode refresh namenode list functionality.
  */
@@ -43,9 +49,13 @@
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
-      conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "namesServerId1");
-      cluster = new MiniDFSCluster.Builder(conf).federation(true).
-          numNameNodes(1).nameNodePort(nnPort1).build();
+      MiniDFSNNTopology topology = new MiniDFSNNTopology()
+        .addNameservice(new NSConf("ns1").addNN(
+            new NNConf(null).setIpcPort(nnPort1)))
+        .setFederation(true);
+      cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(topology)
+        .build();
 
       DataNode dn = cluster.getDataNodes().get(0);
       assertEquals(1, dn.getAllBpOs().length);
@@ -58,21 +68,24 @@
 
       cluster.addNameNode(conf, nnPort4);
 
-      BPOfferService[] bpoList = dn.getAllBpOs();
       // Ensure a BPOfferService in the datanodes corresponds to
       // a namenode in the cluster
+      Set<InetSocketAddress> nnAddrsFromCluster = Sets.newHashSet();
       for (int i = 0; i < 4; i++) {
-        InetSocketAddress addr = cluster.getNameNode(i).getNameNodeAddress();
-        boolean found = false;
-        for (int j = 0; j < bpoList.length; j++) {
-          if (bpoList[j] != null && addr.equals(bpoList[j].nnAddr)) {
-            found = true;
-            bpoList[j] = null; // Erase the address that matched
-            break;
-          }
-        }
-        assertTrue("NameNode address " + addr + " is not found.", found);
+        assertTrue(nnAddrsFromCluster.add(
+            cluster.getNameNode(i).getNameNodeAddress()));
       }
+      
+      Set<InetSocketAddress> nnAddrsFromDN = Sets.newHashSet();
+      for (BPOfferService bpos : dn.getAllBpOs()) {
+        for (BPServiceActor bpsa : bpos.getBPServiceActors()) {
+          assertTrue(nnAddrsFromDN.add(bpsa.getNNSocketAddress()));
+        }
+      }
+      
+      assertEquals("",
+          Joiner.on(",").join(
+            Sets.symmetricDifference(nnAddrsFromCluster, nnAddrsFromDN)));
     } finally {
       if (cluster != null) {
         cluster.shutdown();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
index 4d09815..7962d4a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
@@ -195,7 +195,7 @@
 
     FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
     FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
-    editLog.open();
+    editLog.openForWrite();
     addFiles(editLog, numFiles, replication, numBlocksPerFile, startingBlockId,
              nameGenerator);
     editLog.logSync();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
index b6c69c3..6e9aa8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
@@ -34,8 +34,11 @@
 import java.util.Properties;
 import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -187,13 +190,36 @@
     Mockito.doReturn(sd).when(storage)
       .getStorageDirectory(Matchers.<URI>anyObject());
 
-    return new FSEditLog(new Configuration(), 
+    FSEditLog editLog = new FSEditLog(new Configuration(), 
                          storage,
                          ImmutableList.of(logDir.toURI()));
+    editLog.initJournalsForWrite();
+    return editLog;
   }
   
 
   /**
+   * Create an aborted in-progress log in the given directory, containing
+   * only a specified number of "mkdirs" operations.
+   */
+  public static void createAbortedLogWithMkdirs(File editsLogDir, int numDirs,
+      long firstTxId) throws IOException {
+    FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
+    editLog.setNextTxId(firstTxId);
+    editLog.openForWrite();
+    
+    PermissionStatus perms = PermissionStatus.createImmutable("fakeuser", "fakegroup",
+        FsPermission.createImmutable((short)0755));
+    for (int i = 1; i <= numDirs; i++) {
+      String dirName = "dir" + i;
+      INodeDirectory dir = new INodeDirectory(dirName, perms);
+      editLog.logMkDir("/" + dirName, dir);
+    }
+    editLog.logSync();
+    editLog.abortCurrentLogSegment();
+  }
+
+  /**
    * @param editLog a path of an edit log file
    * @return the count of each type of operation in the log file
    * @throws Exception if there is an error reading it
@@ -410,13 +436,20 @@
    * Assert that the NameNode has checkpoints at the expected
    * transaction IDs.
    */
-  static void assertNNHasCheckpoints(MiniDFSCluster cluster,
+  public static void assertNNHasCheckpoints(MiniDFSCluster cluster,
       List<Integer> txids) {
+    assertNNHasCheckpoints(cluster, 0, txids);
+  }
+  
+  public static void assertNNHasCheckpoints(MiniDFSCluster cluster,
+      int nnIdx, List<Integer> txids) {
 
-    for (File nameDir : getNameNodeCurrentDirs(cluster)) {
+    for (File nameDir : getNameNodeCurrentDirs(cluster, nnIdx)) {
       LOG.info("examining name dir with files: " +
           Joiner.on(",").join(nameDir.listFiles()));
       // Should have fsimage_N for the three checkpoints
+      LOG.info("Examining storage dir " + nameDir + " with contents: "
+          + StringUtils.join(nameDir.listFiles(), ", "));
       for (long checkpointTxId : txids) {
         File image = new File(nameDir,
                               NNStorage.getImageFileName(checkpointTxId));
@@ -425,9 +458,9 @@
     }
   }
 
-  public static List<File> getNameNodeCurrentDirs(MiniDFSCluster cluster) {
+  public static List<File> getNameNodeCurrentDirs(MiniDFSCluster cluster, int nnIdx) {
     List<File> nameDirs = Lists.newArrayList();
-    for (URI u : cluster.getNameDirs(0)) {
+    for (URI u : cluster.getNameDirs(nnIdx)) {
       nameDirs.add(new File(u.getPath(), "current"));
     }
     return nameDirs;
@@ -441,7 +474,7 @@
   throws IOException {
     File currentDir = sd.getCurrentDir();
     List<EditLogFile> foundEditLogs 
-      = Lists.newArrayList(FileJournalManager.matchEditLogs(currentDir.listFiles()));
+      = Lists.newArrayList(FileJournalManager.matchEditLogs(currentDir));
     return Collections.max(foundEditLogs, EditLogFile.COMPARE_BY_START_TXID);
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index d128167..7f18811 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -80,7 +80,7 @@
  * <li>-logLevel L specifies the logging level when the benchmark runs.
  * The default logging level is {@link Level#ERROR}.</li>
  * <li>-UGCacheRefreshCount G will cause the benchmark to call
- * {@link NameNode#refreshUserToGroupsMappings()} after
+ * {@link NameNodeRpcServer#refreshUserToGroupsMappings} after
  * every G operations, which purges the name-node's user group cache.
  * By default the refresh is never called.</li>
  * <li>-keepResults do not clean up the name-space after execution.</li>
@@ -813,7 +813,7 @@
       StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(),
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
       DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
-          rep, 0, 0, 0);
+          rep, 0, 0, 0).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -859,7 +859,7 @@
       StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(),
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
       DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
-          rep, 0, 0, 0);
+          rep, 0, 0, 0).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {
@@ -889,8 +889,10 @@
           receivedDNReg.setStorageInfo(
                           new DataStorage(nsInfo, dnInfo.getStorageID()));
           receivedDNReg.setInfoPort(dnInfo.getInfoPort());
-          ReceivedDeletedBlockInfo[] rdBlocks = { new ReceivedDeletedBlockInfo(
-                  blocks[i], DataNode.EMPTY_DEL_HINT) };
+          ReceivedDeletedBlockInfo[] rdBlocks = {
+            new ReceivedDeletedBlockInfo(
+                  blocks[i], ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK,
+                  null) };
           StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
               receivedDNReg.getStorageID(), rdBlocks) };
           nameNodeProto.blockReceivedAndDeleted(receivedDNReg, nameNode
@@ -1007,7 +1009,8 @@
           int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
           datanodes[dnIdx].addBlock(loc.getBlock().getLocalBlock());
           ReceivedDeletedBlockInfo[] rdBlocks = { new ReceivedDeletedBlockInfo(
-              loc.getBlock().getLocalBlock(), "") };
+              loc.getBlock().getLocalBlock(),
+              ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null) };
           StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
               datanodes[dnIdx].dnRegistration.getStorageID(), rdBlocks) };
           nameNodeProto.blockReceivedAndDeleted(datanodes[dnIdx].dnRegistration, loc
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index fb1fc6b..fead3b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -17,16 +17,27 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.File;
 import java.io.IOException;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.SafeModeInfo;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.security.AccessControlException;
+import org.mockito.Mockito;
 
 /**
  * This is a utility class to expose NameNode functionality for unit tests.
@@ -48,6 +59,38 @@
         src, offset, length, false, true);
   }
   
+  public static HdfsFileStatus getFileInfo(NameNode namenode, String src,
+      boolean resolveLink) throws AccessControlException, UnresolvedLinkException,
+        StandbyException {
+    return namenode.getNamesystem().getFileInfo(src, resolveLink);
+  }
+  
+  public static boolean mkdirs(NameNode namenode, String src,
+      PermissionStatus permissions, boolean createParent)
+      throws UnresolvedLinkException, IOException {
+    return namenode.getNamesystem().mkdirs(src, permissions, createParent);
+  }
+  
+  public static void saveNamespace(NameNode namenode)
+      throws AccessControlException, IOException {
+    namenode.getNamesystem().saveNamespace();
+  }
+  
+  public static void enterSafeMode(NameNode namenode, boolean resourcesLow)
+      throws IOException {
+    namenode.getNamesystem().enterSafeMode(resourcesLow);
+  }
+  
+  public static void leaveSafeMode(NameNode namenode, boolean checkForUpgrades)
+      throws SafeModeException {
+    namenode.getNamesystem().leaveSafeMode(checkForUpgrades);
+  }
+  
+  public static void abortEditLogs(NameNode nn) {
+    FSEditLog el = nn.getFSImage().getEditLog();
+    el.abortCurrentLogSegment();
+  }
+  
   /**
    * Get the internal RPC server instance.
    * @return rpc server
@@ -61,7 +104,7 @@
     return ns.getDelegationTokenSecretManager();
   }
 
-  public static DatanodeCommand[] sendHeartBeat(DatanodeRegistration nodeReg,
+  public static HeartbeatResponse sendHeartBeat(DatanodeRegistration nodeReg,
       DatanodeDescriptor dd, FSNamesystem namesystem) throws IOException {
     return namesystem.handleHeartbeat(nodeReg, dd.getCapacity(), 
         dd.getDfsUsed(), dd.getRemaining(), dd.getBlockPoolUsed(), 0, 0, 0);
@@ -79,7 +122,7 @@
   /** Set the softLimit and hardLimit of client lease periods. */
   public static void setLeasePeriod(final FSNamesystem namesystem, long soft, long hard) {
     getLeaseManager(namesystem).setLeasePeriod(soft, hard);
-    namesystem.lmthread.interrupt();
+    namesystem.leaseManager.triggerMonitorCheckNow();
   }
 
   public static String getLeaseHolderForPath(NameNode namenode, String path) {
@@ -87,6 +130,19 @@
   }
 
   /**
+   * @return the timestamp of the last renewal of the given lease,
+   *   or -1 in the case that the lease doesn't exist.
+   */
+  public static long getLeaseRenewalTime(NameNode nn, String path) {
+    LeaseManager lm = nn.getNamesystem().leaseManager;
+    Lease l = lm.getLeaseByPath(path);
+    if (l == null) {
+      return -1;
+    }
+    return l.getLastUpdate();
+  }
+
+  /**
    * Return the datanode descriptor for the given datanode.
    */
   public static DatanodeDescriptor getDatanode(final FSNamesystem ns,
@@ -100,6 +156,33 @@
   }
   
   /**
+   * Return the FSNamesystem stats
+   */
+  public static long[] getStats(final FSNamesystem fsn) {
+    return fsn.getStats();
+  }
+  
+  public static ReentrantReadWriteLock spyOnFsLock(FSNamesystem fsn) {
+    ReentrantReadWriteLock spy = Mockito.spy(fsn.getFsLockForTests());
+    fsn.setFsLockForTests(spy);
+    return spy;
+  }
+
+  public static FSImage spyOnFsImage(NameNode nn1) {
+    FSImage spy = Mockito.spy(nn1.getNamesystem().dir.fsImage);
+    nn1.getNamesystem().dir.fsImage = spy;
+    return spy;
+  }
+  
+  public static String getMkdirOpPath(FSEditLogOp op) {
+    if (op.opCode == FSEditLogOpCodes.OP_MKDIR) {
+      return ((MkdirOp) op).path;
+    } else {
+      return null;
+    }
+  }
+  
+  /**
    * @return the number of blocks marked safe by safemode, or -1
    * if safemode is not running.
    */
@@ -122,4 +205,8 @@
     }
     return smi.initializedReplQueues;
   }
+  
+  public static File getInProgressEditsFile(StorageDirectory sd, long startTxId) {
+    return NNStorage.getInProgressEditsFile(sd, startTxId);
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
index e22fa29..392cc9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
@@ -108,10 +108,11 @@
     // for security to work (fake JobTracker user)
     config.set("hadoop.security.auth_to_local",
       "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
+    config.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     cluster =
       new MiniDFSCluster.Builder(config).manageNameDfsDirs(false).build();
     cluster.waitClusterUp();
-    cluster.getNamesystem().getDelegationTokenSecretManager().startThreads();
   }
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
index 2d8a115..5d93b8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
@@ -33,6 +33,7 @@
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -123,6 +124,7 @@
   @Test
   public void testBackupNodeTailsEdits() throws Exception {
     Configuration conf = new HdfsConfiguration();
+    HAUtil.setAllowStandbyReads(conf, true);
     MiniDFSCluster cluster = null;
     FileSystem fileSys = null;
     BackupNode backup = null;
@@ -244,11 +246,12 @@
   }  
 
   void testCheckpoint(StartupOption op) throws Exception {
-    Path file1 = new Path("checkpoint.dat");
-    Path file2 = new Path("checkpoint2.dat");
-    Path file3 = new Path("backup.dat");
+    Path file1 = new Path("/checkpoint.dat");
+    Path file2 = new Path("/checkpoint2.dat");
+    Path file3 = new Path("/backup.dat");
 
     Configuration conf = new HdfsConfiguration();
+    HAUtil.setAllowStandbyReads(conf, true);
     short replication = (short)conf.getInt("dfs.replication", 3);
     int numDatanodes = Math.max(3, replication);
     conf.set(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, "0");
@@ -345,11 +348,13 @@
       TestCheckpoint.checkFile(fileSys, file3, replication);
       // should also be on BN right away
       assertTrue("file3 does not exist on BackupNode",
-          op != StartupOption.BACKUP || bnFS.exists(file3));
+          op != StartupOption.BACKUP ||
+          backup.getNamesystem().getFileInfo(
+              file3.toUri().getPath(), false) != null);
 
     } catch(IOException e) {
       LOG.error("Error in TestBackupNode:", e);
-      assertTrue(e.getLocalizedMessage(), false);
+      throw new AssertionError(e);
     } finally {
       if(backup != null) backup.stop();
       if(fileSys != null) fileSys.close();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
index fbbcfc7..20d4c72 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
@@ -22,6 +22,7 @@
 import java.io.*;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -57,18 +58,19 @@
   }
   
   /**
-   * Tests save namepsace.
+   * Tests save namespace.
    */
   @Test
   public void testSaveNamespace() throws IOException {
     DistributedFileSystem fs = null;
     try {
       Configuration conf = new HdfsConfiguration();
+      conf.setBoolean(
+          DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
       cluster.waitActive();
       fs = (DistributedFileSystem)(cluster.getFileSystem());
       FSNamesystem namesystem = cluster.getNamesystem();
-      namesystem.getDelegationTokenSecretManager().startThreads();
       String renewer = UserGroupInformation.getLoginUser().getUserName();
       Token<DelegationTokenIdentifier> token1 = namesystem
           .getDelegationToken(new Text(renewer)); 
@@ -122,7 +124,6 @@
       }
 
       namesystem = cluster.getNamesystem();
-      namesystem.getDelegationTokenSecretManager().startThreads();
       Token<DelegationTokenIdentifier> token3 = namesystem
           .getDelegationToken(new Text(renewer));
       Token<DelegationTokenIdentifier> token4 = namesystem
@@ -136,7 +137,6 @@
       cluster.waitActive();
 
       namesystem = cluster.getNamesystem();
-      namesystem.getDelegationTokenSecretManager().startThreads();
       Token<DelegationTokenIdentifier> token5 = namesystem
           .getDelegationToken(new Text(renewer));
 
@@ -159,7 +159,6 @@
       cluster.waitActive();
 
       namesystem = cluster.getNamesystem();
-      namesystem.getDelegationTokenSecretManager().startThreads();
       try {
         renewToken(token1);
         cancelToken(token1);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 19f4812..daed09b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -46,6 +46,7 @@
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -655,6 +656,7 @@
     sdToLock.lock();
     try {      
       MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .format(false)
         .manageNameDfsDirs(false)
         .numDataNodes(0)
         .build();
@@ -861,7 +863,7 @@
   }
 
   /**
-   * Tests save namepsace.
+   * Tests save namespace.
    */
   public void testSaveNamespace() throws IOException {
     MiniDFSCluster cluster = null;
@@ -911,10 +913,12 @@
         throw new IOException(e);
       }
       
+      final int EXPECTED_TXNS_FIRST_SEG = 12;
+      
       // the following steps should have happened:
-      //   edits_inprogress_1 -> edits_1-8  (finalized)
-      //   fsimage_8 created
-      //   edits_inprogress_9 created
+      //   edits_inprogress_1 -> edits_1-12  (finalized)
+      //   fsimage_12 created
+      //   edits_inprogress_13 created
       //
       for(URI uri : editsDirs) {
         File ed = new File(uri.getPath());
@@ -926,19 +930,21 @@
                                       NNStorage.getInProgressEditsFileName(1));
         assertFalse(originalEdits.exists());
         File finalizedEdits = new File(curDir,
-            NNStorage.getFinalizedEditsFileName(1,8));
-        assertTrue(finalizedEdits.exists());
+            NNStorage.getFinalizedEditsFileName(1, EXPECTED_TXNS_FIRST_SEG));
+        GenericTestUtils.assertExists(finalizedEdits);
         assertTrue(finalizedEdits.length() > Integer.SIZE/Byte.SIZE);
 
-        assertTrue(new File(ed, "current/"
-                       + NNStorage.getInProgressEditsFileName(9)).exists());
+        GenericTestUtils.assertExists(new File(ed, "current/"
+                       + NNStorage.getInProgressEditsFileName(
+                           EXPECTED_TXNS_FIRST_SEG + 1)));
       }
       
       Collection<URI> imageDirs = cluster.getNameDirs(0);
       for (URI uri : imageDirs) {
         File imageDir = new File(uri.getPath());
         File savedImage = new File(imageDir, "current/"
-                                   + NNStorage.getImageFileName(8));
+                                   + NNStorage.getImageFileName(
+                                       EXPECTED_TXNS_FIRST_SEG));
         assertTrue("Should have saved image at " + savedImage,
             savedImage.exists());        
       }
@@ -1059,8 +1065,9 @@
     String nameserviceId2 = "ns2";
     conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nameserviceId1
         + "," + nameserviceId2);
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numNameNodes(2)
-        .nameNodePort(9928).build();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
+        .build();
     Configuration snConf1 = new HdfsConfiguration(cluster.getConfiguration(0));
     Configuration snConf2 = new HdfsConfiguration(cluster.getConfiguration(1));
     InetSocketAddress nn1RpcAddress =
@@ -1076,9 +1083,9 @@
     snConf2.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "");
 
     // Set the nameserviceIds
-    snConf1.set(DFSUtil.getNameServiceIdKey(
+    snConf1.set(DFSUtil.addKeySuffixes(
         DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nameserviceId1), nn1);
-    snConf2.set(DFSUtil.getNameServiceIdKey(
+    snConf2.set(DFSUtil.addKeySuffixes(
         DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nameserviceId2), nn2);
 
     SecondaryNameNode secondary1 = startSecondaryNameNode(snConf1);
@@ -1317,17 +1324,11 @@
       // Let the first one finish
       delayer.proceed();
       
-      // Letting the first node continue should catch an exception
+      // Letting the first node continue, it should try to upload the
+      // same image, and gracefully ignore it, while logging an
+      // error message.
       checkpointThread.join();
-      try {
-        checkpointThread.propagateExceptions();
-        fail("Didn't throw!");
-      } catch (Exception ioe) {
-        assertTrue("Unexpected exception: " +
-            StringUtils.stringifyException(ioe),
-            ioe.toString().contains("Another checkpointer already uploaded"));
-        LOG.info("Caught expected exception", ioe);
-      }
+      checkpointThread.propagateExceptions();
       
       // primary should still consider fsimage_4 the latest
       assertEquals(4, storage.getMostRecentCheckpointTxId());
@@ -1763,7 +1764,7 @@
   private void assertParallelFilesInvariant(MiniDFSCluster cluster,
       ImmutableList<SecondaryNameNode> secondaries) throws Exception {
     List<File> allCurrentDirs = Lists.newArrayList();
-    allCurrentDirs.addAll(getNameNodeCurrentDirs(cluster));
+    allCurrentDirs.addAll(getNameNodeCurrentDirs(cluster, 0));
     for (SecondaryNameNode snn : secondaries) {
       allCurrentDirs.addAll(getCheckpointCurrentDirs(snn));
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
index 68dc9f5..98c17a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
@@ -26,6 +26,7 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.commons.logging.Log;
@@ -47,7 +48,7 @@
   private String getClusterId(Configuration config) throws IOException {
     // see if cluster id not empty.
     Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(config);
-    Collection<URI> editsToFormat = FSNamesystem.getNamespaceEditsDirs(config);
+    List<URI> editsToFormat = FSNamesystem.getNamespaceEditsDirs(config);
     FSImage fsImage = new FSImage(config, dirsToFormat, editsToFormat);
     
     Iterator<StorageDirectory> sdit = 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 301c4d4..82730ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -110,7 +110,9 @@
     DatanodeProtocol dnp = cluster.getNameNodeRpc();
     
     ReceivedDeletedBlockInfo[] blocks = { new ReceivedDeletedBlockInfo(
-        new Block(0), "") };
+        new Block(0), 
+        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK,
+        null) };
     StorageReceivedDeletedBlocks[] storageBlocks = { 
         new StorageReceivedDeletedBlocks(reg.getStorageID(), blocks) };
     
@@ -136,7 +138,7 @@
     // that asks datanode to register again
     StorageReport[] rep = { new StorageReport(reg.getStorageID(), false, 0, 0,
         0, 0) };
-    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0, 0, 0);
+    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0, 0, 0).getCommands();
     Assert.assertEquals(1, cmd.length);
     Assert.assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index b3eeeab..bc41e7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -147,7 +147,7 @@
   public void testPreTxIdEditLogNoEdits() throws Exception {
     FSNamesystem namesys = Mockito.mock(FSNamesystem.class);
     namesys.dir = Mockito.mock(FSDirectory.class);
-    int numEdits = testLoad(
+    long numEdits = testLoad(
         StringUtils.hexStringToByte("ffffffed"), // just version number
         namesys);
     assertEquals(0, numEdits);
@@ -166,7 +166,7 @@
       cluster.waitActive();
       final FSNamesystem namesystem = cluster.getNamesystem();
 
-      int numEdits = testLoad(HADOOP20_SOME_EDITS, namesystem);
+      long numEdits = testLoad(HADOOP20_SOME_EDITS, namesystem);
       assertEquals(3, numEdits);
       // Sanity check the edit
       HdfsFileStatus fileInfo = namesystem.getFileInfo("/myfile", false);
@@ -177,7 +177,7 @@
     }
   }
   
-  private int testLoad(byte[] data, FSNamesystem namesys) throws IOException {
+  private long testLoad(byte[] data, FSNamesystem namesys) throws IOException {
     FSEditLogLoader loader = new FSEditLogLoader(namesys);
     return loader.loadFSEdits(new EditLogByteInputStream(data), 1);
   }
@@ -321,7 +321,7 @@
         assertTrue("Expect " + editFile + " exists", editFile.exists());
         
         System.out.println("Verifying file: " + editFile);
-        int numEdits = loader.loadFSEdits(
+        long numEdits = loader.loadFSEdits(
             new EditLogFileInputStream(editFile), 3);
         int numLeases = namesystem.leaseManager.countLease();
         System.out.println("Number of outstanding leases " + numLeases);
@@ -589,7 +589,6 @@
             currentDir.getAbsolutePath());
         assertNotNull("No image found in " + nameDir, imageFile);
         assertEquals(NNStorage.getImageFileName(0), imageFile.getName());
-        
         // Try to start a new cluster
         LOG.info("\n===========================================\n" +
         "Starting same cluster after simulated crash");
@@ -636,22 +635,26 @@
     }
   }
   
+  // should succeed - only one corrupt log dir
   public void testCrashRecoveryEmptyLogOneDir() throws Exception {
-    doTestCrashRecoveryEmptyLog(false, true);
+    doTestCrashRecoveryEmptyLog(false, true, true);
   }
   
+  // should fail - seen_txid updated to 3, but no log dir contains txid 3
   public void testCrashRecoveryEmptyLogBothDirs() throws Exception {
-    doTestCrashRecoveryEmptyLog(true, true);
+    doTestCrashRecoveryEmptyLog(true, true, false);
   }
 
+  // should succeed - only one corrupt log dir
   public void testCrashRecoveryEmptyLogOneDirNoUpdateSeenTxId() 
       throws Exception {
-    doTestCrashRecoveryEmptyLog(false, false);
+    doTestCrashRecoveryEmptyLog(false, false, true);
   }
   
+  // should succeed - both log dirs corrupt, but seen_txid never updated
   public void testCrashRecoveryEmptyLogBothDirsNoUpdateSeenTxId()
       throws Exception {
-    doTestCrashRecoveryEmptyLog(true, false);
+    doTestCrashRecoveryEmptyLog(true, false, true);
   }
 
   /**
@@ -667,12 +670,13 @@
    * NN should fail to start up, because it's aware that txid 3
    * was reached, but unable to find a non-corrupt log starting there.
    * @param updateTransactionIdFile if true update the seen_txid file.
-   * If false, the it will not be updated. This will simulate a case 
-   * where the NN crashed between creating the new segment and updating
-   * seen_txid. 
+   * If false, it will not be updated. This will simulate a case where
+   * the NN crashed between creating the new segment and updating the
+   * seen_txid file.
+   * @param shouldSucceed true if the test is expected to succeed.
    */
   private void doTestCrashRecoveryEmptyLog(boolean inBothDirs, 
-                                           boolean updateTransactionIdFile) 
+      boolean updateTransactionIdFile, boolean shouldSucceed)
       throws Exception {
     // start a cluster 
     Configuration conf = new HdfsConfiguration();
@@ -691,29 +695,40 @@
       // Make a truncated edits_3_inprogress
       File log = new File(currentDir,
           NNStorage.getInProgressEditsFileName(3));
-      NNStorage storage = new NNStorage(conf, 
-                                        Collections.<URI>emptyList(),
-                                        Lists.newArrayList(uri));
-      if (updateTransactionIdFile) {
-        storage.writeTransactionIdFileToStorage(3);
-      }
-      storage.close();
 
       new EditLogFileOutputStream(log, 1024).create();
       if (!inBothDirs) {
         break;
       }
+      
+      NNStorage storage = new NNStorage(conf, 
+          Collections.<URI>emptyList(),
+          Lists.newArrayList(uri));
+      
+      if (updateTransactionIdFile) {
+        storage.writeTransactionIdFileToStorage(3);
+      }
+      storage.close();
     }
     
     try {
       cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(NUM_DATA_NODES).format(false).build();
-      fail("Did not fail to start with all-corrupt logs");
+      if (!shouldSucceed) {
+        fail("Should not have succeeded in startin cluster");
+      }
     } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains(
-          "No non-corrupt logs for txid 3", ioe);
+      if (shouldSucceed) {
+        LOG.info("Should have succeeded in starting cluster, but failed", ioe);
+        throw ioe;
+      } else {
+        GenericTestUtils.assertExceptionContains(
+            "No non-corrupt logs for txid 3",
+            ioe);
+      }
+    } finally {
+      cluster.shutdown();
     }
-    cluster.shutdown();
   }
 
   
@@ -781,6 +796,11 @@
     public JournalType getType() {
       return JournalType.FILE;
     }
+
+    @Override
+    public boolean isInProgress() {
+      return true;
+    }
   }
 
   public void testFailedOpen() throws Exception {
@@ -789,11 +809,11 @@
     FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
     try {
       logDir.setWritable(false);
-      log.open();
+      log.openForWrite();
       fail("Did no throw exception on only having a bad dir");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
-          "no journals successfully started", ioe);
+          "too few journals successfully started", ioe);
     } finally {
       logDir.setWritable(true);
       log.close();
@@ -813,7 +833,7 @@
         new byte[500]);
     
     try {
-      log.open();
+      log.openForWrite();
       NameNodeMetrics mockMetrics = Mockito.mock(NameNodeMetrics.class);
       log.setMetricsForTests(mockMetrics);
 
@@ -848,6 +868,7 @@
         "[1,100]|[101,200]|[201,]",
         "[1,100]|[101,200]|[201,]");
     log = new FSEditLog(storage);
+    log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
     assertEquals("[[101,200]]",
@@ -859,6 +880,7 @@
         "[1,100]|[101,200]",
         "[1,100]|[201,300]|[301,400]"); // nothing starting at 101
     log = new FSEditLog(storage);
+    log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200], [201,300], [301,400]]",
         log.getEditLogManifest(1).toString());
     
@@ -868,6 +890,7 @@
         "[1,100]|[301,400]", // gap from 101 to 300
         "[301,400]|[401,500]");
     log = new FSEditLog(storage);
+    log.initJournalsForWrite();
     assertEquals("[[301,400], [401,500]]",
         log.getEditLogManifest(1).toString());
     
@@ -877,6 +900,7 @@
         "[1,100]|[101,150]", // short log at 101
         "[1,50]|[101,200]"); // short log at 1
     log = new FSEditLog(storage);
+    log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
     assertEquals("[[101,200]]",
@@ -889,6 +913,7 @@
         "[1,100]|[101,]", 
         "[1,100]|[101,200]"); 
     log = new FSEditLog(storage);
+    log.initJournalsForWrite();
     assertEquals("[[1,100], [101,200]]",
         log.getEditLogManifest(1).toString());
     assertEquals("[[101,200]]",
@@ -967,11 +992,11 @@
    * 
    * @param editUris directories to create edit logs in
    * @param numrolls number of times to roll the edit log during setup
+   * @param closeOnFinish whether to close the edit log after setup
    * @param abortAtRolls Specifications for when to fail, see AbortSpec
    */
-  public static NNStorage setupEdits(List<URI> editUris, int numrolls, 
-                                     AbortSpec... abortAtRolls)
-      throws IOException {
+  public static NNStorage setupEdits(List<URI> editUris, int numrolls,
+      boolean closeOnFinish, AbortSpec... abortAtRolls) throws IOException {
     List<AbortSpec> aborts = new ArrayList<AbortSpec>(Arrays.asList(abortAtRolls));
     NNStorage storage = new NNStorage(new Configuration(),
                                       Collections.<URI>emptyList(),
@@ -981,7 +1006,8 @@
     // open the edit log and add two transactions
     // logGenerationStamp is used, simply because it doesn't 
     // require complex arguments.
-    editlog.open();
+    editlog.initJournalsForWrite();
+    editlog.openForWrite();
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
       editlog.logGenerationStamp((long)0);
     }
@@ -1009,16 +1035,34 @@
       }
       editlog.logSync();
     }
-    editlog.close();
+    
+    if (closeOnFinish) {
+      editlog.close();
+    }
 
     FSImageTestUtil.logStorageContents(LOG, storage);
     return storage;
   }
+    
+  /**
+   * Set up directories for tests. 
+   *
+   * Each rolled file is 10 txns long. 
+   * A failed file is 2 txns long.
+   * 
+   * @param editUris directories to create edit logs in
+   * @param numrolls number of times to roll the edit log during setup
+   * @param abortAtRolls Specifications for when to fail, see AbortSpec
+   */
+  public static NNStorage setupEdits(List<URI> editUris, int numrolls, 
+      AbortSpec... abortAtRolls) throws IOException {
+    return setupEdits(editUris, numrolls, true, abortAtRolls);
+  }
 
   /** 
    * Test loading an editlog which has had both its storage fail
    * on alternating rolls. Two edit log directories are created.
-   * The first on fails on odd rolls, the second on even. Test
+   * The first one fails on odd rolls, the second on even. Test
    * that we are able to load the entire editlog regardless.
    */
   @Test
@@ -1041,6 +1085,7 @@
                                    new AbortSpec(10, 1));
     long totaltxnread = 0;
     FSEditLog editlog = new FSEditLog(storage);
+    editlog.initJournalsForWrite();
     long startTxId = 1;
     Iterable<EditLogInputStream> editStreams = editlog.selectInputStreams(startTxId, 
                                                                           TXNS_PER_ROLL*11);
@@ -1090,11 +1135,10 @@
     assertTrue(files[0].delete());
     
     FSEditLog editlog = new FSEditLog(storage);
+    editlog.initJournalsForWrite();
     long startTxId = 1;
     try {
-      Iterable<EditLogInputStream> editStreams 
-        = editlog.selectInputStreams(startTxId, 4*TXNS_PER_ROLL);
-      
+      editlog.selectInputStreams(startTxId, 4*TXNS_PER_ROLL);
       fail("Should have thrown exception");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
index 77fd686..d14b2b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
@@ -42,6 +42,7 @@
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.mockito.verification.VerificationMode;
 
 public class TestEditLogJournalFailures {
@@ -144,21 +145,35 @@
         DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
     shutDownMiniCluster();
     Configuration conf = new HdfsConfiguration();
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY, editsDirs[1]);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY, editsDirs[0]);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY, 0);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY, 0);
     setUpMiniCluster(conf, true);
     
     assertTrue(doAnEdit());
     // Invalidated the one required edits journal.
-    invalidateEditsDirAtIndex(1, false, false);
+    invalidateEditsDirAtIndex(0, false, false);
+    JournalAndStream nonRequiredJas = getJournalAndStream(1);
+    EditLogFileOutputStream nonRequiredSpy =
+      spyOnStream(nonRequiredJas);
+    
     // Make sure runtime.exit(...) hasn't been called at all yet.
     assertExitInvocations(0);
     
+    // ..and that the other stream is active.
+    assertTrue(nonRequiredJas.isActive());
+    
     // This will actually return true in the tests, since the NN will not in
     // fact call Runtime.exit();
     doAnEdit();
     
+    // Since the required directory failed setReadyToFlush, and that
+    // directory was listed prior to the non-required directory,
+    // we should not call setReadyToFlush on the non-required
+    // directory. Regression test for HDFS-2874.
+    Mockito.verify(nonRequiredSpy, Mockito.never()).setReadyToFlush();
+    assertFalse(nonRequiredJas.isActive());
+    
     // A single failure of a required journal should result in a call to
     // runtime.exit(...).
     assertExitInvocations(atLeast(1));
@@ -217,15 +232,10 @@
    * @param index the index of the journal to take offline.
    * @return the original <code>EditLogOutputStream</code> of the journal.
    */
-  private EditLogOutputStream invalidateEditsDirAtIndex(int index,
+  private void invalidateEditsDirAtIndex(int index,
       boolean failOnFlush, boolean failOnWrite) throws IOException {
-    FSImage fsimage = cluster.getNamesystem().getFSImage();
-    FSEditLog editLog = fsimage.getEditLog();
-
-    JournalAndStream jas = editLog.getJournals().get(index);
-    EditLogFileOutputStream elos =
-      (EditLogFileOutputStream) jas.getCurrentStream();
-    EditLogFileOutputStream spyElos = spy(elos);
+    JournalAndStream jas = getJournalAndStream(index);
+    EditLogFileOutputStream spyElos = spyOnStream(jas);
     if (failOnWrite) {
       doThrow(new IOException("fail on write()")).when(spyElos).write(
           (FSEditLogOp) any());
@@ -237,25 +247,24 @@
         .setReadyToFlush();
     }
     doNothing().when(spyElos).abort();
-     
+  }
+
+  private EditLogFileOutputStream spyOnStream(JournalAndStream jas) {
+    EditLogFileOutputStream elos =
+      (EditLogFileOutputStream) jas.getCurrentStream();
+    EditLogFileOutputStream spyElos = spy(elos);
     jas.setCurrentStreamForTests(spyElos);
-     
-    return elos;
+    return spyElos;
   }
 
   /**
-   * Restore the journal at index <code>index</code> with the passed
-   * {@link EditLogOutputStream}.
-   * 
-   * @param index index of the journal to restore.
-   * @param elos the {@link EditLogOutputStream} to put at that index.
+   * Pull out one of the JournalAndStream objects from the edit log.
    */
-  private void restoreEditsDirAtIndex(int index, EditLogOutputStream elos) {
+  private JournalAndStream getJournalAndStream(int index) {
     FSImage fsimage = cluster.getNamesystem().getFSImage();
     FSEditLog editLog = fsimage.getEditLog();
 
-    JournalAndStream jas = editLog.getJournals().get(index);
-    jas.setCurrentStreamForTests(elos);
+    return editLog.getJournals().get(index);
   }
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
index d3d6459..da66b45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
@@ -237,7 +237,7 @@
         
       System.out.println("Verifying file: " + editFile);
       FSEditLogLoader loader = new FSEditLogLoader(namesystem);
-      int numEditsThisLog = loader.loadFSEdits(new EditLogFileInputStream(editFile), 
+      long numEditsThisLog = loader.loadFSEdits(new EditLogFileInputStream(editFile), 
           startTxId);
       
       System.out.println("Number of edits: " + numEditsThisLog);
@@ -375,6 +375,7 @@
                 true);
             LOG.info("mkdirs complete");
           } catch (Throwable ioe) {
+            LOG.fatal("Got exception", ioe);
             deferredException.set(ioe);
             waitToEnterFlush.countDown();
           }
@@ -469,6 +470,7 @@
                 true);
             LOG.info("mkdirs complete");
           } catch (Throwable ioe) {
+            LOG.fatal("Got exception", ioe);
             deferredException.set(ioe);
             waitToEnterSync.countDown();
           }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index dcbeea6..fd1733a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -92,7 +92,7 @@
     
     StringBuilder bld = new StringBuilder();
     bld.append("^Error replaying edit log at offset \\d+");
-    bld.append("On transaction ID \\d+\n");
+    bld.append(" on transaction ID \\d+\n");
     bld.append("Recent opcode offsets: (\\d+\\s*){4}$");
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES)
@@ -167,7 +167,7 @@
     SortedMap<Long, Long> offsetToTxId = Maps.newTreeMap();
     try {
       fsel = FSImageTestUtil.createStandaloneEditLog(testDir);
-      fsel.open();
+      fsel.openForWrite();
       assertTrue("should exist: " + logFile, logFile.exists());
       
       for (int i = 0; i < NUM_TXNS; i++) {
@@ -245,7 +245,9 @@
       Files.copy(logFileBak, logFile);
       corruptByteInFile(logFile, offset);
       EditLogValidation val = EditLogFileInputStream.validateEditLog(logFile);
-      assertTrue(val.getNumTransactions() >= prevNumValid);
+      assertTrue(String.format("%d should have been >= %d",
+          val.getNumTransactions(), prevNumValid),
+          val.getNumTransactions() >= prevNumValid);
       prevNumValid = val.getNumTransactions();
     }
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
new file mode 100644
index 0000000..de3a89c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
@@ -0,0 +1,48 @@
+/**
+ * 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.namenode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestFSNamesystem {
+
+  /**
+   * Tests that the namenode edits dirs are gotten with duplicates removed
+   */
+  @Test
+  public void testUniqueEditDirs() throws IOException {
+    Configuration config = new Configuration();
+
+    config.set(DFS_NAMENODE_EDITS_DIR_KEY, "file://edits/dir, "
+        + "file://edits/dir1,file://edits/dir1"); // overlapping internally
+
+    // getNamespaceEditsDirs removes duplicates
+    Collection<URI> editsDirs = FSNamesystem.getNamespaceEditsDirs(config);
+    assertEquals(2, editsDirs.size());
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
index e4ff4bb..0ac1944 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
@@ -29,7 +29,9 @@
 import java.io.FilenameFilter;
 import java.io.IOException;
 import org.junit.Test;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.JournalManager.CorruptionException;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.test.GenericTestUtils;
 import static org.apache.hadoop.hdfs.server.namenode.TestEditLog.setupEdits;
@@ -58,8 +60,8 @@
     
     long numJournals = 0;
     for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.EDITS)) {
-      FileJournalManager jm = new FileJournalManager(sd);
-      assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1));
+      FileJournalManager jm = new FileJournalManager(sd, storage);
+      assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1, true));
       numJournals++;
     }
     assertEquals(3, numJournals);
@@ -78,9 +80,9 @@
                                    5, new AbortSpec(5, 0));
     StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
 
-    FileJournalManager jm = new FileJournalManager(sd);
+    FileJournalManager jm = new FileJournalManager(sd, storage);
     assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, 
-                 jm.getNumberOfTransactions(1));
+                 jm.getNumberOfTransactions(1, true));
   }
 
   /**
@@ -101,16 +103,17 @@
                                    5, new AbortSpec(5, 1));
     Iterator<StorageDirectory> dirs = storage.dirIterator(NameNodeDirType.EDITS);
     StorageDirectory sd = dirs.next();
-    FileJournalManager jm = new FileJournalManager(sd);
-    assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1));
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1, true));
     
     sd = dirs.next();
-    jm = new FileJournalManager(sd);
-    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+    jm = new FileJournalManager(sd, storage);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1,
+        true));
 
     sd = dirs.next();
-    jm = new FileJournalManager(sd);
-    assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1));
+    jm = new FileJournalManager(sd, storage);
+    assertEquals(6*TXNS_PER_ROLL, jm.getNumberOfTransactions(1, true));
   }
 
   /** 
@@ -133,16 +136,19 @@
                                    new AbortSpec(5, 2));
     Iterator<StorageDirectory> dirs = storage.dirIterator(NameNodeDirType.EDITS);
     StorageDirectory sd = dirs.next();
-    FileJournalManager jm = new FileJournalManager(sd);
-    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1,
+        true));
     
     sd = dirs.next();
-    jm = new FileJournalManager(sd);
-    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+    jm = new FileJournalManager(sd, storage);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1,
+        true));
 
     sd = dirs.next();
-    jm = new FileJournalManager(sd);
-    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1));
+    jm = new FileJournalManager(sd, storage);
+    assertEquals(5*TXNS_PER_ROLL + TXNS_PER_FAIL, jm.getNumberOfTransactions(1,
+        true));
   }
 
   /** 
@@ -156,6 +162,25 @@
     }
     raf.close();
   }
+  
+  @Test(expected=IllegalStateException.class)
+  public void testFinalizeErrorReportedToNNStorage() throws IOException, InterruptedException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltestError");
+    // abort after 10th roll
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()),
+                                   10, new AbortSpec(10, 0));
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    String sdRootPath = sd.getRoot().getAbsolutePath();
+    FileUtil.chmod(sdRootPath, "-w", true);
+    try {
+      jm.finalizeLogSegment(0, 1);
+    } finally {
+      assertTrue(storage.getRemovedStorageDirs().contains(sd));
+      FileUtil.chmod(sdRootPath, "+w", true);
+    }
+  }
 
   /** 
    * Test that we can read from a stream created by FileJournalManager.
@@ -171,17 +196,17 @@
                                    10, new AbortSpec(10, 0));
     StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
 
-    FileJournalManager jm = new FileJournalManager(sd);
+    FileJournalManager jm = new FileJournalManager(sd, storage);
     long expectedTotalTxnCount = TXNS_PER_ROLL*10 + TXNS_PER_FAIL;
-    assertEquals(expectedTotalTxnCount, jm.getNumberOfTransactions(1));
+    assertEquals(expectedTotalTxnCount, jm.getNumberOfTransactions(1, true));
 
     long skippedTxns = (3*TXNS_PER_ROLL); // skip first 3 files
     long startingTxId = skippedTxns + 1; 
 
-    long numTransactionsToLoad = jm.getNumberOfTransactions(startingTxId);
+    long numTransactionsToLoad = jm.getNumberOfTransactions(startingTxId, true);
     long numLoaded = 0;
     while (numLoaded < numTransactionsToLoad) {
-      EditLogInputStream editIn = jm.getInputStream(startingTxId);
+      EditLogInputStream editIn = jm.getInputStream(startingTxId, true);
       FSEditLogLoader.EditLogValidation val = FSEditLogLoader.validateEditLog(editIn);
       long count = val.getNumTransactions();
 
@@ -194,20 +219,26 @@
   }
 
   /**
-   * Try to make a request with a start transaction id which doesn't
-   * match the start ID of some log segment. 
-   * This should fail as edit logs must currently be treated as indevisable 
-   * units.
+   * Make requests with starting transaction ids which don't match the beginning
+   * txid of some log segments.
+   * 
+   * This should succeed.
    */
-  @Test(expected=IOException.class)
+  @Test
   public void testAskForTransactionsMidfile() throws IOException {
     File f = new File(TestEditLog.TEST_DIR + "/filejournaltest2");
     NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()), 
                                    10);
     StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
     
-    FileJournalManager jm = new FileJournalManager(sd);
-    jm.getNumberOfTransactions(2);    
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    
+    // 10 rolls, so 11 rolled files, 110 txids total.
+    final int TOTAL_TXIDS = 10 * 11;
+    for (int txid = 1; txid <= TOTAL_TXIDS; txid++) {
+      assertEquals((TOTAL_TXIDS - txid) + 1, jm.getNumberOfTransactions(txid,
+          true));
+    }
   }
 
   /** 
@@ -237,19 +268,20 @@
     assertEquals(1, files.length);
     assertTrue(files[0].delete());
     
-    FileJournalManager jm = new FileJournalManager(sd);
-    assertEquals(startGapTxId-1, jm.getNumberOfTransactions(1));
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    assertEquals(startGapTxId-1, jm.getNumberOfTransactions(1, true));
 
     try {
-      jm.getNumberOfTransactions(startGapTxId);
+      jm.getNumberOfTransactions(startGapTxId, true);
       fail("Should have thrown an exception by now");
     } catch (IOException ioe) {
-      assertTrue(true);
+      GenericTestUtils.assertExceptionContains(
+          "Gap in transactions, max txnid is 110, 0 txns from 31", ioe);
     }
 
     // rolled 10 times so there should be 11 files.
     assertEquals(11*TXNS_PER_ROLL - endGapTxId, 
-                 jm.getNumberOfTransactions(endGapTxId+1));
+                 jm.getNumberOfTransactions(endGapTxId + 1, true));
   }
 
   /** 
@@ -274,9 +306,9 @@
     
     corruptAfterStartSegment(files[0]);
 
-    FileJournalManager jm = new FileJournalManager(sd);
+    FileJournalManager jm = new FileJournalManager(sd, storage);
     assertEquals(10*TXNS_PER_ROLL+1, 
-                 jm.getNumberOfTransactions(1)); 
+                 jm.getNumberOfTransactions(1, true));
   }
 
   @Test
@@ -288,14 +320,15 @@
         NNStorage.getInProgressEditsFileName(201),
         NNStorage.getFinalizedEditsFileName(1001, 1100));
         
-    FileJournalManager fjm = new FileJournalManager(sd);
+    // passing null for NNStorage because this unit test will not use it
+    FileJournalManager fjm = new FileJournalManager(sd, null);
     assertEquals("[1,100],[101,200],[1001,1100]", getLogsAsString(fjm, 1));
     assertEquals("[101,200],[1001,1100]", getLogsAsString(fjm, 101));
     assertEquals("[1001,1100]", getLogsAsString(fjm, 201));
     try {
       assertEquals("[]", getLogsAsString(fjm, 150));
       fail("Did not throw when asking for a txn in the middle of a log");
-    } catch (IOException ioe) {
+    } catch (IllegalStateException ioe) {
       GenericTestUtils.assertExceptionContains(
           "150 which is in the middle", ioe);
     }
@@ -303,6 +336,60 @@
         "", getLogsAsString(fjm, 9999));
   }
 
+  /**
+   * tests that passing an invalid dir to matchEditLogs throws IOException 
+   */
+  @Test(expected = IOException.class)
+  public void testMatchEditLogInvalidDirThrowsIOException() throws IOException {
+    File badDir = new File("does not exist");
+    FileJournalManager.matchEditLogs(badDir);
+  }
+  
+  /**
+   * Make sure that we starting reading the correct op when we request a stream
+   * with a txid in the middle of an edit log file.
+   */
+  @Test
+  public void testReadFromMiddleOfEditLog() throws CorruptionException,
+      IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest2");
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()), 
+                                   10);
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+    
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    
+    EditLogInputStream elis = jm.getInputStream(5, true);
+    FSEditLogOp op = elis.readOp();
+    assertEquals("read unexpected op", op.getTransactionId(), 5);
+  }
+
+  /**
+   * Make sure that in-progress streams aren't counted if we don't ask for
+   * them.
+   */
+  @Test
+  public void testExcludeInProgressStreams() throws CorruptionException,
+      IOException {
+    File f = new File(TestEditLog.TEST_DIR + "/filejournaltest2");
+    
+    // Don't close the edit log once the files have been set up.
+    NNStorage storage = setupEdits(Collections.<URI>singletonList(f.toURI()), 
+                                   10, false);
+    StorageDirectory sd = storage.dirIterator(NameNodeDirType.EDITS).next();
+    
+    FileJournalManager jm = new FileJournalManager(sd, storage);
+    
+    // If we exclude the in-progess stream, we should only have 100 tx.
+    assertEquals(100, jm.getNumberOfTransactions(1, false));
+    
+    EditLogInputStream elis = jm.getInputStream(90, false);
+    FSEditLogOp lastReadOp = null;
+    while ((lastReadOp = elis.readOp()) != null) {
+      assertTrue(lastReadOp.getTransactionId() <= 100);
+    }
+  }
+
   private static String getLogsAsString(
       FileJournalManager fjm, long firstTxId) throws IOException {
     return Joiner.on(",").join(fjm.getRemoteEditLogs(firstTxId));
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
index 00fe43f..51e49a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java
@@ -144,13 +144,13 @@
     }
 
     @Override
-    public EditLogInputStream getInputStream(long fromTxnId)
+    public EditLogInputStream getInputStream(long fromTxnId, boolean inProgressOk)
         throws IOException {
       return null;
     }
 
     @Override
-    public long getNumberOfTransactions(long fromTxnId)
+    public long getNumberOfTransactions(long fromTxnId, boolean inProgressOk)
         throws IOException {
       return 0;
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionFunctional.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionFunctional.java
index aad8d7d..e7a9cc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionFunctional.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionFunctional.java
@@ -61,6 +61,7 @@
       throws IOException {
     MiniDFSCluster cluster = null;    
     Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY, 0);
 
     File sd0 = new File(TEST_ROOT_DIR, "nn0");
     File sd1 = new File(TEST_ROOT_DIR, "nn1");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
index aadca5c..4c6334f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
@@ -23,6 +23,7 @@
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
@@ -33,6 +34,7 @@
 
 import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
@@ -46,6 +48,17 @@
 
 
 public class TestNNStorageRetentionManager {
+  Configuration conf = new Configuration();
+
+  /**
+   * For the purpose of this test, purge as many edits as we can 
+   * with no extra "safety cushion"
+   */
+  @Before
+  public void setNoExtraEditRetention() {
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY, 0);
+  }
+  
   /**
    * Test the "easy case" where we have more images in the
    * directory than we need to keep. Should purge the
@@ -163,9 +176,27 @@
     runTest(tc);    
   }
   
-  private void runTest(TestCaseDescription tc) throws IOException {
-    Configuration conf = new Configuration();
+  @Test
+  public void testRetainExtraLogs() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY,
+        50);
+    TestCaseDescription tc = new TestCaseDescription();
+    tc.addRoot("/foo1", NameNodeDirType.IMAGE);
+    tc.addRoot("/foo2", NameNodeDirType.EDITS);
+    tc.addImage("/foo1/current/" + getImageFileName(100), true);
+    tc.addImage("/foo1/current/" + getImageFileName(200), true);
+    tc.addImage("/foo1/current/" + getImageFileName(300), false);
+    tc.addImage("/foo1/current/" + getImageFileName(400), false);
 
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(101, 200), true);
+    // Since we need 50 extra edits, *do* retain the 201-300 segment 
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 300), false);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 400), false);
+    tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
+    runTest(tc);
+  }
+  
+  private void runTest(TestCaseDescription tc) throws IOException {
     StoragePurger mockPurger =
       Mockito.mock(NNStorageRetentionManager.StoragePurger.class);
     ArgumentCaptor<FSImageFile> imagesPurgedCaptor =
@@ -261,8 +292,9 @@
       for (FakeRoot root : dirRoots.values()) {
         if (!root.type.isOfType(NameNodeDirType.EDITS)) continue;
         
+        // passing null NNStorage for unit test because it does not use it
         FileJournalManager fjm = new FileJournalManager(
-            root.mockStorageDir());
+            root.mockStorageDir(), null);
         fjm.purger = purger;
         jms.add(fjm);
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java
index 559d165..49a96e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java
@@ -50,13 +50,7 @@
     assertFalse(testResourceScenario(4, 0, 3, 0, 2));
     assertTrue(testResourceScenario(4, 0, 3, 0, 1));
     assertFalse(testResourceScenario(4, 0, 4, 0, 1));
-    try {
-      testResourceScenario(1, 0, 0, 0, 2);
-      fail("Should fail if there are more minimum redundant resources than " +
-          "total redundant resources");
-    } catch (RuntimeException rte) {
-      assertTrue(rte.getMessage().startsWith("Need a minimum"));
-    }
+    assertFalse(testResourceScenario(1, 0, 0, 0, 2));
   }
   
   @Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
index d4fd72d..596df8d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
@@ -24,6 +24,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -91,6 +92,9 @@
     FileSystem fileSys = null;
 
     try {
+      conf.setBoolean(
+          DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
       cluster.waitActive();
       fileSys = cluster.getFileSystem();
@@ -106,7 +110,6 @@
   
       // set small size of flush buffer
       editLog.setOutputBufferCapacity(2048);
-      namesystem.getDelegationTokenSecretManager().startThreads();
     
       // Create threads and make them run transactions concurrently.
       Thread threadId[] = new Thread[NUM_THREADS];
@@ -141,7 +144,7 @@
         System.out.println("Verifying file: " + editFile);
         
         FSEditLogLoader loader = new FSEditLogLoader(namesystem);        
-        int numEdits = loader.loadFSEdits(
+        long numEdits = loader.loadFSEdits(
             new EditLogFileInputStream(editFile), 1);
         assertEquals("Verification for " + editFile, expectedTransactions, numEdits);
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
index 9233009..74c3cf8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
@@ -512,11 +512,10 @@
     InetAddress inetAddress = InetAddress.getByAddress(b);
     list.add(inetAddress.getHostName());
     writeConfigFile(localFileSys, hostsFile, list);
-    int numNameNodes = 1;
     int numDatanodes = 1;
     
     try {
-      cluster = new MiniDFSCluster.Builder(conf).numNameNodes(numNameNodes)
+      cluster = new MiniDFSCluster.Builder(conf)
       .numDataNodes(numDatanodes).setupHostsFile(true).build();
       cluster.waitActive();
   
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
index 397ad72..53f4f96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
@@ -19,10 +19,12 @@
 
 import static org.junit.Assert.*;
 import org.junit.Test;
+import java.io.File;
 import java.io.IOException;
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -71,4 +73,25 @@
     DFSTestUtil.formatNameNode(conf);
     NameNode nameNode = new NameNode(conf); // should be OK!
   }
+
+  /**
+   * HDFS-3013: NameNode format command doesn't pick up
+   * dfs.namenode.name.dir.NameServiceId configuration.
+   */
+  @Test
+  public void testGenericKeysForNameNodeFormat()
+      throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:8070");
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "ns1");
+    String nameDir = System.getProperty("java.io.tmpdir") + "/test.dfs.name";
+    File dir = new File(nameDir);
+    if (dir.exists()) {
+      FileUtil.fullyDelete(dir);
+    }
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + ".ns1", nameDir);
+    DFSTestUtil.formatNameNode(conf);
+    NameNode nameNode = new NameNode(conf);
+    FileUtil.fullyDelete(dir);
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
new file mode 100644
index 0000000..39667ed
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
@@ -0,0 +1,150 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+
+/**
+ * Utility class to start an HA cluster, and then start threads
+ * to periodically fail back and forth, accelerate block deletion
+ * processing, etc.
+ */
+public class HAStressTestHarness {
+  Configuration conf;
+  private MiniDFSCluster cluster;
+  static final int BLOCK_SIZE = 1024;
+  TestContext testCtx = new TestContext();
+  
+  public HAStressTestHarness() {
+    conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    // Increase max streams so that we re-replicate quickly.
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000);
+  }
+
+  /**
+   * Start and return the MiniDFSCluster.
+   */
+  public MiniDFSCluster startCluster() throws IOException {
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .build();
+    return cluster;
+  }
+
+  /**
+   * Return a filesystem with client-failover configured for the
+   * cluster.
+   */
+  public FileSystem getFailoverFs() throws IOException, URISyntaxException {
+    return HATestUtil.configureFailoverFs(cluster, conf);
+  }
+
+  /**
+   * Add a thread which periodically triggers deletion reports,
+   * heartbeats, and NN-side block work.
+   * @param interval millisecond period on which to run
+   */
+  public void addReplicationTriggerThread(final int interval) {
+
+    testCtx.addThread(new RepeatingTestThread(testCtx) {
+      
+      @Override
+      public void doAnAction() throws Exception {
+        for (DataNode dn : cluster.getDataNodes()) {
+          DataNodeAdapter.triggerDeletionReport(dn);
+          DataNodeAdapter.triggerHeartbeat(dn);
+        }
+        for (int i = 0; i < 2; i++) {
+          NameNode nn = cluster.getNameNode(i);
+          BlockManagerTestUtil.computeAllPendingWork(
+              nn.getNamesystem().getBlockManager());
+        }
+        Thread.sleep(interval);
+      }
+    });
+  }
+
+  /**
+   * Add a thread which periodically triggers failover back and forth between
+   * the two namenodes.
+   */
+  public void addFailoverThread(final int msBetweenFailovers) {
+    testCtx.addThread(new RepeatingTestThread(testCtx) {
+      
+      @Override
+      public void doAnAction() throws Exception {
+        System.err.println("==============================\n" +
+            "Failing over from 0->1\n" +
+            "==================================");
+        cluster.transitionToStandby(0);
+        cluster.transitionToActive(1);
+        
+        Thread.sleep(msBetweenFailovers);
+        System.err.println("==============================\n" +
+            "Failing over from 1->0\n" +
+            "==================================");
+
+        cluster.transitionToStandby(1);
+        cluster.transitionToActive(0);
+        Thread.sleep(msBetweenFailovers);
+      }
+    });
+  }
+
+  /**
+   * Start all of the threads which have been added.
+   */
+  public void startThreads() {
+    this.testCtx.startThreads();
+  }
+
+  /**
+   * Stop threads, propagating any exceptions that might have been thrown.
+   */
+  public void stopThreads() throws Exception {
+    this.testCtx.stop();
+  }
+
+  /**
+   * Shutdown the minicluster, as well as any of the running threads.
+   */
+  public void shutdown() throws Exception {
+    this.testCtx.stop();
+    if (cluster != null) {
+      this.cluster.shutdown();
+      cluster = null;
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
new file mode 100644
index 0000000..bf919ce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -0,0 +1,214 @@
+/**
+ * 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.namenode.ha;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.test.GenericTestUtils;
+
+import com.google.common.base.Supplier;
+
+/**
+ * Static utility functions useful for testing HA.
+ */
+public abstract class HATestUtil {
+  private static Log LOG = LogFactory.getLog(HATestUtil.class);
+  
+  private static final String LOGICAL_HOSTNAME = "ha-nn-uri-%d";
+  
+  /**
+   * Trigger an edits log roll on the active and then wait for the standby to
+   * catch up to all the edits done by the active. This method will check
+   * repeatedly for up to NN_LAG_TIMEOUT milliseconds, and then fail throwing
+   * {@link CouldNotCatchUpException}
+   * 
+   * @param active active NN
+   * @param standby standby NN which should catch up to active
+   * @throws IOException if an error occurs rolling the edit log
+   * @throws CouldNotCatchUpException if the standby doesn't catch up to the
+   *         active in NN_LAG_TIMEOUT milliseconds
+   */
+  static void waitForStandbyToCatchUp(NameNode active,
+      NameNode standby) throws InterruptedException, IOException, CouldNotCatchUpException {
+    
+    long activeTxId = active.getNamesystem().getFSImage().getEditLog()
+      .getLastWrittenTxId();
+    
+    active.getRpcServer().rollEditLog();
+    
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start < TestEditLogTailer.NN_LAG_TIMEOUT) {
+      long nn2HighestTxId = standby.getNamesystem().getFSImage()
+        .getLastAppliedTxId();
+      if (nn2HighestTxId >= activeTxId) {
+        return;
+      }
+      Thread.sleep(TestEditLogTailer.SLEEP_TIME);
+    }
+    throw new CouldNotCatchUpException("Standby did not catch up to txid " +
+        activeTxId + " (currently at " +
+        standby.getNamesystem().getFSImage().getLastAppliedTxId() + ")");
+  }
+
+  /**
+   * Wait for the datanodes in the cluster to process any block
+   * deletions that have already been asynchronously queued.
+   */
+  static void waitForDNDeletions(final MiniDFSCluster cluster)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        for (DataNode dn : cluster.getDataNodes()) {
+          if (DataNodeAdapter.getPendingAsyncDeletions(dn) > 0) {
+            return false;
+          }
+        }
+        return true;
+      }
+    }, 1000, 10000);
+    
+  }
+
+  /**
+   * Wait for the NameNode to issue any deletions that are already
+   * pending (i.e. for the pendingDeletionBlocksCount to go to 0)
+   */
+  static void waitForNNToIssueDeletions(final NameNode nn)
+      throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        LOG.info("Waiting for NN to issue block deletions to DNs");
+        return nn.getNamesystem().getBlockManager().getPendingDeletionBlocksCount() == 0;
+      }
+    }, 250, 10000);
+  }
+
+  public static class CouldNotCatchUpException extends IOException {
+    private static final long serialVersionUID = 1L;
+
+    public CouldNotCatchUpException(String message) {
+      super(message);
+    }
+  }
+  
+  /** Gets the filesystem instance by setting the failover configurations */
+  public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf)
+      throws IOException, URISyntaxException {
+    return configureFailoverFs(cluster, conf, 0);
+  }
+
+  /** 
+   * Gets the filesystem instance by setting the failover configurations
+   * @param cluster the single process DFS cluster
+   * @param conf cluster configuration
+   * @param nsIndex namespace index starting with zero
+   * @throws IOException if an error occurs rolling the edit log
+   */
+  public static FileSystem configureFailoverFs(MiniDFSCluster cluster, Configuration conf,
+      int nsIndex) throws IOException, URISyntaxException {
+    conf = new Configuration(conf);
+    String logicalName = getLogicalHostname(cluster);
+    setFailoverConfigurations(cluster, conf, logicalName, nsIndex);
+    FileSystem fs = FileSystem.get(new URI("hdfs://" + logicalName), conf);
+    return fs;
+  }
+  
+  public static void setFailoverConfigurations(MiniDFSCluster cluster,
+      Configuration conf) {
+    setFailoverConfigurations(cluster, conf, getLogicalHostname(cluster));
+  }
+  
+  /** Sets the required configurations for performing failover of default namespace. */
+  public static void setFailoverConfigurations(MiniDFSCluster cluster,
+      Configuration conf, String logicalName) {
+    setFailoverConfigurations(cluster, conf, logicalName, 0);
+  }
+  
+  /** Sets the required configurations for performing failover.  */
+  public static void setFailoverConfigurations(MiniDFSCluster cluster,
+      Configuration conf, String logicalName, int nsIndex) {
+    InetSocketAddress nnAddr1 = cluster.getNameNode(2 * nsIndex).getNameNodeAddress();
+    InetSocketAddress nnAddr2 = cluster.getNameNode(2 * nsIndex + 1).getNameNodeAddress();
+    String nameNodeId1 = "nn1";
+    String nameNodeId2 = "nn2";
+    String address1 = "hdfs://" + nnAddr1.getHostName() + ":" + nnAddr1.getPort();
+    String address2 = "hdfs://" + nnAddr2.getHostName() + ":" + nnAddr2.getPort();
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
+        logicalName, nameNodeId1), address1);
+    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
+        logicalName, nameNodeId2), address2);
+    
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, logicalName);
+    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, logicalName),
+        nameNodeId1 + "," + nameNodeId2);
+    conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
+        ConfiguredFailoverProxyProvider.class.getName());
+    conf.set("fs.defaultFS", "hdfs://" + logicalName);
+  }
+  
+
+  public static String getLogicalHostname(MiniDFSCluster cluster) {
+    return String.format(LOGICAL_HOSTNAME, cluster.getInstanceId());
+  }
+  
+  public static URI getLogicalUri(MiniDFSCluster cluster)
+      throws URISyntaxException {
+    return new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
+        getLogicalHostname(cluster));
+  }
+  
+  public static void waitForCheckpoint(MiniDFSCluster cluster, int nnIdx,
+      List<Integer> txids) throws InterruptedException {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        FSImageTestUtil.assertNNHasCheckpoints(cluster, nnIdx, txids);
+        return;
+      } catch (AssertionError err) {
+        if (System.currentTimeMillis() - start > 10000) {
+          throw err;
+        } else {
+          Thread.sleep(300);
+        }
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
new file mode 100644
index 0000000..ccc46a2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
@@ -0,0 +1,107 @@
+/**
+* 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Tests for upgrading with HA enabled.
+ */
+public class TestDFSUpgradeWithHA {
+  
+  private static final Log LOG = LogFactory.getLog(TestDFSUpgradeWithHA.class);
+
+  /**
+   * Make sure that an HA NN refuses to start if given an upgrade-related
+   * startup option.
+   */
+  @Test
+  public void testStartingWithUpgradeOptionsFails() throws IOException {
+    for (StartupOption startOpt : Lists.newArrayList(new StartupOption[] {
+        StartupOption.UPGRADE, StartupOption.FINALIZE,
+        StartupOption.ROLLBACK })) {
+      MiniDFSCluster cluster = null;
+      try {
+        cluster = new MiniDFSCluster.Builder(new Configuration())
+            .nnTopology(MiniDFSNNTopology.simpleHATopology())
+            .startupOption(startOpt)
+            .numDataNodes(0)
+            .build();
+        fail("Should not have been able to start an HA NN in upgrade mode");
+      } catch (IllegalArgumentException iae) {
+        GenericTestUtils.assertExceptionContains(
+            "Cannot perform DFS upgrade with HA enabled.", iae);
+        LOG.info("Got expected exception", iae);
+      } finally {
+        if (cluster != null) {
+          cluster.shutdown();
+        }
+      }
+    }
+  }
+  
+  /**
+   * Make sure that an HA NN won't start if a previous upgrade was in progress.
+   */
+  @Test
+  public void testStartingWithUpgradeInProgressFails() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(new Configuration())
+          .nnTopology(MiniDFSNNTopology.simpleHATopology())
+          .numDataNodes(0)
+          .build();
+      
+      // Simulate an upgrade having started.
+      for (int i = 0; i < 2; i++) {
+        for (URI uri : cluster.getNameDirs(i)) {
+          File prevTmp = new File(new File(uri), Storage.STORAGE_TMP_PREVIOUS);
+          LOG.info("creating previous tmp dir: " + prevTmp);
+          assertTrue(prevTmp.mkdirs());
+        }
+      }
+      
+      cluster.restartNameNodes();
+      fail("Should not have been able to start an HA NN with an in-progress upgrade");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot start an HA namenode with name dirs that need recovery.",
+          ioe);
+      LOG.info("Got expected exception", ioe);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
new file mode 100644
index 0000000..ea769c0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
@@ -0,0 +1,605 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.AppendTestUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+
+
+public class TestDNFencing {
+  
+  protected static final Log LOG = LogFactory.getLog(
+      TestDNFencing.class);
+  private static final String TEST_FILE_DATA = "hello highly available world";
+  private static final String TEST_FILE = "/testStandbyIsHot";
+  private static final Path TEST_FILE_PATH = new Path(TEST_FILE);
+  private static final int SMALL_BLOCK = 1024;
+  
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private NameNode nn1, nn2;
+  private FileSystem fs;
+
+  static {
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(BlockManager.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+  }
+  
+  @Before
+  public void setupCluster() throws Exception {
+    conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, SMALL_BLOCK);
+    // Bump up replication interval so that we only run replication
+    // checks explicitly.
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 600);
+    // Increase max streams so that we re-replicate quickly.
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 1000);
+    // See RandomDeleterPolicy javadoc.
+    conf.setClass("dfs.block.replicator.classname", RandomDeleterPolicy.class,
+        BlockPlacementPolicy.class); 
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .build();
+    nn1 = cluster.getNameNode(0);
+    nn2 = cluster.getNameNode(1);
+    
+    cluster.waitActive();
+    cluster.transitionToActive(0);
+    // Trigger block reports so that the first NN trusts all
+    // of the DNs, and will issue deletions
+    cluster.triggerBlockReports();
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+  }
+  
+  @After
+  public void shutdownCluster() throws Exception {
+    if (cluster != null) {
+      banner("Shutting down cluster. NN1 metadata:");
+      doMetasave(nn1);
+      banner("Shutting down cluster. NN2 metadata:");
+      doMetasave(nn2);
+      cluster.shutdown();
+    }
+  }
+  
+
+  @Test
+  public void testDnFencing() throws Exception {
+    // Create a file with replication level 3.
+    DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30*SMALL_BLOCK, (short)3, 1L);
+    ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_FILE_PATH);
+    
+    // Drop its replication count to 1, so it becomes over-replicated.
+    // Then compute the invalidation of the extra blocks and trigger
+    // heartbeats so the invalidations are flushed to the DNs.
+    nn1.getRpcServer().setReplication(TEST_FILE, (short) 1);
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn1.getNamesystem().getBlockManager());
+    cluster.triggerHeartbeats();
+    
+    // Transition nn2 to active even though nn1 still thinks it's active.
+    banner("Failing to NN2 but let NN1 continue to think it's active");
+    NameNodeAdapter.abortEditLogs(nn1);
+    NameNodeAdapter.enterSafeMode(nn1, false);
+    cluster.transitionToActive(1);
+    
+    // Check that the standby picked up the replication change.
+    assertEquals(1,
+        nn2.getRpcServer().getFileInfo(TEST_FILE).getReplication());
+
+    // Dump some info for debugging purposes.
+    banner("NN2 Metadata immediately after failover");
+    doMetasave(nn2);
+    
+    // Even though NN2 considers the blocks over-replicated, it should
+    // post-pone the block invalidation because the DNs are still "stale".
+    assertEquals(30, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
+    
+    banner("Triggering heartbeats and block reports so that fencing is completed");
+    cluster.triggerHeartbeats();
+    cluster.triggerBlockReports();
+    
+    banner("Metadata after nodes have all block-reported");
+    doMetasave(nn2);
+    
+    // The blocks should no longer be postponed.
+    assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
+    
+    // Wait for NN2 to enact its deletions (replication monitor has to run, etc)
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn2.getNamesystem().getBlockManager());
+    cluster.triggerHeartbeats();
+    HATestUtil.waitForDNDeletions(cluster);
+    cluster.triggerDeletionReports();
+    assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
+    assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());
+    
+    banner("Making sure the file is still readable");
+    FileSystem fs2 = cluster.getFileSystem(1);
+    DFSTestUtil.readFile(fs2, TEST_FILE_PATH);
+
+    banner("Waiting for the actual block files to get deleted from DNs.");
+    waitForTrueReplication(cluster, block, 1);
+  }
+  
+  /**
+   * Test case which restarts the standby node in such a way that,
+   * when it exits safemode, it will want to invalidate a bunch
+   * of over-replicated block replicas. Ensures that if we failover
+   * at this point it won't lose data.
+   */
+  @Test
+  public void testNNClearsCommandsOnFailoverAfterStartup()
+      throws Exception {
+    // Make lots of blocks to increase chances of triggering a bug.
+    DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30*SMALL_BLOCK, (short)3, 1L);
+
+    banner("Shutting down NN2");
+    cluster.shutdownNameNode(1);
+
+    banner("Setting replication to 1, rolling edit log.");
+    nn1.getRpcServer().setReplication(TEST_FILE, (short) 1);
+    nn1.getRpcServer().rollEditLog();
+    
+    // Start NN2 again. When it starts up, it will see all of the
+    // blocks as over-replicated, since it has the metadata for
+    // replication=1, but the DNs haven't yet processed the deletions.
+    banner("Starting NN2 again.");
+    cluster.restartNameNode(1);
+    nn2 = cluster.getNameNode(1);
+    
+    banner("triggering BRs");
+    cluster.triggerBlockReports();
+
+    // We expect that both NN1 and NN2 will have some number of
+    // deletions queued up for the DNs.
+    banner("computing invalidation on nn1");
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn1.getNamesystem().getBlockManager());
+
+    banner("computing invalidation on nn2");
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn2.getNamesystem().getBlockManager());
+    
+    // Dump some info for debugging purposes.
+    banner("Metadata immediately before failover");
+    doMetasave(nn2);
+
+
+    // Transition nn2 to active even though nn1 still thinks it's active
+    banner("Failing to NN2 but let NN1 continue to think it's active");
+    NameNodeAdapter.abortEditLogs(nn1);
+    NameNodeAdapter.enterSafeMode(nn1, false);
+
+    cluster.transitionToActive(1);
+
+    // Check that the standby picked up the replication change.
+    assertEquals(1,
+        nn2.getRpcServer().getFileInfo(TEST_FILE).getReplication());
+
+    // Dump some info for debugging purposes.
+    banner("Metadata immediately after failover");
+    doMetasave(nn2);
+    
+    banner("Triggering heartbeats and block reports so that fencing is completed");
+    cluster.triggerHeartbeats();
+    cluster.triggerBlockReports();
+    
+    banner("Metadata after nodes have all block-reported");
+    doMetasave(nn2);
+    
+    // The block should no longer be postponed.
+    assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
+    
+    // Wait for NN2 to enact its deletions (replication monitor has to run, etc)
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn2.getNamesystem().getBlockManager());
+
+    HATestUtil.waitForNNToIssueDeletions(nn2);
+    cluster.triggerHeartbeats();
+    HATestUtil.waitForDNDeletions(cluster);
+    cluster.triggerDeletionReports();
+    assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
+    assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());
+    
+    banner("Making sure the file is still readable");
+    FileSystem fs2 = cluster.getFileSystem(1);
+    DFSTestUtil.readFile(fs2, TEST_FILE_PATH);
+  }
+  
+  /**
+   * Test case that reduces replication of a file with a lot of blocks
+   * and then fails over right after those blocks enter the DN invalidation
+   * queues on the active. Ensures that fencing is correct and no replicas
+   * are lost.
+   */
+  @Test
+  public void testNNClearsCommandsOnFailoverWithReplChanges()
+      throws Exception {
+    // Make lots of blocks to increase chances of triggering a bug.
+    DFSTestUtil.createFile(fs, TEST_FILE_PATH, 30*SMALL_BLOCK, (short)1, 1L);
+
+    banner("rolling NN1's edit log, forcing catch-up");
+    HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
+    
+    // Get some new replicas reported so that NN2 now considers
+    // them over-replicated and schedules some more deletions
+    nn1.getRpcServer().setReplication(TEST_FILE, (short) 2);
+    while (BlockManagerTestUtil.getComputedDatanodeWork(
+        nn1.getNamesystem().getBlockManager()) > 0) {
+      LOG.info("Getting more replication work computed");
+    }
+    BlockManager bm1 = nn1.getNamesystem().getBlockManager();
+    while (bm1.getPendingReplicationBlocksCount() > 0) {
+      BlockManagerTestUtil.updateState(bm1);
+      cluster.triggerHeartbeats();
+      Thread.sleep(1000);
+    }
+    
+    banner("triggering BRs");
+    cluster.triggerBlockReports();
+    
+    nn1.getRpcServer().setReplication(TEST_FILE, (short) 1);
+
+    
+    banner("computing invalidation on nn1");
+
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn1.getNamesystem().getBlockManager());
+    doMetasave(nn1);
+
+    banner("computing invalidation on nn2");
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn2.getNamesystem().getBlockManager());
+    doMetasave(nn2);
+
+    // Dump some info for debugging purposes.
+    banner("Metadata immediately before failover");
+    doMetasave(nn2);
+
+
+    // Transition nn2 to active even though nn1 still thinks it's active
+    banner("Failing to NN2 but let NN1 continue to think it's active");
+    NameNodeAdapter.abortEditLogs(nn1);
+    NameNodeAdapter.enterSafeMode(nn1, false);
+
+    
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn2.getNamesystem().getBlockManager());
+    cluster.transitionToActive(1);
+
+    // Check that the standby picked up the replication change.
+    assertEquals(1,
+        nn2.getRpcServer().getFileInfo(TEST_FILE).getReplication());
+
+    // Dump some info for debugging purposes.
+    banner("Metadata immediately after failover");
+    doMetasave(nn2);
+    
+    banner("Triggering heartbeats and block reports so that fencing is completed");
+    cluster.triggerHeartbeats();
+    cluster.triggerBlockReports();
+    
+    banner("Metadata after nodes have all block-reported");
+    doMetasave(nn2);
+    
+    // The block should no longer be postponed.
+    assertEquals(0, nn2.getNamesystem().getPostponedMisreplicatedBlocks());
+    
+    // Wait for NN2 to enact its deletions (replication monitor has to run, etc)
+    BlockManagerTestUtil.computeInvalidationWork(
+        nn2.getNamesystem().getBlockManager());
+
+    HATestUtil.waitForNNToIssueDeletions(nn2);
+    cluster.triggerHeartbeats();
+    HATestUtil.waitForDNDeletions(cluster);
+    cluster.triggerDeletionReports();
+    assertEquals(0, nn2.getNamesystem().getUnderReplicatedBlocks());
+    assertEquals(0, nn2.getNamesystem().getPendingReplicationBlocks());
+    
+    banner("Making sure the file is still readable");
+    FileSystem fs2 = cluster.getFileSystem(1);
+    DFSTestUtil.readFile(fs2, TEST_FILE_PATH);
+  }
+  
+  /**
+   * Regression test for HDFS-2742. The issue in this bug was:
+   * - DN does a block report while file is open. This BR contains
+   *   the block in RBW state.
+   * - Standby queues the RBW state in PendingDatanodeMessages
+   * - Standby processes edit logs during failover. Before fixing
+   *   this bug, it was mistakenly applying the RBW reported state
+   *   after the block had been completed, causing the block to get
+   *   marked corrupt. Instead, we should now be applying the RBW
+   *   message on OP_ADD, and then the FINALIZED message on OP_CLOSE.
+   */
+  @Test
+  public void testBlockReportsWhileFileBeingWritten() throws Exception {
+    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
+    try {
+      AppendTestUtil.write(out, 0, 10);
+      out.hflush();
+      
+      // Block report will include the RBW replica, but will be
+      // queued on the StandbyNode.
+      cluster.triggerBlockReports();
+      
+    } finally {
+      IOUtils.closeStream(out);
+    }
+
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+    
+    // Verify that no replicas are marked corrupt, and that the
+    // file is readable from the failed-over standby.
+    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
+    BlockManagerTestUtil.updateState(nn2.getNamesystem().getBlockManager());
+    assertEquals(0, nn1.getNamesystem().getCorruptReplicaBlocks());
+    assertEquals(0, nn2.getNamesystem().getCorruptReplicaBlocks());
+    
+    DFSTestUtil.readFile(fs, TEST_FILE_PATH);
+  }
+  
+  /**
+   * Test that, when a block is re-opened for append, the related
+   * datanode messages are correctly queued by the SBN because
+   * they have future states and genstamps.
+   */
+  @Test
+  public void testQueueingWithAppend() throws Exception {
+    int numQueued = 0;
+    int numDN = cluster.getDataNodes().size();
+    
+    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
+    try {
+      AppendTestUtil.write(out, 0, 10);
+      out.hflush();
+
+      // Opening the file will report RBW replicas, but will be
+      // queued on the StandbyNode.
+      numQueued += numDN; // RBW messages
+    } finally {
+      IOUtils.closeStream(out);
+      numQueued += numDN; // blockReceived messages
+    }
+    
+    cluster.triggerBlockReports();
+    numQueued += numDN;
+    
+    try {
+      out = fs.append(TEST_FILE_PATH);
+      AppendTestUtil.write(out, 10, 10);
+      // RBW replicas once it's opened for append
+      numQueued += numDN;
+
+    } finally {
+      IOUtils.closeStream(out);
+      numQueued += numDN; // blockReceived
+    }
+    
+    cluster.triggerBlockReports();
+    numQueued += numDN;
+
+    assertEquals(numQueued, cluster.getNameNode(1).getNamesystem().
+        getPendingDataNodeMessageCount());
+
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+    
+    // Verify that no replicas are marked corrupt, and that the
+    // file is readable from the failed-over standby.
+    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
+    BlockManagerTestUtil.updateState(nn2.getNamesystem().getBlockManager());
+    assertEquals(0, nn1.getNamesystem().getCorruptReplicaBlocks());
+    assertEquals(0, nn2.getNamesystem().getCorruptReplicaBlocks());
+    
+    AppendTestUtil.check(fs, TEST_FILE_PATH, 20);
+  }
+  
+  /**
+   * Another regression test for HDFS-2742. This tests the following sequence:
+   * - DN does a block report while file is open. This BR contains
+   *   the block in RBW state.
+   * - The block report is delayed in reaching the standby.
+   * - The file is closed.
+   * - The standby processes the OP_ADD and OP_CLOSE operations before
+   *   the RBW block report arrives.
+   * - The standby should not mark the block as corrupt.
+   */
+  @Test
+  public void testRBWReportArrivesAfterEdits() throws Exception {
+    final CountDownLatch brFinished = new CountDownLatch(1);
+    DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG) {
+      @Override
+      protected Object passThrough(InvocationOnMock invocation)
+          throws Throwable {
+        try {
+          return super.passThrough(invocation);
+        } finally {
+          // inform the test that our block report went through.
+          brFinished.countDown();
+        }
+      }
+    };
+
+    FSDataOutputStream out = fs.create(TEST_FILE_PATH);
+    try {
+      AppendTestUtil.write(out, 0, 10);
+      out.hflush();
+
+      DataNode dn = cluster.getDataNodes().get(0);
+      DatanodeProtocolClientSideTranslatorPB spy =
+        DataNodeAdapter.spyOnBposToNN(dn, nn2);
+      
+      Mockito.doAnswer(delayer)
+        .when(spy).blockReport(
+          Mockito.<DatanodeRegistration>anyObject(),
+          Mockito.anyString(),
+          Mockito.<StorageBlockReport[]>anyObject());
+      dn.scheduleAllBlockReport(0);
+      delayer.waitForCall();
+      
+    } finally {
+      IOUtils.closeStream(out);
+    }
+
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+    
+    delayer.proceed();
+    brFinished.await();
+    
+    // Verify that no replicas are marked corrupt, and that the
+    // file is readable from the failed-over standby.
+    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
+    BlockManagerTestUtil.updateState(nn2.getNamesystem().getBlockManager());
+    assertEquals(0, nn1.getNamesystem().getCorruptReplicaBlocks());
+    assertEquals(0, nn2.getNamesystem().getCorruptReplicaBlocks());
+    
+    DFSTestUtil.readFile(fs, TEST_FILE_PATH);
+  }
+
+  /**
+   * Print a big banner in the test log to make debug easier.
+   */
+  private void banner(String string) {
+    LOG.info("\n\n\n\n================================================\n" +
+        string + "\n" +
+        "==================================================\n\n");
+  }
+
+  private void doMetasave(NameNode nn2) {
+    nn2.getNamesystem().writeLock();
+    try {
+      PrintWriter pw = new PrintWriter(System.err);
+      nn2.getNamesystem().getBlockManager().metaSave(pw);
+      pw.flush();
+    } finally {
+      nn2.getNamesystem().writeUnlock();
+    }
+  }
+
+  private void waitForTrueReplication(final MiniDFSCluster cluster,
+      final ExtendedBlock block, final int waitFor) throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          return getTrueReplication(cluster, block) == waitFor;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }, 500, 10000);
+  }
+
+  private int getTrueReplication(MiniDFSCluster cluster, ExtendedBlock block)
+      throws IOException {
+    int count = 0;
+    for (DataNode dn : cluster.getDataNodes()) {
+      if (DataNodeTestUtils.getFSDataset(dn).getStoredBlock(
+          block.getBlockPoolId(), block.getBlockId()) != null) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  /**
+   * A BlockPlacementPolicy which, rather than using space available, makes
+   * random decisions about which excess replica to delete. This is because,
+   * in the test cases, the two NNs will usually (but not quite always)
+   * make the same decision of which replica to delete. The fencing issues
+   * are exacerbated when the two NNs make different decisions, which can
+   * happen in "real life" when they have slightly out-of-sync heartbeat
+   * information regarding disk usage.
+   */
+  public static class RandomDeleterPolicy extends BlockPlacementPolicyDefault {
+
+    public RandomDeleterPolicy() {
+      super();
+    }
+
+    @Override
+    public DatanodeDescriptor chooseReplicaToDelete(FSInodeInfo inode,
+        Block block, short replicationFactor,
+        Collection<DatanodeDescriptor> first,
+        Collection<DatanodeDescriptor> second) {
+      
+      Collection<DatanodeDescriptor> chooseFrom =
+        !first.isEmpty() ? first : second;
+
+      List<DatanodeDescriptor> l = Lists.newArrayList(chooseFrom);
+      return l.get(DFSUtil.getRandom().nextInt(l.size()));
+    }
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
new file mode 100644
index 0000000..95d5eb9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
@@ -0,0 +1,148 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.base.Supplier;
+
+
+/**
+ * Stress-test for potential bugs when replication is changing
+ * on blocks during a failover.
+ */
+public class TestDNFencingWithReplication {
+  static {
+    ((Log4JLogger)FSNamesystem.auditLog).getLogger().setLevel(Level.WARN);
+    ((Log4JLogger)Server.LOG).getLogger().setLevel(Level.FATAL);
+    ((Log4JLogger)LogFactory.getLog(
+        "org.apache.hadoop.io.retry.RetryInvocationHandler"))
+        .getLogger().setLevel(Level.FATAL);
+  }
+
+  private static final int NUM_THREADS = 20;
+  // How long should the test try to run for. In practice
+  // it runs for ~20-30s longer than this constant due to startup/
+  // shutdown time.
+  private static final long RUNTIME = 35000;
+  private static final int BLOCK_SIZE = 1024;
+  
+  private static class ReplicationToggler extends RepeatingTestThread {
+    private final FileSystem fs;
+    private final Path path;
+
+    public ReplicationToggler(TestContext ctx, FileSystem fs, Path p) {
+      super(ctx);
+      this.fs = fs;
+      this.path = p;
+    }
+
+    @Override
+    public void doAnAction() throws Exception {
+      fs.setReplication(path, (short)1);
+      waitForReplicas(1);
+      fs.setReplication(path, (short)2);
+      waitForReplicas(2);
+    }
+    
+    private void waitForReplicas(final int replicas) throws Exception {
+      try {
+        GenericTestUtils.waitFor(new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            try {
+              BlockLocation[] blocks = fs.getFileBlockLocations(path, 0, 10);
+              Assert.assertEquals(1, blocks.length);
+              return blocks[0].getHosts().length == replicas;
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+          }
+        }, 100, 60000);
+      } catch (TimeoutException te) {
+        throw new IOException("Timed out waiting for " + replicas + " replicas " +
+            "on path " + path);
+      }
+    }
+    
+    public String toString() {
+      return "Toggler for " + path;
+    }
+  }
+  
+  @Test
+  public void testFencingStress() throws Exception {
+    HAStressTestHarness harness = new HAStressTestHarness();
+    harness.conf.setInt(
+        DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
+
+    final MiniDFSCluster cluster = harness.startCluster();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      
+      FileSystem fs = harness.getFailoverFs();
+      TestContext togglers = new TestContext();
+      for (int i = 0; i < NUM_THREADS; i++) {
+        Path p = new Path("/test-" + i);
+        DFSTestUtil.createFile(fs, p, BLOCK_SIZE*10, (short)3, (long)i);
+        togglers.addThread(new ReplicationToggler(togglers, fs, p));
+      }
+      
+      // Start a separate thread which will make sure that replication
+      // happens quickly by triggering deletion reports and replication
+      // work calculation frequently.
+      harness.addReplicationTriggerThread(500);
+      harness.addFailoverThread(5000);
+      harness.startThreads();
+      togglers.startThreads();
+      
+      togglers.waitFor(RUNTIME);
+      togglers.stop();
+      harness.stopThreads();
+
+      // CHeck that the files can be read without throwing
+      for (int i = 0; i < NUM_THREADS; i++) {
+        Path p = new Path("/test-" + i);
+        DFSTestUtil.readFile(fs, p);
+      }
+    } finally {
+      System.err.println("===========================\n\n\n\n");
+      harness.shutdown();
+    }
+
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
new file mode 100644
index 0000000..561e4d6
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
@@ -0,0 +1,234 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+
+/**
+ * Test case for client support of delegation tokens in an HA cluster.
+ * See HDFS-2904 for more info.
+ **/
+public class TestDelegationTokensWithHA {
+  private static Configuration conf = new Configuration();
+  private static final Log LOG =
+    LogFactory.getLog(TestDelegationTokensWithHA.class);
+  private static MiniDFSCluster cluster;
+  private static NameNode nn0;
+  private static NameNode nn1;
+  private static FileSystem fs;
+  private static DelegationTokenSecretManager dtSecretManager;
+  private static DistributedFileSystem dfs;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.set("hadoop.security.auth_to_local",
+        "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
+
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+    cluster.waitActive();
+    
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+    dfs = (DistributedFileSystem)fs;
+
+    cluster.transitionToActive(0);
+    dtSecretManager = NameNodeAdapter.getDtSecretManager(
+        nn0.getNamesystem());
+  }
+
+  @AfterClass
+  public static void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+
+  @Test
+  public void testDelegationTokenDFSApi() throws Exception {
+    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
+    DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
+    byte[] tokenId = token.getIdentifier();
+    identifier.readFields(new DataInputStream(
+             new ByteArrayInputStream(tokenId)));
+
+    // Ensure that it's present in the NN's secret manager and can
+    // be renewed directly from there.
+    LOG.info("A valid token should have non-null password, " +
+        "and should be renewed successfully");
+    assertTrue(null != dtSecretManager.retrievePassword(identifier));
+    dtSecretManager.renewToken(token, "JobTracker");
+    
+    // Use the client conf with the failover info present to check
+    // renewal.
+    Configuration clientConf = dfs.getConf();
+    doRenewOrCancel(token, clientConf, TokenTestAction.RENEW);
+    
+    // Using a configuration that doesn't have the logical nameservice
+    // configured should result in a reasonable error message.
+    Configuration emptyConf = new Configuration();
+    try {
+      doRenewOrCancel(token, emptyConf, TokenTestAction.RENEW);
+      fail("Did not throw trying to renew with an empty conf!");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Unable to map logical nameservice URI", ioe);
+    }
+
+    
+    // Ensure that the token can be renewed again after a failover.
+    cluster.transitionToStandby(0);
+    cluster.transitionToActive(1);
+    doRenewOrCancel(token, clientConf, TokenTestAction.RENEW);
+    
+    doRenewOrCancel(token, clientConf, TokenTestAction.CANCEL);
+  }
+  
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testDelegationTokenWithDoAs() throws Exception {
+    final Token<DelegationTokenIdentifier> token = 
+        dfs.getDelegationToken("JobTracker");
+    final UserGroupInformation longUgi = UserGroupInformation
+        .createRemoteUser("JobTracker/foo.com@FOO.COM");
+    final UserGroupInformation shortUgi = UserGroupInformation
+        .createRemoteUser("JobTracker");
+    longUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        DistributedFileSystem dfs = (DistributedFileSystem)
+            HATestUtil.configureFailoverFs(cluster, conf);
+        // try renew with long name
+        dfs.renewDelegationToken(token);
+        return null;
+      }
+    });
+    shortUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        DistributedFileSystem dfs = (DistributedFileSystem)
+            HATestUtil.configureFailoverFs(cluster, conf);
+        dfs.renewDelegationToken(token);
+        return null;
+      }
+    });
+    longUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        DistributedFileSystem dfs = (DistributedFileSystem)
+            HATestUtil.configureFailoverFs(cluster, conf);
+        // try cancel with long name
+        dfs.cancelDelegationToken(token);
+        return null;
+      }
+    });
+  }
+  
+  @Test
+  public void testHAUtilClonesDelegationTokens() throws Exception {
+    final Token<DelegationTokenIdentifier> token = 
+      dfs.getDelegationToken("test");
+
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test");
+    
+    URI haUri = new URI("hdfs://my-ha-uri/");
+    token.setService(HAUtil.buildTokenServiceForLogicalUri(haUri));
+    ugi.addToken(token);
+    HAUtil.cloneDelegationTokenForLogicalUri(ugi, haUri, nn0.getNameNodeAddress());
+    HAUtil.cloneDelegationTokenForLogicalUri(ugi, haUri, nn1.getNameNodeAddress());
+    
+    Collection<Token<? extends TokenIdentifier>> tokens = ugi.getTokens();
+    assertEquals(3, tokens.size());
+    
+    LOG.info("Tokens:\n" + Joiner.on("\n").join(tokens));
+    
+    // check that the token selected for one of the physical IPC addresses
+    // matches the one we received
+    InetSocketAddress addr = nn0.getNameNodeAddress();
+    Text ipcDtService = new Text(
+        addr.getAddress().getHostAddress() + ":" + addr.getPort());
+    Token<DelegationTokenIdentifier> token2 =
+        DelegationTokenSelector.selectHdfsDelegationToken(ipcDtService, ugi);
+    assertNotNull(token2);
+    assertArrayEquals(token.getIdentifier(), token2.getIdentifier());
+    assertArrayEquals(token.getPassword(), token2.getPassword());
+  }
+  
+  enum TokenTestAction {
+    RENEW, CANCEL;
+  }
+  
+  private static void doRenewOrCancel(
+      final Token<DelegationTokenIdentifier> token, final Configuration conf,
+      final TokenTestAction action)
+      throws IOException, InterruptedException {
+    UserGroupInformation.createRemoteUser("JobTracker").doAs(
+        new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            switch (action) {
+            case RENEW:
+              token.renew(conf);
+              break;
+            case CANCEL:
+              token.cancel(conf);
+              break;
+            default:
+              fail("bad action:" + action);
+            }
+            return null;
+          }
+        });
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
new file mode 100644
index 0000000..bc5c487
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -0,0 +1,162 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.Test;
+
+import com.google.common.base.Supplier;
+
+public class TestEditLogTailer {
+  
+  private static final String DIR_PREFIX = "/dir";
+  private static final int DIRS_TO_MAKE = 20;
+  static final long SLEEP_TIME = 1000;
+  static final long NN_LAG_TIMEOUT = 10 * 1000;
+  
+  static {
+    ((Log4JLogger)FSImage.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)EditLogTailer.LOG).getLogger().setLevel(Level.ALL);
+  }
+  
+  @Test
+  public void testTailer() throws IOException, InterruptedException,
+      ServiceFailedException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+
+    HAUtil.setAllowStandbyReads(conf, true);
+    
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+    cluster.waitActive();
+    
+    cluster.transitionToActive(0);
+    
+    NameNode nn1 = cluster.getNameNode(0);
+    NameNode nn2 = cluster.getNameNode(1);
+    try {
+      for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
+        NameNodeAdapter.mkdirs(nn1, getDirPath(i),
+            new PermissionStatus("test","test", new FsPermission((short)00755)),
+            true);
+      }
+      
+      HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
+      
+      for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
+        assertTrue(NameNodeAdapter.getFileInfo(nn2,
+            getDirPath(i), false).isDir());
+      }
+      
+      for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
+        NameNodeAdapter.mkdirs(nn1, getDirPath(i),
+            new PermissionStatus("test","test", new FsPermission((short)00755)),
+            true);
+      }
+      
+      HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
+      
+      for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
+        assertTrue(NameNodeAdapter.getFileInfo(nn2,
+            getDirPath(i), false).isDir());
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test
+  public void testNN0TriggersLogRolls() throws Exception {
+    testStandbyTriggersLogRolls(0);
+  }
+  
+  @Test
+  public void testNN1TriggersLogRolls() throws Exception {
+    testStandbyTriggersLogRolls(1);
+  }
+  
+  private static void testStandbyTriggersLogRolls(int activeIndex)
+      throws Exception {
+    Configuration conf = new Configuration();
+    // Roll every 1s
+    conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    
+    // Have to specify IPC ports so the NNs can talk to each other.
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+      .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002)));
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(topology)
+      .numDataNodes(0)
+      .build();
+    try {
+      cluster.transitionToActive(activeIndex);
+      waitForLogRollInSharedDir(cluster, 3);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  private static String getDirPath(int suffix) {
+    return DIR_PREFIX + suffix;
+  }
+  
+  private static void waitForLogRollInSharedDir(MiniDFSCluster cluster,
+      long startTxId) throws Exception {
+    URI sharedUri = cluster.getSharedEditsDir(0, 1);
+    File sharedDir = new File(sharedUri.getPath(), "current");
+    final File expectedLog = new File(sharedDir,
+        NNStorage.getInProgressEditsFileName(startTxId));
+    
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return expectedLog.exists();
+      }
+    }, 100, 10000);
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
new file mode 100644
index 0000000..a245301
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
@@ -0,0 +1,182 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+/**
+ * Test cases for the handling of edit logs during failover
+ * and startup of the standby node.
+ */
+public class TestEditLogsDuringFailover {
+  private static final Log LOG =
+    LogFactory.getLog(TestEditLogsDuringFailover.class);
+  private static final int NUM_DIRS_IN_LOG = 5;
+  
+  @Test
+  public void testStartup() throws Exception {
+    Configuration conf = new Configuration();
+    HAUtil.setAllowStandbyReads(conf, true);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+    try {
+      // During HA startup, both nodes should be in
+      // standby and we shouldn't have any edits files
+      // in any edits directory!
+      List<URI> allDirs = Lists.newArrayList();
+      allDirs.addAll(cluster.getNameDirs(0));
+      allDirs.addAll(cluster.getNameDirs(1));
+      allDirs.add(cluster.getSharedEditsDir(0, 1));
+      assertNoEditFiles(allDirs);
+      
+      // Set the first NN to active, make sure it creates edits
+      // in its own dirs and the shared dir. The standby
+      // should still have no edits!
+      cluster.getNameNode(0).getRpcServer().transitionToActive();
+      
+      assertEditFiles(cluster.getNameDirs(0),
+          NNStorage.getInProgressEditsFileName(1));
+      assertEditFiles(
+          Collections.singletonList(cluster.getSharedEditsDir(0, 1)),
+          NNStorage.getInProgressEditsFileName(1));
+      assertNoEditFiles(cluster.getNameDirs(1));
+      
+      cluster.getNameNode(0).getRpcServer().mkdirs("/test",
+          FsPermission.createImmutable((short)0755), true);
+
+      // Restarting the standby should not finalize any edits files
+      // in the shared directory when it starts up!
+      cluster.restartNameNode(1);
+      
+      assertEditFiles(cluster.getNameDirs(0),
+          NNStorage.getInProgressEditsFileName(1));
+      assertEditFiles(
+          Collections.singletonList(cluster.getSharedEditsDir(0, 1)),
+          NNStorage.getInProgressEditsFileName(1));
+      assertNoEditFiles(cluster.getNameDirs(1));
+      
+      // Additionally it should not have applied any in-progress logs
+      // at start-up -- otherwise, it would have read half-way into
+      // the current log segment, and on the next roll, it would have to
+      // either replay starting in the middle of the segment (not allowed)
+      // or double-replay the edits (incorrect).
+      assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test", true));
+      
+      cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
+          FsPermission.createImmutable((short)0755), true);
+
+      // If we restart NN0, it'll come back as standby, and we can
+      // transition NN1 to active and make sure it reads edits correctly at this point.
+      cluster.restartNameNode(0);
+      cluster.getNameNode(1).getRpcServer().transitionToActive();
+
+      // NN1 should have both the edits that came before its restart, and the edits that
+      // came after its restart.
+      assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test", true));
+      assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test2", true));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test
+  public void testFailoverFinalizesAndReadsInProgress() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+    try {
+      // Create a fake in-progress edit-log in the shared directory
+      URI sharedUri = cluster.getSharedEditsDir(0, 1);
+      File sharedDir = new File(sharedUri.getPath(), "current");
+      FSImageTestUtil.createAbortedLogWithMkdirs(sharedDir, NUM_DIRS_IN_LOG, 1);
+      assertEditFiles(Collections.singletonList(sharedUri),
+          NNStorage.getInProgressEditsFileName(1));
+
+      // Transition one of the NNs to active
+      cluster.getNameNode(0).getRpcServer().transitionToActive();
+      
+      // In the transition to active, it should have read the log -- and
+      // hence see one of the dirs we made in the fake log.
+      String testPath = "/dir" + NUM_DIRS_IN_LOG;
+      assertNotNull(cluster.getNameNode(0).getRpcServer().getFileInfo(testPath));
+      
+      // It also should have finalized that log in the shared directory and started
+      // writing to a new one at the next txid.
+      assertEditFiles(Collections.singletonList(sharedUri),
+          NNStorage.getFinalizedEditsFileName(1, NUM_DIRS_IN_LOG + 1),
+          NNStorage.getInProgressEditsFileName(NUM_DIRS_IN_LOG + 2));
+    } finally {
+      cluster.shutdown();
+    }
+
+  }
+
+  /**
+   * Check that no edits files are present in the given storage dirs.
+   */
+  private void assertNoEditFiles(Iterable<URI> dirs) throws IOException {
+    assertEditFiles(dirs, new String[]{});
+  }
+  
+  /**
+   * Check that the given list of edits files are present in the given storage
+   * dirs.
+   */
+  private void assertEditFiles(Iterable<URI> dirs, String ... files)
+      throws IOException {
+    for (URI u : dirs) {
+      File editDirRoot = new File(u.getPath());
+      File editDir = new File(editDirRoot, "current");
+      GenericTestUtils.assertExists(editDir);
+      if (files.length == 0) {
+        LOG.info("Checking no edit files exist in " + editDir);
+      } else {
+        LOG.info("Checking for following edit files in " + editDir
+            + ": " + Joiner.on(",").join(files));
+      }
+      
+      GenericTestUtils.assertGlobEquals(editDir, "edits_.*", files);
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java
new file mode 100644
index 0000000..cc9552a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java
@@ -0,0 +1,196 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.base.Joiner;
+
+public class TestFailureOfSharedDir {
+  
+  private static final Log LOG = LogFactory.getLog(TestFailureOfSharedDir.class);
+
+  /**
+   * Test that the shared edits dir is automatically added to the list of edits
+   * dirs that are marked required.
+   */
+  @Test
+  public void testSharedDirIsAutomaticallyMarkedRequired()
+      throws URISyntaxException {
+    URI foo = new URI("file:/foo");
+    URI bar = new URI("file:/bar");
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, Joiner.on(",").join(foo, bar));
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY, foo.toString());
+    assertFalse(FSNamesystem.getRequiredNamespaceEditsDirs(conf).contains(
+        bar));
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, bar.toString());
+    Collection<URI> requiredEditsDirs = FSNamesystem
+        .getRequiredNamespaceEditsDirs(conf); 
+    assertTrue(Joiner.on(",").join(requiredEditsDirs) + " does not contain " + bar,
+        requiredEditsDirs.contains(bar));
+  }
+
+  /**
+   * Multiple shared edits directories is an invalid configuration.
+   */
+  @Test
+  public void testMultipleSharedDirsFails() throws Exception {
+    Configuration conf = new Configuration();
+    URI sharedA = new URI("file:///shared-A");
+    URI sharedB = new URI("file:///shared-B");
+    URI localA = new URI("file:///local-A");
+
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        Joiner.on(",").join(sharedA,sharedB));
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        localA.toString());
+
+    try {
+      FSNamesystem.getNamespaceEditsDirs(conf);
+      fail("Allowed multiple shared edits directories");
+    } catch (IOException ioe) {
+      assertEquals("Multiple shared edits directories are not yet supported",
+          ioe.getMessage());
+    }
+  }
+  
+  /**
+   * Make sure that the shared edits dirs are listed before non-shared dirs
+   * when the configuration is parsed. This ensures that the shared journals
+   * are synced before the local ones.
+   */
+  @Test
+  public void testSharedDirsComeFirstInEditsList() throws Exception {
+    Configuration conf = new Configuration();
+    URI sharedA = new URI("file:///shared-A");
+    URI localA = new URI("file:///local-A");
+    URI localB = new URI("file:///local-B");
+    URI localC = new URI("file:///local-C");
+    
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        sharedA.toString());
+    // List them in reverse order, to make sure they show up in
+    // the order listed, regardless of lexical sort order.
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        Joiner.on(",").join(localC, localB, localA));
+    List<URI> dirs = FSNamesystem.getNamespaceEditsDirs(conf);
+    assertEquals(
+        "Shared dirs should come first, then local dirs, in the order " +
+        "they were listed in the configuration.",
+        Joiner.on(",").join(sharedA, localC, localB, localA),
+        Joiner.on(",").join(dirs));
+  }
+  
+  /**
+   * Test that marking the shared edits dir as being "required" causes the NN to
+   * fail if that dir can't be accessed.
+   */
+  @Test
+  public void testFailureOfSharedDir() throws Exception {
+    Configuration conf = new Configuration();
+    
+    // The shared edits dir will automatically be marked required.
+    MiniDFSCluster cluster = null;
+    File sharedEditsDir = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(0)
+        .build();
+      
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      
+      assertTrue(fs.mkdirs(new Path("/test1")));
+      
+      // Blow away the shared edits dir.
+      Runtime mockRuntime = Mockito.mock(Runtime.class);
+      URI sharedEditsUri = cluster.getSharedEditsDir(0, 1);
+      sharedEditsDir = new File(sharedEditsUri);
+      assertEquals(0, FileUtil.chmod(sharedEditsDir.getAbsolutePath(), "-w",
+          true));
+
+      NameNode nn0 = cluster.getNameNode(0);
+      nn0.getNamesystem().getFSImage().getEditLog().getJournalSet()
+          .setRuntimeForTesting(mockRuntime);
+      try {
+        // Make sure that subsequent operations on the NN fail.
+        nn0.getRpcServer().rollEditLog();
+        fail("Succeeded in rolling edit log despite shared dir being deleted");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "Unable to start log segment 4: too few journals successfully started",
+            ioe);
+        // By current policy the NN should exit upon this error.
+        // exit() should be called once, but since it is mocked, exit gets
+        // called once during FSEditsLog.endCurrentLogSegment() and then after
+        // that during FSEditsLog.startLogSegment(). So the check is atLeast(1)
+        Mockito.verify(mockRuntime, Mockito.atLeastOnce()).exit(
+            Mockito.anyInt());
+        LOG.info("Got expected exception", ioe);
+      }
+      
+      // Check that none of the edits dirs rolled, since the shared edits
+      // dir didn't roll. Regression test for HDFS-2874.
+      for (URI editsUri : cluster.getNameEditsDirs(0)) {
+        if (editsUri.equals(sharedEditsUri)) {
+          continue;
+        }
+        File editsDir = new File(editsUri.getPath());
+        File curDir = new File(editsDir, "current");
+        GenericTestUtils.assertGlobEquals(curDir,
+            "edits_.*",
+            NNStorage.getInProgressEditsFileName(1));
+      }
+    } finally {
+      if (sharedEditsDir != null) {
+        // without this test cleanup will fail
+        FileUtil.chmod(sharedEditsDir.getAbsolutePath(), "+w", true);
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
new file mode 100644
index 0000000..7bc2d8e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
@@ -0,0 +1,326 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ha.ServiceFailedException;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputException;
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.collect.ImmutableList;
+
+public class TestFailureToReadEdits {
+  
+  private static final Log LOG = LogFactory.getLog(TestFailureToReadEdits.class);
+  
+  private static final String TEST_DIR1 = "/test1";
+  private static final String TEST_DIR2 = "/test2";
+  private static final String TEST_DIR3 = "/test3";
+  
+  private Configuration conf;
+  private Runtime mockRuntime = mock(Runtime.class);
+  private MiniDFSCluster cluster;
+  private NameNode nn0;
+  private NameNode nn1;
+  private FileSystem fs;
+  
+  @Before
+  public void setUpCluster() throws Exception {
+    conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY, 10);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    HAUtil.setAllowStandbyReads(conf, true);
+    
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+      .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(topology)
+      .numDataNodes(0)
+      .build();
+    
+    cluster.waitActive();
+    
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    nn1.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
+    
+    cluster.transitionToActive(0);
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+  }
+  
+  @After
+  public void tearDownCluster() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+    
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that the standby NN won't double-replay earlier edits if it encounters
+   * a failure to read a later edit.
+   */
+  @Test
+  public void testFailuretoReadEdits() throws Exception {
+    assertTrue(fs.mkdirs(new Path(TEST_DIR1)));
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    
+    // If these two ops are applied twice, the first op will throw an
+    // exception the second time its replayed.
+    fs.setOwner(new Path(TEST_DIR1), "foo", "bar");
+    assertTrue(fs.delete(new Path(TEST_DIR1), true));
+    
+    // This op should get applied just fine.
+    assertTrue(fs.mkdirs(new Path(TEST_DIR2)));
+    
+    // This is the op the mocking will cause to fail to be read.
+    assertTrue(fs.mkdirs(new Path(TEST_DIR3)));
+    
+    LimitedEditLogAnswer answer = causeFailureOnEditLogRead();
+    
+    try {
+      HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+      fail("Standby fully caught up, but should not have been able to");
+    } catch (HATestUtil.CouldNotCatchUpException e) {
+      verify(mockRuntime, times(0)).exit(anyInt());
+    }
+    
+    // Null because it was deleted.
+    assertNull(NameNodeAdapter.getFileInfo(nn1,
+        TEST_DIR1, false));
+    // Should have been successfully created.
+    assertTrue(NameNodeAdapter.getFileInfo(nn1,
+        TEST_DIR2, false).isDir());
+    // Null because it hasn't been created yet.
+    assertNull(NameNodeAdapter.getFileInfo(nn1,
+        TEST_DIR3, false));
+    
+    // Now let the standby read ALL the edits.
+    answer.setThrowExceptionOnRead(false);
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    
+    // Null because it was deleted.
+    assertNull(NameNodeAdapter.getFileInfo(nn1,
+        TEST_DIR1, false));
+    // Should have been successfully created.
+    assertTrue(NameNodeAdapter.getFileInfo(nn1,
+        TEST_DIR2, false).isDir());
+    // Should now have been successfully created.
+    assertTrue(NameNodeAdapter.getFileInfo(nn1,
+        TEST_DIR3, false).isDir());
+  }
+  
+  /**
+   * Test the following case:
+   * 1. SBN is reading a finalized edits file when NFS disappears halfway
+   *    through (or some intermittent error happens)
+   * 2. SBN performs a checkpoint and uploads it to the NN
+   * 3. NN receives a checkpoint that doesn't correspond to the end of any log
+   *    segment
+   * 4. Both NN and SBN should be able to restart at this point.
+   * 
+   * This is a regression test for HDFS-2766.
+   */
+  @Test
+  public void testCheckpointStartingMidEditsFile() throws Exception {
+    assertTrue(fs.mkdirs(new Path(TEST_DIR1)));
+    
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    
+    // Once the standby catches up, it should notice that it needs to
+    // do a checkpoint and save one to its local directories.
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 3));
+    
+    // It should also upload it back to the active.
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 3));
+    
+    causeFailureOnEditLogRead();
+    
+    assertTrue(fs.mkdirs(new Path(TEST_DIR2)));
+    assertTrue(fs.mkdirs(new Path(TEST_DIR3)));
+    
+    try {
+      HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+      fail("Standby fully caught up, but should not have been able to");
+    } catch (HATestUtil.CouldNotCatchUpException e) {
+      verify(mockRuntime, times(0)).exit(anyInt());
+    }
+    
+    // 5 because we should get OP_START_LOG_SEGMENT and one successful OP_MKDIR
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 3, 5));
+    
+    // It should also upload it back to the active.
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 3, 5));
+
+    // Restart the active NN
+    cluster.restartNameNode(0);
+    
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 3, 5));
+    
+    FileSystem fs0 = null;
+    try {
+      // Make sure that when the active restarts, it loads all the edits.
+      fs0 = FileSystem.get(NameNode.getUri(nn0.getNameNodeAddress()),
+          conf);
+      
+      assertTrue(fs0.exists(new Path(TEST_DIR1)));
+      assertTrue(fs0.exists(new Path(TEST_DIR2)));
+      assertTrue(fs0.exists(new Path(TEST_DIR3)));
+    } finally {
+      if (fs0 != null)
+        fs0.close();
+    }
+  }
+
+  /**
+   * Ensure that the standby fails to become active if it cannot read all
+   * available edits in the shared edits dir when it is transitioning to active
+   * state.
+   */
+  @Test
+  public void testFailureToReadEditsOnTransitionToActive() throws Exception {
+    assertTrue(fs.mkdirs(new Path(TEST_DIR1)));
+    
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    
+    // It should also upload it back to the active.
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 3));
+    
+    causeFailureOnEditLogRead();
+    
+    assertTrue(fs.mkdirs(new Path(TEST_DIR2)));
+    assertTrue(fs.mkdirs(new Path(TEST_DIR3)));
+    
+    try {
+      HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+      fail("Standby fully caught up, but should not have been able to");
+    } catch (HATestUtil.CouldNotCatchUpException e) {
+      verify(mockRuntime, times(0)).exit(anyInt());
+    }
+    
+    // Shutdown the active NN.
+    cluster.shutdownNameNode(0);
+    
+    try {
+      // Transition the standby to active.
+      cluster.transitionToActive(1);
+      fail("Standby transitioned to active, but should not have been able to");
+    } catch (ServiceFailedException sfe) {
+      LOG.info("got expected exception: " + sfe.toString(), sfe);
+      assertTrue("Standby failed to catch up for some reason other than "
+          + "failure to read logs", sfe.toString().contains(
+              EditLogInputException.class.getName()));
+    }
+  }
+  
+  private LimitedEditLogAnswer causeFailureOnEditLogRead() throws IOException {
+    FSEditLog spyEditLog = spy(nn1.getNamesystem().getEditLogTailer()
+        .getEditLog());
+    LimitedEditLogAnswer answer = new LimitedEditLogAnswer(); 
+    doAnswer(answer).when(spyEditLog).selectInputStreams(
+        anyLong(), anyLong(), anyBoolean());
+    nn1.getNamesystem().getEditLogTailer().setEditLog(spyEditLog);
+    
+    return answer;
+  }
+  
+  private static class LimitedEditLogAnswer
+      implements Answer<Collection<EditLogInputStream>> {
+    
+    private boolean throwExceptionOnRead = true;
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Collection<EditLogInputStream> answer(InvocationOnMock invocation)
+        throws Throwable {
+      Collection<EditLogInputStream> streams = (Collection<EditLogInputStream>)
+          invocation.callRealMethod();
+  
+      if (!throwExceptionOnRead) {
+        return streams;
+      } else {
+        Collection<EditLogInputStream> ret = new LinkedList<EditLogInputStream>();
+        for (EditLogInputStream stream : streams) {
+          EditLogInputStream spyStream = spy(stream);
+          doAnswer(new Answer<FSEditLogOp>() {
+
+            @Override
+            public FSEditLogOp answer(InvocationOnMock invocation)
+                throws Throwable {
+              FSEditLogOp op = (FSEditLogOp) invocation.callRealMethod();
+              if (throwExceptionOnRead &&
+                  TEST_DIR3.equals(NameNodeAdapter.getMkdirOpPath(op))) {
+                throw new IOException("failed to read op creating " + TEST_DIR3);
+              } else {
+                return op;
+              }
+            }
+            
+          }).when(spyStream).readOp();
+          ret.add(spyStream);
+        }
+        return ret;
+      }
+    }
+    
+    public void setThrowExceptionOnRead(boolean throwExceptionOnRead) {
+      this.throwExceptionOnRead = throwExceptionOnRead;
+    }
+  }
+  
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestGetGroupsWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestGetGroupsWithHA.java
new file mode 100644
index 0000000..e548817
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestGetGroupsWithHA.java
@@ -0,0 +1,57 @@
+/**
+ * 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.namenode.ha;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.tools.GetGroups;
+import org.apache.hadoop.tools.GetGroupsTestBase;
+import org.apache.hadoop.util.Tool;
+import org.junit.After;
+import org.junit.Before;
+
+public class TestGetGroupsWithHA extends GetGroupsTestBase {
+  
+  private MiniDFSCluster cluster;
+  
+  @Before
+  public void setUpNameNode() throws IOException {
+    conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(0).build();
+    HATestUtil.setFailoverConfigurations(cluster, conf);
+  }
+  
+  @After
+  public void tearDownNameNode() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Override
+  protected Tool getTool(PrintStream o) {
+    return new GetGroups(conf, o);
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
new file mode 100644
index 0000000..9cd6ab7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
@@ -0,0 +1,126 @@
+/**
+ * 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.namenode.ha;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * Test cases that the HA configuration is reasonably validated and
+ * interpreted in various places. These should be proper unit tests
+ * which don't start daemons.
+ */
+public class TestHAConfiguration {
+
+  private FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
+
+  @Test
+  public void testCheckpointerValidityChecks() throws Exception {
+    try {
+      Configuration conf = new Configuration();
+      new StandbyCheckpointer(conf, fsn);
+      fail("Bad config did not throw an error");
+    } catch (IllegalArgumentException iae) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid URI for NameNode address", iae);
+    }
+  }
+
+  private Configuration getHAConf(String nsId, String host1, String host2) {
+    Configuration conf = new Configuration();
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, nsId);    
+    conf.set(DFSUtil.addKeySuffixes(
+        DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, nsId),
+        "nn1,nn2");    
+    conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
+    conf.set(DFSUtil.addKeySuffixes(
+        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, "nn1"),
+        host1 + ":12345");
+    conf.set(DFSUtil.addKeySuffixes(
+        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, "nn2"),
+        host2 + ":12345");
+    return conf;
+  }
+
+  @Test
+  public void testGetOtherNNHttpAddress() {
+    // Use non-local addresses to avoid host address matching
+    Configuration conf = getHAConf("ns1", "1.2.3.1", "1.2.3.2");
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, "ns1");
+
+    // This is done by the NN before the StandbyCheckpointer is created
+    NameNode.initializeGenericKeys(conf, "ns1", "nn1");
+
+    // Since we didn't configure the HTTP address, and the default is
+    // 0.0.0.0, it should substitute the address from the RPC configuration
+    // above.
+    StandbyCheckpointer checkpointer = new StandbyCheckpointer(conf, fsn);
+    assertEquals("1.2.3.2:" + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
+        checkpointer.getActiveNNAddress());
+  }
+  
+  /**
+   * Tests that the namenode edits dirs and shared edits dirs are gotten with
+   * duplicates removed
+   */
+  @Test
+  public void testHAUniqueEditDirs() throws IOException {
+    Configuration conf = new Configuration();
+
+    conf.set(DFS_NAMENODE_EDITS_DIR_KEY, "file://edits/dir, "
+        + "file://edits/shared/dir"); // overlapping
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, "file://edits/shared/dir");
+
+    // getNamespaceEditsDirs removes duplicates across edits and shared.edits
+    Collection<URI> editsDirs = FSNamesystem.getNamespaceEditsDirs(conf);
+    assertEquals(2, editsDirs.size());
+  }
+  
+  /**
+   * Test that the 2NN does not start if given a config with HA NNs.
+   */
+  @Test
+  public void testSecondaryNameNodeDoesNotStart() throws IOException {
+    // Note we're not explicitly setting the nameservice Id in the
+    // config as it is not required to be set and we want to test
+    // that we can determine if HA is enabled when the nameservice Id
+    // is not explicitly defined.
+    Configuration conf = getHAConf("ns1", "1.2.3.1", "1.2.3.2");
+    try {
+      new SecondaryNameNode(conf);
+      fail("Created a 2NN with an HA config");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot use SecondaryNameNode in an HA cluster", ioe);
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java
new file mode 100644
index 0000000..10218f2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java
@@ -0,0 +1,103 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+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.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.tools.DFSck;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.junit.Test;
+
+public class TestHAFsck {
+  
+  static {
+    ((Log4JLogger)LogFactory.getLog(DFSUtil.class)).getLogger().setLevel(Level.ALL);
+  }
+  
+  /**
+   * Test that fsck still works with HA enabled.
+   */
+  @Test
+  public void testHaFsck() throws Exception {
+    Configuration conf = new Configuration();
+    
+    // need some HTTP ports
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+      .addNameservice(new MiniDFSNNTopology.NSConf("ha-nn-uri-0")
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+    
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(topology)
+      .numDataNodes(0)
+      .build();
+    FileSystem fs = null;
+    try {
+      cluster.waitActive();
+    
+      cluster.transitionToActive(0);
+      
+      // Make sure conf has the relevant HA configs.
+      HATestUtil.setFailoverConfigurations(cluster, conf, "ha-nn-uri-0", 0);
+      
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      fs.mkdirs(new Path("/test1"));
+      fs.mkdirs(new Path("/test2"));
+      
+      runFsck(conf);
+      
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      
+      runFsck(conf);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  static void runFsck(Configuration conf) throws Exception {
+    ByteArrayOutputStream bStream = new ByteArrayOutputStream();
+    PrintStream out = new PrintStream(bStream, true);
+    int errCode = ToolRunner.run(new DFSck(conf, out),
+        new String[]{"/", "-files"});
+    String result = bStream.toString();
+    System.out.println("output from fsck:\n" + result);
+    Assert.assertEquals(0, errCode);
+    assertTrue(result.contains("/test1"));
+    assertTrue(result.contains("/test2"));
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java
new file mode 100644
index 0000000..cc85c83
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java
@@ -0,0 +1,109 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+
+/**
+ * Make sure HA-related metrics are updated and reported appropriately.
+ */
+public class TestHAMetrics {
+  
+  private static final Log LOG = LogFactory.getLog(TestHAMetrics.class);
+  
+  @Test
+  public void testHAMetrics() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY, Integer.MAX_VALUE);
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(1)
+        .build();
+    FileSystem fs = null;
+    try {
+      cluster.waitActive();
+      
+      FSNamesystem nn0 = cluster.getNamesystem(0);
+      FSNamesystem nn1 = cluster.getNamesystem(1);
+      
+      assertEquals(nn0.getHAState(), "standby");
+      assertTrue(0 < nn0.getMillisSinceLastLoadedEdits());
+      assertEquals(nn1.getHAState(), "standby");
+      assertTrue(0 < nn1.getMillisSinceLastLoadedEdits());
+
+      cluster.transitionToActive(0);
+      
+      assertEquals("active", nn0.getHAState());
+      assertEquals(0, nn0.getMillisSinceLastLoadedEdits());
+      assertEquals("standby", nn1.getHAState());
+      assertTrue(0 < nn1.getMillisSinceLastLoadedEdits());
+      
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      
+      assertEquals("standby", nn0.getHAState());
+      assertTrue(0 < nn0.getMillisSinceLastLoadedEdits());
+      assertEquals("active", nn1.getHAState());
+      assertEquals(0, nn1.getMillisSinceLastLoadedEdits());
+      
+      Thread.sleep(2000); // make sure standby gets a little out-of-date
+      assertTrue(2000 <= nn0.getMillisSinceLastLoadedEdits());
+      
+      assertEquals(0, nn0.getPendingDataNodeMessageCount());
+      assertEquals(0, nn1.getPendingDataNodeMessageCount());
+      
+      fs = HATestUtil.configureFailoverFs(cluster, conf);
+      DFSTestUtil.createFile(fs, new Path("/foo"),
+          10, (short)1, 1L);
+      
+      assertTrue(0 < nn0.getPendingDataNodeMessageCount());
+      assertEquals(0, nn1.getPendingDataNodeMessageCount());
+      long millisSinceLastLoadedEdits = nn0.getMillisSinceLastLoadedEdits();
+      
+      HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(1),
+          cluster.getNameNode(0));
+      
+      assertEquals(0, nn0.getPendingDataNodeMessageCount());
+      assertEquals(0, nn1.getPendingDataNodeMessageCount());
+      long newMillisSinceLastLoadedEdits = nn0.getMillisSinceLastLoadedEdits();
+      // Since we just waited for the standby to catch up, the time since we
+      // last loaded edits should be very low.
+      assertTrue("expected " + millisSinceLastLoadedEdits + " > " +
+          newMillisSinceLastLoadedEdits,
+          millisSinceLastLoadedEdits > newMillisSinceLastLoadedEdits);
+    } finally {
+      IOUtils.cleanup(LOG, fs);
+      cluster.shutdown();
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
new file mode 100644
index 0000000..8790d0f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
@@ -0,0 +1,648 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+
+/**
+ * Tests that exercise safemode in an HA cluster.
+ */
+public class TestHASafeMode {
+  private static final Log LOG = LogFactory.getLog(TestHASafeMode.class);
+  private static final int BLOCK_SIZE = 1024;
+  private NameNode nn0;
+  private NameNode nn1;
+  private FileSystem fs;
+  private MiniDFSCluster cluster;
+  private Runtime mockRuntime = mock(Runtime.class);
+  
+  static {
+    ((Log4JLogger)LogFactory.getLog(FSImage.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+  }
+  
+  @Before
+  public void setupCluster() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .waitSafeMode(false)
+      .build();
+    cluster.waitActive();
+    
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+    
+    nn0.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
+
+    cluster.transitionToActive(0);
+  }
+  
+  @After
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      verify(mockRuntime, times(0)).exit(anyInt());
+      cluster.shutdown();
+    }
+  }
+  
+  private void restartStandby() throws IOException {
+    cluster.shutdownNameNode(1);
+    // Set the safemode extension to be lengthy, so that the tests
+    // can check the safemode message after the safemode conditions
+    // have been achieved, without being racy.
+    cluster.getConfiguration(1).setInt(
+        DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 30000);
+    cluster.getConfiguration(1).setInt(
+        DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+
+    cluster.restartNameNode(1);
+    nn1 = cluster.getNameNode(1);
+    assertEquals(nn1.getNamesystem().getTransactionsSinceLastLogRoll(), 0L);
+  }
+  
+  /**
+   * Test case for enter safemode in active namenode, when it is already in startup safemode.
+   * It is a regression test for HDFS-2747.
+   */
+  @Test
+  public void testEnterSafeModeInANNShouldNotThrowNPE() throws Exception {
+    banner("Restarting active");
+    DFSTestUtil
+      .createFile(fs, new Path("/test"), 3 * BLOCK_SIZE, (short) 3, 1L);
+    restartActive();
+    nn0.getRpcServer().transitionToActive();
+
+    FSNamesystem namesystem = nn0.getNamesystem();
+    String status = namesystem.getSafemode();
+    assertTrue("Bad safemode status: '" + status + "'", status
+        .startsWith("Safe mode is ON."));
+    NameNodeAdapter.enterSafeMode(nn0, false);
+    assertTrue("Failed to enter into safemode in active", namesystem
+        .isInSafeMode());
+    NameNodeAdapter.enterSafeMode(nn0, false);
+    assertTrue("Failed to enter into safemode in active", namesystem
+        .isInSafeMode());
+  }
+
+  /**
+   * Test case for enter safemode in standby namenode, when it is already in startup safemode.
+   * It is a regression test for HDFS-2747.
+   */
+  @Test
+  public void testEnterSafeModeInSBNShouldNotThrowNPE() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil
+        .createFile(fs, new Path("/test"), 3 * BLOCK_SIZE, (short) 3, 1L);
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup and enter safemode.
+    nn0.getRpcServer().rollEditLog();
+    banner("Creating some blocks that won't be in the edit log");
+    DFSTestUtil.createFile(fs, new Path("/test2"), 5 * BLOCK_SIZE, (short) 3,
+        1L);
+    banner("Deleting the original blocks");
+    fs.delete(new Path("/test"), true);
+    banner("Restarting standby");
+    restartStandby();
+    FSNamesystem namesystem = nn1.getNamesystem();
+    String status = namesystem.getSafemode();
+    assertTrue("Bad safemode status: '" + status + "'", status
+        .startsWith("Safe mode is ON."));
+    NameNodeAdapter.enterSafeMode(nn1, false);
+    assertTrue("Failed to enter into safemode in standby", namesystem
+        .isInSafeMode());
+    NameNodeAdapter.enterSafeMode(nn1, false);
+    assertTrue("Failed to enter into safemode in standby", namesystem
+        .isInSafeMode());
+  }
+
+  private void restartActive() throws IOException {
+    cluster.shutdownNameNode(0);
+    // Set the safemode extension to be lengthy, so that the tests
+    // can check the safemode message after the safemode conditions
+    // have been achieved, without being racy.
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 30000);
+    cluster.restartNameNode(0);
+    nn0 = cluster.getNameNode(0);
+  }
+  
+  /**
+   * Tests the case where, while a standby is down, more blocks are
+   * added to the namespace, but not rolled. So, when it starts up,
+   * it receives notification about the new blocks during
+   * the safemode extension period.
+   */
+  @Test
+  public void testBlocksAddedBeforeStandbyRestart() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil.createFile(fs, new Path("/test"), 3*BLOCK_SIZE, (short) 3, 1L);
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup.
+    nn0.getRpcServer().rollEditLog();
+
+    banner("Creating some blocks that won't be in the edit log");
+    DFSTestUtil.createFile(fs, new Path("/test2"), 5*BLOCK_SIZE, (short) 3, 1L);
+    
+    banner("Restarting standby");
+    restartStandby();
+
+    // We expect it not to be stuck in safemode, since those blocks
+    // that are already visible to the SBN should be processed
+    // in the initial block reports.
+    assertSafeMode(nn1, 3, 3);
+
+    banner("Waiting for standby to catch up to active namespace");
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    assertSafeMode(nn1, 8, 8);
+  }
+  
+  /**
+   * Similar to {@link #testBlocksAddedBeforeStandbyRestart()} except that
+   * the new blocks are allocated after the SBN has restarted. So, the
+   * blocks were not present in the original block reports at startup
+   * but are reported separately by blockReceived calls.
+   */
+  @Test
+  public void testBlocksAddedWhileInSafeMode() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil.createFile(fs, new Path("/test"), 3*BLOCK_SIZE, (short) 3, 1L);
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup.
+    nn0.getRpcServer().rollEditLog();
+    
+    banner("Restarting standby");
+    restartStandby();
+    
+    assertSafeMode(nn1, 3, 3);
+    
+    // Create a few blocks which will send blockReceived calls to the
+    // SBN.
+    banner("Creating some blocks while SBN is in safe mode");
+    DFSTestUtil.createFile(fs, new Path("/test2"), 5*BLOCK_SIZE, (short) 3, 1L);
+
+    
+    banner("Waiting for standby to catch up to active namespace");
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+
+    assertSafeMode(nn1, 8, 8);
+  }
+
+  /**
+   * Test for the following case proposed by ATM:
+   * 1. Both NNs are up, one is active. There are 100 blocks. Both are
+   *    out of safemode.
+   * 2. 10 block deletions get processed by NN1. NN2 enqueues these DN messages
+   *     until it next reads from a checkpointed edits file.
+   * 3. NN2 gets restarted. Its queues are lost.
+   * 4. NN2 comes up, reads from all the finalized edits files. Concludes there
+   *    should still be 100 blocks.
+   * 5. NN2 receives a block report from all the DNs, which only accounts for
+   *    90 blocks. It doesn't leave safemode.
+   * 6. NN1 dies or is transitioned to standby.
+   * 7. NN2 is transitioned to active. It reads all the edits from NN1. It now
+   *    knows there should only be 90 blocks, but it's still in safemode.
+   * 8. NN2 doesn't ever recheck whether it should leave safemode.
+   * 
+   * This is essentially the inverse of {@link #testBlocksAddedBeforeStandbyRestart()}
+   */
+  @Test
+  public void testBlocksRemovedBeforeStandbyRestart() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil.createFile(fs, new Path("/test"), 5*BLOCK_SIZE, (short) 3, 1L);
+
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup.
+    nn0.getRpcServer().rollEditLog();
+
+    // Delete those blocks again, so they won't get reported to the SBN
+    // once it starts up
+    banner("Removing the blocks without rolling the edit log");
+    fs.delete(new Path("/test"), true);
+    BlockManagerTestUtil.computeAllPendingWork(
+        nn0.getNamesystem().getBlockManager());
+    cluster.triggerHeartbeats();
+
+    banner("Restarting standby");
+    restartStandby();
+    assertSafeMode(nn1, 0, 5);
+    
+    banner("Waiting for standby to catch up to active namespace");
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    assertSafeMode(nn1, 0, 0);
+  }
+  
+  /**
+   * Similar to {@link #testBlocksRemovedBeforeStandbyRestart()} except that
+   * the blocks are removed after the SBN has restarted. So, the
+   * blocks were present in the original block reports at startup
+   * but are deleted separately later by deletion reports.
+   */
+  @Test
+  public void testBlocksRemovedWhileInSafeMode() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil.createFile(fs, new Path("/test"), 10*BLOCK_SIZE, (short) 3, 1L);
+
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup.
+    nn0.getRpcServer().rollEditLog();
+ 
+    banner("Restarting standby");
+    restartStandby();
+    
+    // It will initially have all of the blocks necessary.
+    assertSafeMode(nn1, 10, 10);
+
+    // Delete those blocks while the SBN is in safe mode.
+    // This doesn't affect the SBN, since deletions are not
+    // ACKed when due to block removals.
+    banner("Removing the blocks without rolling the edit log");
+    fs.delete(new Path("/test"), true);
+    BlockManagerTestUtil.computeAllPendingWork(
+        nn0.getNamesystem().getBlockManager());
+    
+    banner("Triggering deletions on DNs and Deletion Reports");
+    cluster.triggerHeartbeats();
+    HATestUtil.waitForDNDeletions(cluster);
+    cluster.triggerDeletionReports();
+
+    assertSafeMode(nn1, 10, 10);
+
+    // When we catch up to active namespace, it will restore back
+    // to 0 blocks.
+    banner("Waiting for standby to catch up to active namespace");
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+
+    assertSafeMode(nn1, 0, 0);
+  }
+  
+  /**
+   * Tests that the standby node properly tracks the number of total
+   * and safe blocks while it is in safe mode. Since safe-mode only
+   * counts completed blocks, append needs to decrement the total
+   * number of blocks and then re-increment when the file is closed
+   * again.
+   */
+  @Test
+  public void testAppendWhileInSafeMode() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    // Make 4.5 blocks so that append() will re-open an existing block
+    // instead of just adding a new one
+    DFSTestUtil.createFile(fs, new Path("/test"),
+        4*BLOCK_SIZE + BLOCK_SIZE/2, (short) 3, 1L);
+
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup.
+    nn0.getRpcServer().rollEditLog();
+ 
+    banner("Restarting standby");
+    restartStandby();
+    
+    // It will initially have all of the blocks necessary.
+    assertSafeMode(nn1, 5, 5);
+
+    // Append to a block while SBN is in safe mode. This should
+    // not affect safemode initially, since the DN message
+    // will get queued.
+    FSDataOutputStream stm = fs.append(new Path("/test"));
+    try {
+      assertSafeMode(nn1, 5, 5);
+      
+      // if we roll edits now, the SBN should see that it's under construction
+      // and change its total count and safe count down by one, since UC
+      // blocks are not counted by safe mode.
+      HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+      assertSafeMode(nn1, 4, 4);
+    } finally {
+      IOUtils.closeStream(stm);
+    }
+    
+    // Delete those blocks while the SBN is in safe mode.
+    // This will not ACK the deletions to the SBN, so it won't
+    // notice until we roll the edit log.
+    banner("Removing the blocks without rolling the edit log");
+    fs.delete(new Path("/test"), true);
+    BlockManagerTestUtil.computeAllPendingWork(
+        nn0.getNamesystem().getBlockManager());
+    
+    banner("Triggering deletions on DNs and Deletion Reports");
+    cluster.triggerHeartbeats();
+    HATestUtil.waitForDNDeletions(cluster);
+    cluster.triggerDeletionReports();
+
+    assertSafeMode(nn1, 4, 4);
+
+    // When we roll the edit log, the deletions will go through.
+    banner("Waiting for standby to catch up to active namespace");
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+
+    assertSafeMode(nn1, 0, 0);
+  }
+  
+  /**
+   * Regression test for a bug experienced while developing
+   * HDFS-2742. The scenario here is:
+   * - image contains some blocks
+   * - edits log contains at least one block addition, followed
+   *   by deletion of more blocks than were added.
+   * - When node starts up, some incorrect accounting of block
+   *   totals caused an assertion failure.
+   */
+  @Test
+  public void testBlocksDeletedInEditLog() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    // Make 4 blocks persisted in the image.
+    DFSTestUtil.createFile(fs, new Path("/test"),
+        4*BLOCK_SIZE, (short) 3, 1L);
+    NameNodeAdapter.enterSafeMode(nn0, false);
+    NameNodeAdapter.saveNamespace(nn0);
+    NameNodeAdapter.leaveSafeMode(nn0, false);
+    
+    // OP_ADD for 2 blocks
+    DFSTestUtil.createFile(fs, new Path("/test2"),
+        2*BLOCK_SIZE, (short) 3, 1L);
+    
+    // OP_DELETE for 4 blocks
+    fs.delete(new Path("/test"), true);
+
+    restartActive();
+  }
+  
+  private void assertSafeMode(NameNode nn, int safe, int total) {
+    String status = nn1.getNamesystem().getSafemode();
+    if (safe == total) {
+      assertTrue("Bad safemode status: '" + status + "'",
+          status.startsWith(
+            "Safe mode is ON." +
+            "The reported blocks " + safe + " has reached the threshold " +
+            "0.9990 of total blocks " + total + ". Safe mode will be " +
+            "turned off automatically"));
+    } else {
+      int additional = total - safe;
+      assertTrue("Bad safemode status: '" + status + "'",
+          status.startsWith(
+              "Safe mode is ON." +
+              "The reported blocks " + safe + " needs additional " +
+              additional + " blocks"));
+    }
+  }
+
+  /**
+   * Set up a namesystem with several edits, both deletions and
+   * additions, and failover to a new NN while that NN is in
+   * safemode. Ensure that it will exit safemode.
+   */
+  @Test
+  public void testComplexFailoverIntoSafemode() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil.createFile(fs, new Path("/test"), 3*BLOCK_SIZE, (short) 3, 1L);
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup and enter safemode.
+    nn0.getRpcServer().rollEditLog();
+
+    banner("Creating some blocks that won't be in the edit log");
+    DFSTestUtil.createFile(fs, new Path("/test2"), 5*BLOCK_SIZE, (short) 3, 1L);
+    
+    banner("Deleting the original blocks");
+    fs.delete(new Path("/test"), true);
+    
+    banner("Restarting standby");
+    restartStandby();
+
+    // We expect it to be on its way out of safemode, since all of the blocks
+    // from the edit log have been reported.
+    assertSafeMode(nn1, 3, 3);
+    
+    // Initiate a failover into it while it's in safemode
+    banner("Initiating a failover into NN1 in safemode");
+    NameNodeAdapter.abortEditLogs(nn0);
+    cluster.transitionToActive(1);
+
+    assertSafeMode(nn1, 5, 5);
+  }
+  
+  /**
+   * Similar to {@link #testBlocksRemovedWhileInSafeMode()} except that
+   * the OP_DELETE edits arrive at the SBN before the block deletion reports.
+   * The tracking of safe blocks needs to properly account for the removal
+   * of the blocks as well as the safe count. This is a regression test for
+   * HDFS-2742.
+   */
+  @Test
+  public void testBlocksRemovedWhileInSafeModeEditsArriveFirst() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some blocks");
+    DFSTestUtil.createFile(fs, new Path("/test"), 10*BLOCK_SIZE, (short) 3, 1L);
+
+    // Roll edit log so that, when the SBN restarts, it will load
+    // the namespace during startup.
+    nn0.getRpcServer().rollEditLog();
+ 
+    banner("Restarting standby");
+    restartStandby();
+    
+    // It will initially have all of the blocks necessary.
+    String status = nn1.getNamesystem().getSafemode();
+    assertTrue("Bad safemode status: '" + status + "'",
+        status.startsWith(
+            "Safe mode is ON." +
+            "The reported blocks 10 has reached the threshold 0.9990 of " +
+            "total blocks 10. Safe mode will be turned off automatically"));
+
+    // Delete those blocks while the SBN is in safe mode.
+    // Immediately roll the edit log before the actual deletions are sent
+    // to the DNs.
+    banner("Removing the blocks without rolling the edit log");
+    fs.delete(new Path("/test"), true);
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+
+    // Should see removal of the blocks as well as their contribution to safe block count.
+    assertSafeMode(nn1, 0, 0);
+
+    
+    banner("Triggering sending deletions to DNs and Deletion Reports");
+    BlockManagerTestUtil.computeAllPendingWork(
+        nn0.getNamesystem().getBlockManager());    
+    cluster.triggerHeartbeats();
+    HATestUtil.waitForDNDeletions(cluster);
+    cluster.triggerDeletionReports();
+
+    // No change in assertion status here, but some of the consistency checks
+    // in safemode will fire here if we accidentally decrement safe block count
+    // below 0.    
+    assertSafeMode(nn1, 0, 0);
+  }
+  
+
+  /**
+   * Test that the number of safe blocks is accounted correctly even when
+   * blocks move between under-construction state and completed state.
+   * If a FINALIZED report arrives at the SBN before the block is marked
+   * COMPLETE, then when we get the OP_CLOSE we need to count it as "safe"
+   * at that point. This is a regression test for HDFS-2742.
+   */
+  @Test
+  public void testSafeBlockTracking() throws Exception {
+    banner("Starting with NN0 active and NN1 standby, creating some " +
+    		"UC blocks plus some other blocks to force safemode");
+    DFSTestUtil.createFile(fs, new Path("/other-blocks"), 10*BLOCK_SIZE, (short) 3, 1L);
+
+    List<FSDataOutputStream> stms = Lists.newArrayList();
+    try {
+      for (int i = 0; i < 5; i++) {
+        FSDataOutputStream stm = fs.create(new Path("/test-uc-" + i));
+        stms.add(stm);
+        stm.write(1);
+        stm.hflush();
+      }
+      // Roll edit log so that, when the SBN restarts, it will load
+      // the namespace during startup and enter safemode.
+      nn0.getRpcServer().rollEditLog();
+    } finally {
+      for (FSDataOutputStream stm : stms) {
+        IOUtils.closeStream(stm);
+      }
+    }
+    
+    banner("Restarting SBN");
+    restartStandby();
+    assertSafeMode(nn1, 10, 10);
+
+    banner("Allowing SBN to catch up");
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    assertSafeMode(nn1, 15, 15);
+  }
+  
+  /**
+   * Regression test for HDFS-2753. In this bug, the following sequence was
+   * observed:
+   * - Some blocks are written to DNs while the SBN was down. This causes
+   *   the blockReceived messages to get queued in the BPServiceActor on the
+   *   DN.
+   * - When the SBN returns, the DN re-registers with the SBN, and then
+   *   flushes its blockReceived queue to the SBN before it sends its
+   *   first block report. This caused the first block report to be
+   *   incorrect ignored.
+   * - The SBN would become stuck in safemode.
+   */
+  @Test
+  public void testBlocksAddedWhileStandbyIsDown() throws Exception {
+    DFSTestUtil.createFile(fs, new Path("/test"), 3*BLOCK_SIZE, (short) 3, 1L);
+
+    banner("Stopping standby");
+    cluster.shutdownNameNode(1);
+    
+    DFSTestUtil.createFile(fs, new Path("/test2"), 3*BLOCK_SIZE, (short) 3, 1L);
+
+    banner("Rolling edit log so standby gets all edits on restart");
+    nn0.getRpcServer().rollEditLog();
+    
+    restartStandby();
+    assertSafeMode(nn1, 6, 6);
+  }
+  
+  /**
+   * Regression test for HDFS-2804: standby should not populate replication
+   * queues when exiting safe mode.
+   */
+  @Test
+  public void testNoPopulatingReplQueuesWhenExitingSafemode() throws Exception {
+    DFSTestUtil.createFile(fs, new Path("/test"), 15*BLOCK_SIZE, (short)3, 1L);
+    
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    
+    // get some blocks in the SBN's image
+    nn1.getRpcServer().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    NameNodeAdapter.saveNamespace(nn1);
+    nn1.getRpcServer().setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+    // and some blocks in the edit logs
+    DFSTestUtil.createFile(fs, new Path("/test2"), 15*BLOCK_SIZE, (short)3, 1L);
+    nn0.getRpcServer().rollEditLog();
+    
+    cluster.stopDataNode(1);
+    cluster.shutdownNameNode(1);
+
+    //Configuration sbConf = cluster.getConfiguration(1);
+    //sbConf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 1);
+    cluster.restartNameNode(1, false);
+    nn1 = cluster.getNameNode(1);
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return !nn1.isInSafeMode();
+      }
+    }, 100, 10000);
+    
+    BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
+    assertEquals(0L, nn1.getNamesystem().getUnderReplicatedBlocks());
+    assertEquals(0L, nn1.getNamesystem().getPendingReplicationBlocks());
+  }
+  
+  /**
+   * Print a big banner in the test log to make debug easier.
+   */
+  static void banner(String string) {
+    LOG.info("\n\n\n\n================================================\n" +
+        string + "\n" +
+        "==================================================\n\n");
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAStateTransitions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAStateTransitions.java
new file mode 100644
index 0000000..092bb5a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAStateTransitions.java
@@ -0,0 +1,545 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * Tests state transition from active->standby, and manual failover
+ * and failback between two namenodes.
+ */
+public class TestHAStateTransitions {
+  protected static final Log LOG = LogFactory.getLog(
+      TestStandbyIsHot.class);
+  private static final Path TEST_DIR = new Path("/test");
+  private static final Path TEST_FILE_PATH = new Path(TEST_DIR, "foo");
+  private static final String TEST_FILE_STR = TEST_FILE_PATH.toUri().getPath();
+  private static final String TEST_FILE_DATA =
+    "Hello state transitioning world";
+  
+  static {
+    ((Log4JLogger)EditLogTailer.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  /**
+   * Test which takes a single node and flip flops between
+   * active and standby mode, making sure it doesn't
+   * double-play any edits.
+   */
+  @Test
+  public void testTransitionActiveToStandby() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      FileSystem fs = cluster.getFileSystem(0);
+      
+      fs.mkdirs(TEST_DIR);
+      cluster.transitionToStandby(0);
+      try {
+        fs.mkdirs(new Path("/x"));
+        fail("Didn't throw trying to mutate FS in standby state");
+      } catch (Throwable t) {
+        GenericTestUtils.assertExceptionContains(
+            "Operation category WRITE is not supported", t);
+      }
+      cluster.transitionToActive(0);
+      
+      // Create a file, then delete the whole directory recursively.
+      DFSTestUtil.createFile(fs, new Path(TEST_DIR, "foo"),
+          10, (short)1, 1L);
+      fs.delete(TEST_DIR, true);
+      
+      // Now if the standby tries to replay the last segment that it just
+      // wrote as active, it would fail since it's trying to create a file
+      // in a non-existent directory.
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(0);
+      
+      assertFalse(fs.exists(TEST_DIR));
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that transitioning a service to the state that it is already
+   * in is a nop, specifically, an exception is not thrown.
+   */
+  @Test
+  public void testTransitionToCurrentStateIsANop() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      cluster.transitionToActive(0);
+      cluster.transitionToStandby(0);
+      cluster.transitionToStandby(0);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test manual failover failback for one namespace
+   * @param cluster single process test cluster
+   * @param conf cluster configuration
+   * @param nsIndex namespace index starting from zero
+   * @throws Exception
+   */
+  private void testManualFailoverFailback(MiniDFSCluster cluster, 
+		  Configuration conf, int nsIndex) throws Exception {
+      int nn0 = 2 * nsIndex, nn1 = 2 * nsIndex + 1;
+
+      cluster.transitionToActive(nn0);
+      
+      LOG.info("Starting with NN 0 active in namespace " + nsIndex);
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      fs.mkdirs(TEST_DIR);
+
+      LOG.info("Failing over to NN 1 in namespace " + nsIndex);
+      cluster.transitionToStandby(nn0);
+      cluster.transitionToActive(nn1);
+      assertTrue(fs.exists(TEST_DIR));
+      DFSTestUtil.writeFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+
+      LOG.info("Failing over to NN 0 in namespace " + nsIndex);
+      cluster.transitionToStandby(nn1);
+      cluster.transitionToActive(nn0);
+      assertTrue(fs.exists(TEST_DIR));
+      assertEquals(TEST_FILE_DATA, 
+          DFSTestUtil.readFile(fs, TEST_FILE_PATH));
+
+      LOG.info("Removing test file");
+      fs.delete(TEST_DIR, true);
+      assertFalse(fs.exists(TEST_DIR));
+
+      LOG.info("Failing over to NN 1 in namespace " + nsIndex);
+      cluster.transitionToStandby(nn0);
+      cluster.transitionToActive(nn1);
+      assertFalse(fs.exists(TEST_DIR));
+  }
+  
+  /**
+   * Tests manual failover back and forth between two NameNodes.
+   */
+  @Test
+  public void testManualFailoverAndFailback() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    try {
+      cluster.waitActive();
+      // test the only namespace
+      testManualFailoverFailback(cluster, conf, 0);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Regression test for HDFS-2693: when doing state transitions, we need to
+   * lock the FSNamesystem so that we don't end up doing any writes while it's
+   * "in between" states.
+   * This test case starts up several client threads which do mutation operations
+   * while flipping a NN back and forth from active to standby.
+   */
+  @Test(timeout=120000)
+  public void testTransitionSynchronization() throws Exception {
+    Configuration conf = new Configuration();
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+    try {
+      cluster.waitActive();
+      ReentrantReadWriteLock spyLock = NameNodeAdapter.spyOnFsLock(
+          cluster.getNameNode(0).getNamesystem());
+      Mockito.doAnswer(new GenericTestUtils.SleepAnswer(50))
+        .when(spyLock).writeLock();
+      
+      final FileSystem fs = HATestUtil.configureFailoverFs(
+          cluster, conf);
+      
+      TestContext ctx = new TestContext();
+      for (int i = 0; i < 50; i++) {
+        final int finalI = i;
+        ctx.addThread(new RepeatingTestThread(ctx) {
+          @Override
+          public void doAnAction() throws Exception {
+            Path p = new Path("/test-" + finalI);
+            fs.mkdirs(p);
+            fs.delete(p, true);
+          }
+        });
+      }
+      
+      ctx.addThread(new RepeatingTestThread(ctx) {
+        @Override
+        public void doAnAction() throws Exception {
+          cluster.transitionToStandby(0);
+          Thread.sleep(50);
+          cluster.transitionToActive(0);
+        }
+      });
+      ctx.startThreads();
+      ctx.waitFor(20000);
+      ctx.stop();
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Test for HDFS-2812. Since lease renewals go from the client
+   * only to the active NN, the SBN will have out-of-date lease
+   * info when it becomes active. We need to make sure we don't
+   * accidentally mark the leases as expired when the failover
+   * proceeds.
+   */
+  @Test(timeout=120000)
+  public void testLeasesRenewedOnTransition() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    FSDataOutputStream stm = null;
+    FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+    NameNode nn0 = cluster.getNameNode(0);
+    NameNode nn1 = cluster.getNameNode(1);
+
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      
+      LOG.info("Starting with NN 0 active");
+
+      stm = fs.create(TEST_FILE_PATH);
+      long nn0t0 = NameNodeAdapter.getLeaseRenewalTime(nn0, TEST_FILE_STR);
+      assertTrue(nn0t0 > 0);
+      long nn1t0 = NameNodeAdapter.getLeaseRenewalTime(nn1, TEST_FILE_STR);
+      assertEquals("Lease should not yet exist on nn1",
+          -1, nn1t0);
+      
+      Thread.sleep(5); // make sure time advances!
+
+      HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+      long nn1t1 = NameNodeAdapter.getLeaseRenewalTime(nn1, TEST_FILE_STR);
+      assertTrue("Lease should have been created on standby. Time was: " +
+          nn1t1, nn1t1 > nn0t0);
+          
+      Thread.sleep(5); // make sure time advances!
+      
+      LOG.info("Failing over to NN 1");
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      long nn1t2 = NameNodeAdapter.getLeaseRenewalTime(nn1, TEST_FILE_STR);
+      assertTrue("Lease should have been renewed by failover process",
+          nn1t2 > nn1t1);
+    } finally {
+      IOUtils.closeStream(stm);
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Test that delegation tokens continue to work after the failover.
+   */
+  @Test
+  public void testDelegationTokensAfterFailover() throws IOException,
+      URISyntaxException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(0)
+        .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      NameNode nn1 = cluster.getNameNode(0);
+      NameNode nn2 = cluster.getNameNode(1);
+
+      String renewer = UserGroupInformation.getLoginUser().getUserName();
+      Token<DelegationTokenIdentifier> token = nn1.getRpcServer()
+          .getDelegationToken(new Text(renewer));
+
+      LOG.info("Failing over to NN 1");
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+
+      nn2.getRpcServer().renewDelegationToken(token);
+      nn2.getRpcServer().cancelDelegationToken(token);
+      token = nn2.getRpcServer().getDelegationToken(new Text(renewer));
+      Assert.assertTrue(token != null);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Tests manual failover back and forth between two NameNodes
+   * for federation cluster with two namespaces.
+   */
+  @Test
+  public void testManualFailoverFailbackFederationHA() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
+      .numDataNodes(1)
+      .build();
+    try {
+      cluster.waitActive();
+   
+      // test for namespace 0
+      testManualFailoverFailback(cluster, conf, 0);
+      
+      // test for namespace 1
+      testManualFailoverFailback(cluster, conf, 1); 
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testFailoverWithEmptyInProgressEditLog() throws Exception {
+    testFailoverAfterCrashDuringLogRoll(false);
+  }
+  
+  @Test
+  public void testFailoverWithEmptyInProgressEditLogWithHeader()
+      throws Exception {
+    testFailoverAfterCrashDuringLogRoll(true);
+  }
+  
+  private static void testFailoverAfterCrashDuringLogRoll(boolean writeHeader)
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, Integer.MAX_VALUE);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(0)
+      .build();
+    FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+    try {
+      cluster.transitionToActive(0);
+      NameNode nn0 = cluster.getNameNode(0);
+      nn0.getRpcServer().rollEditLog();
+      cluster.shutdownNameNode(0);
+      createEmptyInProgressEditLog(cluster, nn0, writeHeader);
+      cluster.transitionToActive(1);
+    } finally {
+      IOUtils.cleanup(LOG, fs);
+      cluster.shutdown();
+    }
+  }
+  
+  private static void createEmptyInProgressEditLog(MiniDFSCluster cluster,
+      NameNode nn, boolean writeHeader) throws IOException {
+    long txid = nn.getNamesystem().getEditLog().getLastWrittenTxId();
+    URI sharedEditsUri = cluster.getSharedEditsDir(0, 1);
+    File sharedEditsDir = new File(sharedEditsUri.getPath());
+    StorageDirectory storageDir = new StorageDirectory(sharedEditsDir);
+    File inProgressFile = NameNodeAdapter.getInProgressEditsFile(storageDir,
+        txid + 1);
+    assertTrue("Failed to create in-progress edits file",
+        inProgressFile.createNewFile());
+    
+    if (writeHeader) {
+      DataOutputStream out = new DataOutputStream(new FileOutputStream(
+          inProgressFile));
+      EditLogFileOutputStream.writeHeader(out);
+      out.close();
+    }
+  }
+  
+
+  /**
+   * The secret manager needs to start/stop - the invariant should be that
+   * the secret manager runs if and only if the NN is active and not in
+   * safe mode. As a state diagram, we need to test all of the following
+   * transitions to make sure the secret manager is started when we transition
+   * into state 4, but none of the others.
+   * <pre>
+   *         SafeMode     Not SafeMode 
+   * Standby   1 <------> 2
+   *           ^          ^
+   *           |          |
+   *           v          v
+   * Active    3 <------> 4
+   * </pre>
+   */
+  @Test(timeout=60000)
+  public void testSecretManagerState() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.setInt(
+        DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY, 50);
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(1)
+         .waitSafeMode(false)
+        .build();
+    try {
+      cluster.transitionToActive(0);
+      DFSTestUtil.createFile(cluster.getFileSystem(0),
+          TEST_FILE_PATH, 6000, (short)1, 1L);
+      
+      cluster.getConfiguration(0).setInt(
+          DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, 60000);
+
+      cluster.restartNameNode(0);
+      NameNode nn = cluster.getNameNode(0);
+      
+      banner("Started in state 1.");
+      assertTrue(nn.isStandbyState());
+      assertTrue(nn.isInSafeMode());
+      assertFalse(isDTRunning(nn));
+      
+      banner("Transition 1->2. Should not start secret manager");
+      NameNodeAdapter.leaveSafeMode(nn, false);
+      assertTrue(nn.isStandbyState());
+      assertFalse(nn.isInSafeMode());
+      assertFalse(isDTRunning(nn));
+  
+      banner("Transition 2->1. Should not start secret manager.");
+      NameNodeAdapter.enterSafeMode(nn, false);
+      assertTrue(nn.isStandbyState());
+      assertTrue(nn.isInSafeMode());
+      assertFalse(isDTRunning(nn));
+  
+      banner("Transition 1->3. Should not start secret manager.");
+      nn.getRpcServer().transitionToActive();
+      assertFalse(nn.isStandbyState());
+      assertTrue(nn.isInSafeMode());
+      assertFalse(isDTRunning(nn));
+  
+      banner("Transition 3->1. Should not start secret manager.");
+      nn.getRpcServer().transitionToStandby();
+      assertTrue(nn.isStandbyState());
+      assertTrue(nn.isInSafeMode());
+      assertFalse(isDTRunning(nn));
+  
+      banner("Transition 1->3->4. Should start secret manager.");
+      nn.getRpcServer().transitionToActive();
+      NameNodeAdapter.leaveSafeMode(nn, false);
+      assertFalse(nn.isStandbyState());
+      assertFalse(nn.isInSafeMode());
+      assertTrue(isDTRunning(nn));
+      
+      banner("Transition 4->3. Should stop secret manager");
+      NameNodeAdapter.enterSafeMode(nn, false);
+      assertFalse(nn.isStandbyState());
+      assertTrue(nn.isInSafeMode());
+      assertFalse(isDTRunning(nn));
+  
+      banner("Transition 3->4. Should start secret manager");
+      NameNodeAdapter.leaveSafeMode(nn, false);
+      assertFalse(nn.isStandbyState());
+      assertFalse(nn.isInSafeMode());
+      assertTrue(isDTRunning(nn));
+      
+      for (int i = 0; i < 20; i++) {
+        // Loop the last check to suss out races.
+        banner("Transition 4->2. Should stop secret manager.");
+        nn.getRpcServer().transitionToStandby();
+        assertTrue(nn.isStandbyState());
+        assertFalse(nn.isInSafeMode());
+        assertFalse(isDTRunning(nn));
+    
+        banner("Transition 2->4. Should start secret manager");
+        nn.getRpcServer().transitionToActive();
+        assertFalse(nn.isStandbyState());
+        assertFalse(nn.isInSafeMode());
+        assertTrue(isDTRunning(nn));
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  private boolean isDTRunning(NameNode nn) {
+    return NameNodeAdapter.getDtSecretManager(nn.getNamesystem()).isRunning();
+  }
+
+  /**
+   * Print a big banner in the test log to make debug easier.
+   */
+  static void banner(String string) {
+    LOG.info("\n\n\n\n================================================\n" +
+        string + "\n" +
+        "==================================================\n\n");
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAWebUI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAWebUI.java
new file mode 100644
index 0000000..be01430
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAWebUI.java
@@ -0,0 +1,73 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.net.URL;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.junit.Test;
+
+public class TestHAWebUI {
+
+  /**
+   * Tests that the web UI of the name node provides a link to browse the file
+   * system and summary of under-replicated blocks only in active state
+   * 
+   */
+  @Test
+  public void testLinkAndClusterSummary() throws Exception {
+    Configuration conf = new Configuration();
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
+        .build();
+    try {
+      cluster.waitActive();
+
+      cluster.transitionToActive(0);
+      String pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
+          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
+          + "/dfshealth.jsp"));
+      assertTrue(pageContents.contains("Browse the filesystem"));
+      assertTrue(pageContents.contains("Number of Under-Replicated Blocks"));
+
+      cluster.transitionToStandby(0);
+      pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
+          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
+          + "/dfshealth.jsp"));
+      assertFalse(pageContents.contains("Browse the filesystem"));
+      assertFalse(pageContents.contains("Number of Under-Replicated Blocks"));
+
+      cluster.transitionToActive(0);
+      pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
+          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
+          + "/dfshealth.jsp"));
+      assertTrue(pageContents.contains("Browse the filesystem"));
+      assertTrue(pageContents.contains("Number of Under-Replicated Blocks"));
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestNNHealthCheck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestNNHealthCheck.java
new file mode 100644
index 0000000..ab2a8dd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestNNHealthCheck.java
@@ -0,0 +1,73 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HealthCheckFailedException;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestNNHealthCheck {
+
+  @Test
+  public void testNNHealthCheck() throws IOException {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
+          .nnTopology(MiniDFSNNTopology.simpleHATopology())
+          .build();
+
+      NameNodeResourceChecker mockResourceChecker = Mockito.mock(
+          NameNodeResourceChecker.class);
+      Mockito.doReturn(true).when(mockResourceChecker).hasAvailableDiskSpace();
+      cluster.getNameNode(0).getNamesystem()
+          .setNNResourceChecker(mockResourceChecker);
+      
+      NamenodeProtocols rpc = cluster.getNameNodeRpc(0);
+      
+      // Should not throw error, which indicates healthy.
+      rpc.monitorHealth();
+      
+      Mockito.doReturn(false).when(mockResourceChecker).hasAvailableDiskSpace();
+      
+      try {
+        // Should throw error - NN is unhealthy.
+        rpc.monitorHealth();
+        fail("Should not have succeeded in calling monitorHealth");
+      } catch (HealthCheckFailedException hcfe) {
+        GenericTestUtils.assertExceptionContains(
+            "The NameNode has no resources available", hcfe);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
new file mode 100644
index 0000000..547ba72
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -0,0 +1,506 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.AppendTestUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+
+import org.apache.log4j.Level;
+
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.base.Supplier;
+
+/**
+ * Test cases regarding pipeline recovery during NN failover.
+ */
+public class TestPipelinesFailover {
+  static {
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(BlockManager.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(
+        "org.apache.hadoop.io.retry.RetryInvocationHandler")).getLogger().setLevel(Level.ALL);
+
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+  }
+  
+  protected static final Log LOG = LogFactory.getLog(
+      TestPipelinesFailover.class);
+  private static final Path TEST_PATH =
+    new Path("/test-file");
+  private static final int BLOCK_SIZE = 4096;
+  private static final int BLOCK_AND_A_HALF = BLOCK_SIZE * 3 / 2;
+  
+  private static final int STRESS_NUM_THREADS = 25;
+  private static final int STRESS_RUNTIME = 40000;
+
+  /**
+   * Tests continuing a write pipeline over a failover.
+   */
+  @Test(timeout=30000)
+  public void testWriteOverFailover() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    // Don't check replication periodically.
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
+    
+    FSDataOutputStream stm = null;
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      Thread.sleep(500);
+
+      LOG.info("Starting with NN 0 active");
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      stm = fs.create(TEST_PATH);
+      
+      // write a block and a half
+      AppendTestUtil.write(stm, 0, BLOCK_AND_A_HALF);
+      
+      // Make sure all of the blocks are written out before failover.
+      stm.hflush();
+
+      LOG.info("Failing over to NN 1");
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+
+      assertTrue(fs.exists(TEST_PATH));
+      FSNamesystem ns1 = cluster.getNameNode(1).getNamesystem();
+      BlockManagerTestUtil.updateState(ns1.getBlockManager());
+      assertEquals(0, ns1.getPendingReplicationBlocks());
+      assertEquals(0, ns1.getCorruptReplicaBlocks());
+      assertEquals(0, ns1.getMissingBlocksCount());
+
+      // write another block and a half
+      AppendTestUtil.write(stm, BLOCK_AND_A_HALF, BLOCK_AND_A_HALF);
+
+      stm.close();
+      stm = null;
+      
+      AppendTestUtil.check(fs, TEST_PATH, BLOCK_SIZE * 3);
+    } finally {
+      IOUtils.closeStream(stm);
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Tests continuing a write pipeline over a failover when a DN fails
+   * after the failover - ensures that updating the pipeline succeeds
+   * even when the pipeline was constructed on a different NN.
+   */
+  @Test(timeout=30000)
+  public void testWriteOverFailoverWithDnFail() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    
+    FSDataOutputStream stm = null;
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(5)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      Thread.sleep(500);
+
+      LOG.info("Starting with NN 0 active");
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      stm = fs.create(TEST_PATH);
+      
+      // write a block and a half
+      AppendTestUtil.write(stm, 0, BLOCK_AND_A_HALF);
+      
+      // Make sure all the blocks are written before failover
+      stm.hflush();
+
+      LOG.info("Failing over to NN 1");
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+
+      assertTrue(fs.exists(TEST_PATH));
+      
+      cluster.stopDataNode(0);
+
+      // write another block and a half
+      AppendTestUtil.write(stm, BLOCK_AND_A_HALF, BLOCK_AND_A_HALF);
+      stm.hflush();
+      
+      LOG.info("Failing back to NN 0");
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      
+      cluster.stopDataNode(1);
+      
+      AppendTestUtil.write(stm, BLOCK_AND_A_HALF*2, BLOCK_AND_A_HALF);
+      stm.hflush();
+      
+      
+      stm.close();
+      stm = null;
+      
+      AppendTestUtil.check(fs, TEST_PATH, BLOCK_AND_A_HALF * 3);
+    } finally {
+      IOUtils.closeStream(stm);
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Tests lease recovery if a client crashes. This approximates the
+   * use case of HBase WALs being recovered after a NN failover.
+   */
+  @Test(timeout=30000)
+  public void testLeaseRecoveryAfterFailover() throws Exception {
+    final Configuration conf = new Configuration();
+    // Disable permissions so that another user can recover the lease.
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    
+    FSDataOutputStream stm = null;
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      Thread.sleep(500);
+
+      LOG.info("Starting with NN 0 active");
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      stm = fs.create(TEST_PATH);
+      
+      // write a block and a half
+      AppendTestUtil.write(stm, 0, BLOCK_AND_A_HALF);
+      stm.hflush();
+      
+      LOG.info("Failing over to NN 1");
+      
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      
+      assertTrue(fs.exists(TEST_PATH));
+
+      FileSystem fsOtherUser = createFsAsOtherUser(cluster, conf);
+      loopRecoverLease(fsOtherUser, TEST_PATH);
+      
+      AppendTestUtil.check(fs, TEST_PATH, BLOCK_AND_A_HALF);
+      
+      // Fail back to ensure that the block locations weren't lost on the
+      // original node.
+      cluster.transitionToStandby(1);
+      cluster.transitionToActive(0);
+      AppendTestUtil.check(fs, TEST_PATH, BLOCK_AND_A_HALF);      
+    } finally {
+      IOUtils.closeStream(stm);
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test the scenario where the NN fails over after issuing a block
+   * synchronization request, but before it is committed. The
+   * DN running the recovery should then fail to commit the synchronization
+   * and a later retry will succeed.
+   */
+  @Test(timeout=30000)
+  public void testFailoverRightBeforeCommitSynchronization() throws Exception {
+    final Configuration conf = new Configuration();
+    // Disable permissions so that another user can recover the lease.
+    conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    
+    FSDataOutputStream stm = null;
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      Thread.sleep(500);
+
+      LOG.info("Starting with NN 0 active");
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      stm = fs.create(TEST_PATH);
+      
+      // write a half block
+      AppendTestUtil.write(stm, 0, BLOCK_SIZE / 2);
+      stm.hflush();
+      
+      // Look into the block manager on the active node for the block
+      // under construction.
+      
+      NameNode nn0 = cluster.getNameNode(0);
+      ExtendedBlock blk = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
+      DatanodeDescriptor expectedPrimary = getExpectedPrimaryNode(nn0, blk);
+      LOG.info("Expecting block recovery to be triggered on DN " +
+          expectedPrimary);
+      
+      // Find the corresponding DN daemon, and spy on its connection to the
+      // active.
+      DataNode primaryDN = cluster.getDataNode(expectedPrimary.getIpcPort());
+      DatanodeProtocolClientSideTranslatorPB nnSpy =
+          DataNodeAdapter.spyOnBposToNN(primaryDN, nn0);
+      
+      // Delay the commitBlockSynchronization call
+      DelayAnswer delayer = new DelayAnswer(LOG);
+      Mockito.doAnswer(delayer).when(nnSpy).commitBlockSynchronization(
+          Mockito.eq(blk),
+          Mockito.anyInt(), // new genstamp
+          Mockito.anyLong(), // new length
+          Mockito.eq(true), // close file
+          Mockito.eq(false), // delete block
+          (DatanodeID[]) Mockito.anyObject()); // new targets
+
+      DistributedFileSystem fsOtherUser = createFsAsOtherUser(cluster, conf);
+      assertFalse(fsOtherUser.recoverLease(TEST_PATH));
+      
+      LOG.info("Waiting for commitBlockSynchronization call from primary");
+      delayer.waitForCall();
+
+      LOG.info("Failing over to NN 1");
+      
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      
+      // Let the commitBlockSynchronization call go through, and check that
+      // it failed with the correct exception.
+      delayer.proceed();
+      delayer.waitForResult();
+      Throwable t = delayer.getThrown();
+      if (t == null) {
+        fail("commitBlockSynchronization call did not fail on standby");
+      }
+      GenericTestUtils.assertExceptionContains(
+          "Operation category WRITE is not supported",
+          t);
+      
+      // Now, if we try again to recover the block, it should succeed on the new
+      // active.
+      loopRecoverLease(fsOtherUser, TEST_PATH);
+      
+      AppendTestUtil.check(fs, TEST_PATH, BLOCK_SIZE/2);
+    } finally {
+      IOUtils.closeStream(stm);
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Stress test for pipeline/lease recovery. Starts a number of
+   * threads, each of which creates a file and has another client
+   * break the lease. While these threads run, failover proceeds
+   * back and forth between two namenodes.
+   */
+  @Test(timeout=STRESS_RUNTIME*3)
+  public void testPipelineRecoveryStress() throws Exception {
+    HAStressTestHarness harness = new HAStressTestHarness();
+    // Disable permissions so that another user can recover the lease.
+    harness.conf.setBoolean(
+        DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
+
+    final MiniDFSCluster cluster = harness.startCluster();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      
+      FileSystem fs = harness.getFailoverFs();
+      DistributedFileSystem fsAsOtherUser = createFsAsOtherUser(
+          cluster, harness.conf);
+      
+      TestContext testers = new TestContext();
+      for (int i = 0; i < STRESS_NUM_THREADS; i++) {
+        Path p = new Path("/test-" + i);
+        testers.addThread(new PipelineTestThread(
+            testers, fs, fsAsOtherUser, p));
+      }
+      
+      // Start a separate thread which will make sure that replication
+      // happens quickly by triggering deletion reports and replication
+      // work calculation frequently.
+      harness.addReplicationTriggerThread(500);
+      harness.addFailoverThread(5000);
+      harness.startThreads();
+      testers.startThreads();
+      
+      testers.waitFor(STRESS_RUNTIME);
+      testers.stop();
+      harness.stopThreads();
+    } finally {
+      System.err.println("===========================\n\n\n\n");
+      harness.shutdown();
+    }
+  }
+
+  /**
+   * Test thread which creates a file, has another fake user recover
+   * the lease on the file, and then ensures that the file's contents
+   * are properly readable. If any of these steps fails, propagates
+   * an exception back to the test context, causing the test case
+   * to fail.
+   */
+  private static class PipelineTestThread extends RepeatingTestThread {
+    private final FileSystem fs;
+    private final FileSystem fsOtherUser;
+    private final Path path;
+    
+
+    public PipelineTestThread(TestContext ctx,
+        FileSystem fs, FileSystem fsOtherUser, Path p) {
+      super(ctx);
+      this.fs = fs;
+      this.fsOtherUser = fsOtherUser;
+      this.path = p;
+    }
+
+    @Override
+    public void doAnAction() throws Exception {
+      FSDataOutputStream stm = fs.create(path, true);
+      try {
+        AppendTestUtil.write(stm, 0, 100);
+        stm.hflush();
+        loopRecoverLease(fsOtherUser, path);
+        AppendTestUtil.check(fs, path, 100);
+      } finally {
+        try {
+          stm.close();
+        } catch (IOException e) {
+          // should expect this since we lost the lease
+        }
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return "Pipeline test thread for " + path;
+    }
+  }
+
+
+
+  /**
+   * @return the node which is expected to run the recovery of the
+   * given block, which is known to be under construction inside the
+   * given NameNOde.
+   */
+  private DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
+      ExtendedBlock blk) {
+    BlockManager bm0 = nn.getNamesystem().getBlockManager();
+    BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
+    assertTrue("Block " + blk + " should be under construction, " +
+        "got: " + storedBlock,
+        storedBlock instanceof BlockInfoUnderConstruction);
+    BlockInfoUnderConstruction ucBlock =
+      (BlockInfoUnderConstruction)storedBlock;
+    // We expect that the first indexed replica will be the one
+    // to be in charge of the synchronization / recovery protocol.
+    DatanodeDescriptor expectedPrimary = ucBlock.getExpectedLocations()[0];
+    return expectedPrimary;
+  }
+
+  private DistributedFileSystem createFsAsOtherUser(
+      final MiniDFSCluster cluster, final Configuration conf)
+      throws IOException, InterruptedException {
+    return (DistributedFileSystem) UserGroupInformation.createUserForTesting(
+        "otheruser", new String[] { "othergroup"})
+    .doAs(new PrivilegedExceptionAction<FileSystem>() {
+      @Override
+      public FileSystem run() throws Exception {
+        return HATestUtil.configureFailoverFs(
+            cluster, conf);
+      }
+    });
+  }
+  
+  /**
+   * Try to cover the lease on the given file for up to 30
+   * seconds.
+   * @param fsOtherUser the filesystem to use for the recoverLease call
+   * @param testPath the path on which to run lease recovery
+   * @throws TimeoutException if lease recover does not succeed within 30
+   * seconds
+   * @throws InterruptedException if the thread is interrupted
+   */
+  private static void loopRecoverLease(
+      final FileSystem fsOtherUser, final Path testPath)
+      throws TimeoutException, InterruptedException {
+    try {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          boolean success;
+          try {
+            success = ((DistributedFileSystem)fsOtherUser)
+              .recoverLease(testPath);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          if (!success) {
+            LOG.info("Waiting to recover lease successfully");
+          }
+          return success;
+        }
+      }, 1000, 30000);
+    } catch (TimeoutException e) {
+      throw new TimeoutException("Timed out recovering lease for " +
+          testPath);
+    }
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java
new file mode 100644
index 0000000..5800d3a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java
@@ -0,0 +1,133 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestQuotasWithHA {
+  private static final Path TEST_DIR = new Path("/test");
+  private static final Path TEST_FILE = new Path(TEST_DIR, "file");
+  private static final String TEST_DIR_STR = TEST_DIR.toUri().getPath();
+  
+  private static final long NS_QUOTA = 10000;
+  private static final long DS_QUOTA = 10000;
+  private static final long BLOCK_SIZE = 1024; // 1KB blocks
+  
+  private MiniDFSCluster cluster;
+  private NameNode nn0;
+  private NameNode nn1;
+  private FileSystem fs;
+
+  @Before
+  public void setupCluster() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    HAUtil.setAllowStandbyReads(conf, true);
+    
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .waitSafeMode(false)
+      .build();
+    cluster.waitActive();
+    
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+    
+    cluster.transitionToActive(0);
+  }
+  
+  @After
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that quotas are properly tracked by the standby through
+   * create, append, delete.
+   */
+  @Test(timeout=60000)
+  public void testQuotasTrackedOnStandby() throws Exception {
+    fs.mkdirs(TEST_DIR);
+    DistributedFileSystem dfs = (DistributedFileSystem)fs;
+    dfs.setQuota(TEST_DIR, NS_QUOTA, DS_QUOTA);
+    long expectedSize = 3 * BLOCK_SIZE + BLOCK_SIZE/2;
+    DFSTestUtil.createFile(fs, TEST_FILE, expectedSize, (short)1, 1L);
+
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    ContentSummary cs = nn1.getRpcServer().getContentSummary(TEST_DIR_STR);
+    assertEquals(NS_QUOTA, cs.getQuota());
+    assertEquals(DS_QUOTA, cs.getSpaceQuota());
+    assertEquals(expectedSize, cs.getSpaceConsumed());
+    assertEquals(1, cs.getDirectoryCount());
+    assertEquals(1, cs.getFileCount());
+
+    // Append to the file and make sure quota is updated correctly.
+    FSDataOutputStream stm = fs.append(TEST_FILE);
+    try {
+      byte[] data = new byte[(int) (BLOCK_SIZE * 3 / 2)];
+      stm.write(data);
+      expectedSize += data.length;
+    } finally {
+      IOUtils.closeStream(stm);
+    }
+    
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    cs = nn1.getRpcServer().getContentSummary(TEST_DIR_STR);
+    assertEquals(NS_QUOTA, cs.getQuota());
+    assertEquals(DS_QUOTA, cs.getSpaceQuota());
+    assertEquals(expectedSize, cs.getSpaceConsumed());
+    assertEquals(1, cs.getDirectoryCount());
+    assertEquals(1, cs.getFileCount());
+
+    
+    fs.delete(TEST_FILE, true);
+    expectedSize = 0;
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    cs = nn1.getRpcServer().getContentSummary(TEST_DIR_STR);
+    assertEquals(NS_QUOTA, cs.getQuota());
+    assertEquals(DS_QUOTA, cs.getSpaceQuota());
+    assertEquals(expectedSize, cs.getSpaceConsumed());
+    assertEquals(1, cs.getDirectoryCount());
+    assertEquals(0, cs.getFileCount());
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
new file mode 100644
index 0000000..5440c38c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -0,0 +1,213 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+
+public class TestStandbyCheckpoints {
+  private static final int NUM_DIRS_IN_LOG = 200000;
+  private MiniDFSCluster cluster;
+  private NameNode nn0, nn1;
+  private FileSystem fs;
+
+  @Before
+  public void setupCluster() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+      .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
+        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+    
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(topology)
+      .numDataNodes(0)
+      .build();
+    cluster.waitActive();
+    
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+
+    cluster.transitionToActive(0);
+  }
+  
+  @After
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testSBNCheckpoints() throws Exception {
+    doEdits(0, 10);
+    
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    // Once the standby catches up, it should notice that it needs to
+    // do a checkpoint and save one to its local directories.
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 12));
+    
+    // It should also upload it back to the active.
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 12));
+  }
+
+  /**
+   * Test for the case when both of the NNs in the cluster are
+   * in the standby state, and thus are both creating checkpoints
+   * and uploading them to each other.
+   * In this circumstance, they should receive the error from the
+   * other node indicating that the other node already has a
+   * checkpoint for the given txid, but this should not cause
+   * an abort, etc.
+   */
+  @Test
+  public void testBothNodesInStandbyState() throws Exception {
+    doEdits(0, 10);
+    
+    cluster.transitionToStandby(0);
+
+    // Transitioning to standby closed the edit log on the active,
+    // so the standby will catch up. Then, both will be in standby mode
+    // with enough uncheckpointed txns to cause a checkpoint, and they
+    // will each try to take a checkpoint and upload to each other.
+    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(0, 12));
+    HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(0, 12));
+    
+    assertEquals(12, nn0.getNamesystem().getFSImage()
+        .getMostRecentCheckpointTxId());
+    assertEquals(12, nn1.getNamesystem().getFSImage()
+        .getMostRecentCheckpointTxId());
+    
+    List<File> dirs = Lists.newArrayList();
+    dirs.addAll(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0));
+    dirs.addAll(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 1));
+    FSImageTestUtil.assertParallelFilesAreIdentical(dirs, ImmutableSet.<String>of());
+  }
+  
+  /**
+   * Test for the case when the SBN is configured to checkpoint based
+   * on a time period, but no transactions are happening on the
+   * active. Thus, it would want to save a second checkpoint at the
+   * same txid, which is a no-op. This test makes sure this doesn't
+   * cause any problem.
+   */
+  @Test
+  public void testCheckpointWhenNoNewTransactionsHappened()
+      throws Exception {
+    // Checkpoint as fast as we can, in a tight loop.
+    cluster.getConfiguration(1).setInt(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 0);
+    cluster.restartNameNode(1);
+    nn1 = cluster.getNameNode(1);
+ 
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
+    
+    // We shouldn't save any checkpoints at txid=0
+    Thread.sleep(1000);
+    Mockito.verify(spyImage1, Mockito.never())
+      .saveNamespace((FSNamesystem) Mockito.anyObject());
+ 
+    // Roll the primary and wait for the standby to catch up
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    Thread.sleep(2000);
+    
+    // We should make exactly one checkpoint at this new txid. 
+    Mockito.verify(spyImage1, Mockito.times(1))
+      .saveNamespace((FSNamesystem) Mockito.anyObject());       
+  }
+  
+  /**
+   * Test cancellation of ongoing checkpoints when failover happens
+   * mid-checkpoint. 
+   */
+  @Test
+  public void testCheckpointCancellation() throws Exception {
+    cluster.transitionToStandby(0);
+    
+    // Create an edit log in the shared edits dir with a lot
+    // of mkdirs operations. This is solely so that the image is
+    // large enough to take a non-trivial amount of time to load.
+    // (only ~15MB)
+    URI sharedUri = cluster.getSharedEditsDir(0, 1);
+    File sharedDir = new File(sharedUri.getPath(), "current");
+    File tmpDir = new File(MiniDFSCluster.getBaseDirectory(),
+        "testCheckpointCancellation-tmp");
+    FSImageTestUtil.createAbortedLogWithMkdirs(tmpDir, NUM_DIRS_IN_LOG,
+        3);
+    String fname = NNStorage.getInProgressEditsFileName(3); 
+    new File(tmpDir, fname).renameTo(new File(sharedDir, fname));
+
+    // Checkpoint as fast as we can, in a tight loop.
+    cluster.getConfiguration(1).setInt(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 0);
+    cluster.restartNameNode(1);
+    nn1 = cluster.getNameNode(1);
+
+    cluster.transitionToActive(0);    
+    
+    for (int i = 0; i < 10; i++) {
+      
+      doEdits(i*10, i*10 + 10);
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      cluster.transitionToStandby(1);
+      cluster.transitionToActive(0);
+    }
+    
+    assertTrue(StandbyCheckpointer.getCanceledCount() > 0);
+  }
+
+  private void doEdits(int start, int stop) throws IOException {
+    for (int i = start; i < stop; i++) {
+      Path p = new Path("/test" + i);
+      fs.mkdirs(p);
+    }
+  }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
new file mode 100644
index 0000000..ce5814b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
@@ -0,0 +1,240 @@
+/**
+ * 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.namenode.ha;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.base.Supplier;
+
+/**
+ * The hotornot.com of unit tests: makes sure that the standby not only
+ * has namespace information, but also has the correct block reports, etc.
+ */
+public class TestStandbyIsHot {
+  protected static final Log LOG = LogFactory.getLog(
+      TestStandbyIsHot.class);
+  private static final String TEST_FILE_DATA = "hello highly available world";
+  private static final String TEST_FILE = "/testStandbyIsHot";
+  private static final Path TEST_FILE_PATH = new Path(TEST_FILE);
+
+  static {
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LogFactory.getLog(BlockManager.class)).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+  }
+
+  @Test
+  public void testStandbyIsHot() throws Exception {
+    Configuration conf = new Configuration();
+    // We read from the standby to watch block locations
+    HAUtil.setAllowStandbyReads(conf, true);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(3)
+      .build();
+    Runtime mockRuntime = mock(Runtime.class);
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+      
+      NameNode nn1 = cluster.getNameNode(0);
+      NameNode nn2 = cluster.getNameNode(1);
+      
+      nn2.getNamesystem().getEditLogTailer().setRuntime(mockRuntime);
+      
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      
+      Thread.sleep(1000);
+      System.err.println("==================================");
+      DFSTestUtil.writeFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+      // Have to force an edit log roll so that the standby catches up
+      nn1.getRpcServer().rollEditLog();
+      System.err.println("==================================");
+
+      // Block locations should show up on standby.
+      LOG.info("Waiting for block locations to appear on standby node");
+      waitForBlockLocations(cluster, nn2, TEST_FILE, 3);
+
+      // Trigger immediate heartbeats and block reports so
+      // that the active "trusts" all of the DNs
+      cluster.triggerHeartbeats();
+      cluster.triggerBlockReports();
+
+      // Change replication
+      LOG.info("Changing replication to 1");
+      fs.setReplication(TEST_FILE_PATH, (short)1);
+      waitForBlockLocations(cluster, nn1, TEST_FILE, 1);
+
+      nn1.getRpcServer().rollEditLog();
+      
+      LOG.info("Waiting for lowered replication to show up on standby");
+      waitForBlockLocations(cluster, nn2, TEST_FILE, 1);
+      
+      // Change back to 3
+      LOG.info("Changing replication to 3");
+      fs.setReplication(TEST_FILE_PATH, (short)3);
+      nn1.getRpcServer().rollEditLog();
+      
+      LOG.info("Waiting for higher replication to show up on standby");
+      waitForBlockLocations(cluster, nn2, TEST_FILE, 3);
+      
+    } finally {
+      verify(mockRuntime, times(0)).exit(anyInt());
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Regression test for HDFS-2795:
+   *  - Start an HA cluster with a DN.
+   *  - Write several blocks to the FS with replication 1.
+   *  - Shutdown the DN
+   *  - Wait for the NNs to declare the DN dead. All blocks will be under-replicated.
+   *  - Restart the DN.
+   * In the bug, the standby node would only very slowly notice the blocks returning
+   * to the cluster.
+   */
+  @Test
+  public void testDatanodeRestarts() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
+    // We read from the standby to watch block locations
+    HAUtil.setAllowStandbyReads(conf, true);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .build();
+    try {
+      NameNode nn0 = cluster.getNameNode(0);
+      NameNode nn1 = cluster.getNameNode(1);
+
+      cluster.transitionToActive(0);
+      
+      // Create 5 blocks.
+      DFSTestUtil.createFile(cluster.getFileSystem(0), 
+          TEST_FILE_PATH, 5*1024, (short)1, 1L);
+      
+      HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+      
+      // Stop the DN.
+      DataNode dn = cluster.getDataNodes().get(0);
+      String dnName = dn.getDatanodeId().getName(); 
+      DataNodeProperties dnProps = cluster.stopDataNode(0);
+      
+      // Make sure both NNs register it as dead.
+      BlockManagerTestUtil.noticeDeadDatanode(nn0, dnName);
+      BlockManagerTestUtil.noticeDeadDatanode(nn1, dnName);
+      
+      BlockManagerTestUtil.updateState(nn0.getNamesystem().getBlockManager());
+      BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
+      assertEquals(5, nn0.getNamesystem().getUnderReplicatedBlocks());
+      
+      // The SBN will not have any blocks in its neededReplication queue
+      // since the SBN doesn't process replication.
+      assertEquals(0, nn1.getNamesystem().getUnderReplicatedBlocks());
+      
+      LocatedBlocks locs = nn1.getRpcServer().getBlockLocations(
+          TEST_FILE, 0, 1);
+      assertEquals("Standby should have registered that the block has no replicas",
+          0, locs.get(0).getLocations().length);
+      
+      cluster.restartDataNode(dnProps);
+      // Wait for both NNs to re-register the DN.
+      cluster.waitActive(0);
+      cluster.waitActive(1);
+      
+      BlockManagerTestUtil.updateState(nn0.getNamesystem().getBlockManager());
+      BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
+      assertEquals(0, nn0.getNamesystem().getUnderReplicatedBlocks());
+      assertEquals(0, nn1.getNamesystem().getUnderReplicatedBlocks());
+      
+      locs = nn1.getRpcServer().getBlockLocations(
+          TEST_FILE, 0, 1);
+      assertEquals("Standby should have registered that the block has replicas again",
+          1, locs.get(0).getLocations().length);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  static void waitForBlockLocations(final MiniDFSCluster cluster,
+      final NameNode nn,
+      final String path, final int expectedReplicas)
+      throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      
+      @Override
+      public Boolean get() {
+        try {
+          LocatedBlocks locs = NameNodeAdapter.getBlockLocations(nn, path, 0, 1000);
+          DatanodeInfo[] dnis = locs.getLastLocatedBlock().getLocations();
+          for (DatanodeInfo dni : dnis) {
+            Assert.assertNotNull(dni);
+          }
+          int numReplicas = dnis.length;
+          
+          LOG.info("Got " + numReplicas + " locs: " + locs);
+          if (numReplicas > expectedReplicas) {
+            for (DataNode dn : cluster.getDataNodes()) {
+              DataNodeAdapter.triggerDeletionReport(dn);
+            }
+          }
+          return numReplicas == expectedReplicas;
+        } catch (IOException e) {
+          LOG.warn("No block locations yet: " + e.getMessage());
+          return false;
+        }
+      }
+    }, 500, 10000);
+    
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index c993f6c..79c7047 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -163,8 +163,13 @@
     // Corrupt first replica of the block
     LocatedBlock block = NameNodeAdapter.getBlockLocations(
         cluster.getNameNode(), file.toString(), 0, 1).get(0);
-    bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
-        "TEST");
+    cluster.getNamesystem().writeLock();
+    try {
+      bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
+          "TEST");
+    } finally {
+      cluster.getNamesystem().writeUnlock();
+    }
     Thread.sleep(1000); // Wait for block to be marked corrupt
     MetricsRecordBuilder rb = getMetrics(NS_METRICS);
     assertGauge("CorruptBlocks", 1L, rb);
@@ -202,8 +207,13 @@
     // Corrupt the only replica of the block to result in a missing block
     LocatedBlock block = NameNodeAdapter.getBlockLocations(
         cluster.getNameNode(), file.toString(), 0, 1).get(0);
-    bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
-        "TEST");
+    cluster.getNamesystem().writeLock();
+    try {
+      bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
+          "TEST");
+    } finally {
+      cluster.getNamesystem().writeUnlock();
+    }
     Thread.sleep(1000); // Wait for block to be marked corrupt
     MetricsRecordBuilder rb = getMetrics(NS_METRICS);
     assertGauge("UnderReplicatedBlocks", 1L, rb);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
new file mode 100644
index 0000000..355009a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
@@ -0,0 +1,234 @@
+/**
+ * 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.tools;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.ha.HealthCheckFailedException;
+import org.apache.hadoop.ha.NodeFencer;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.when;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+
+public class TestDFSHAAdmin {
+  private static final Log LOG = LogFactory.getLog(TestDFSHAAdmin.class);
+  
+  private DFSHAAdmin tool;
+  private ByteArrayOutputStream errOutBytes = new ByteArrayOutputStream();
+  private String errOutput;
+  private HAServiceProtocol mockProtocol;
+  
+  private static final String NSID = "ns1";
+  private static String HOST_A = "1.2.3.1";
+  private static String HOST_B = "1.2.3.2";
+
+  private HdfsConfiguration getHAConf() {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, NSID);    
+    conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID, NSID);
+    conf.set(DFSUtil.addKeySuffixes(
+        DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, NSID), "nn1,nn2");    
+    conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
+    conf.set(DFSUtil.addKeySuffixes(
+            DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, NSID, "nn1"),
+        HOST_A + ":12345");
+    conf.set(DFSUtil.addKeySuffixes(
+            DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, NSID, "nn2"),
+        HOST_B + ":12345");
+    return conf;
+  }
+
+  @Before
+  public void setup() throws IOException {
+    mockProtocol = Mockito.mock(HAServiceProtocol.class);
+    when(mockProtocol.readyToBecomeActive()).thenReturn(true);
+    tool = new DFSHAAdmin() {
+      @Override
+      protected HAServiceProtocol getProtocol(String serviceId) throws IOException {
+        getServiceAddr(serviceId);
+        return mockProtocol;
+      }
+    };
+    tool.setConf(getHAConf());
+    tool.setErrOut(new PrintStream(errOutBytes));
+  }
+
+  private void assertOutputContains(String string) {
+    if (!errOutput.contains(string)) {
+      fail("Expected output to contain '" + string + "' but was:\n" +
+          errOutput);
+    }
+  }
+  
+  @Test
+  public void testNameserviceOption() throws Exception {
+    assertEquals(-1, runTool("-ns"));
+    assertOutputContains("Missing nameservice ID");
+    assertEquals(-1, runTool("-ns", "ns1"));
+    assertOutputContains("Missing command");
+    // "ns1" isn't defined but we check this lazily and help doesn't use the ns
+    assertEquals(0, runTool("-ns", "ns1", "-help", "transitionToActive"));
+    assertOutputContains("Transitions the service into Active");
+  }
+
+  @Test
+  public void testNamenodeResolution() throws Exception {
+    assertEquals(0, runTool("-getServiceState", "nn1"));
+    Mockito.verify(mockProtocol).getServiceState();
+    assertEquals(-1, runTool("-getServiceState", "undefined"));
+    assertOutputContains(
+        "Unable to determine service address for namenode 'undefined'");
+  }
+
+  @Test
+  public void testHelp() throws Exception {
+    assertEquals(-1, runTool("-help"));
+    assertEquals(0, runTool("-help", "transitionToActive"));
+    assertOutputContains("Transitions the service into Active");
+  }
+  
+  @Test
+  public void testTransitionToActive() throws Exception {
+    assertEquals(0, runTool("-transitionToActive", "nn1"));
+    Mockito.verify(mockProtocol).transitionToActive();
+  }
+
+  @Test
+  public void testTransitionToStandby() throws Exception {
+    assertEquals(0, runTool("-transitionToStandby", "nn1"));
+    Mockito.verify(mockProtocol).transitionToStandby();
+  }
+
+  @Test
+  public void testFailoverWithNoFencerConfigured() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    assertEquals(-1, runTool("-failover", "nn1", "nn2"));
+  }
+
+  @Test
+  public void testFailoverWithFencerConfigured() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "nn1", "nn2"));
+  }
+
+  @Test
+  public void testFailoverWithFencerAndNameservice() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-ns", "ns1", "-failover", "nn1", "nn2"));
+  }
+
+  @Test
+  public void testFailoverWithFencerConfiguredAndForce() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
+  }
+
+  @Test
+  public void testFailoverWithForceActive() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "nn1", "nn2", "--forceactive"));
+  }
+
+  @Test
+  public void testFailoverWithInvalidFenceArg() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "notforcefence"));
+  }
+
+  @Test
+  public void testFailoverWithFenceButNoFencer() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
+  }
+
+  @Test
+  public void testFailoverWithFenceAndBadFencer() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "foobar!");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
+  }
+
+  @Test
+  public void testForceFenceOptionListedBeforeArgs() throws Exception {
+    Mockito.doReturn(HAServiceState.STANDBY).when(mockProtocol).getServiceState();
+    HdfsConfiguration conf = getHAConf();
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "--forcefence", "nn1", "nn2"));
+  }
+
+  @Test
+  public void testGetServiceState() throws Exception {
+    assertEquals(0, runTool("-getServiceState", "nn1"));
+    Mockito.verify(mockProtocol).getServiceState();
+  }
+
+  @Test
+  public void testCheckHealth() throws Exception {
+    assertEquals(0, runTool("-checkHealth", "nn1"));
+    Mockito.verify(mockProtocol).monitorHealth();
+    
+    Mockito.doThrow(new HealthCheckFailedException("fake health check failure"))
+      .when(mockProtocol).monitorHealth();
+    assertEquals(-1, runTool("-checkHealth", "nn1"));
+    assertOutputContains("Health check failed: fake health check failure");
+  }
+
+  private Object runTool(String ... args) throws Exception {
+    errOutBytes.reset();
+    LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args));
+    int ret = tool.run(args);
+    errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8);
+    LOG.info("Output:\n" + errOutput);
+    return ret;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
new file mode 100644
index 0000000..0302c8e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
@@ -0,0 +1,143 @@
+/**
+ * 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.tools;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.ha.NodeFencer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+
+/**
+ * Tests for HAAdmin command with {@link MiniDFSCluster} set up in HA mode.
+ */
+public class TestDFSHAAdminMiniCluster {
+  private static final Log LOG = LogFactory.getLog(TestDFSHAAdminMiniCluster.class);
+  
+  private MiniDFSCluster cluster;
+  private Configuration conf; 
+  private DFSHAAdmin tool;
+  
+  @Before
+  public void setup() throws IOException {
+    conf = new Configuration();
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
+        .build();
+    tool = new DFSHAAdmin();  
+    tool.setConf(conf);
+    cluster.waitActive();
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    cluster.shutdown();
+  }
+  
+  @Test
+  public void testGetServiceState() throws Exception {
+    assertEquals(0, runTool("-getServiceState", "nn1"));
+    assertEquals(0, runTool("-getServiceState", "nn2"));
+  }
+    
+  @Test 
+  public void testStateTransition() throws Exception {
+    NameNode nnode1 = cluster.getNameNode(0);
+    assertTrue(nnode1.isStandbyState());
+    assertEquals(0, runTool("-transitionToActive", "nn1"));
+    assertFalse(nnode1.isStandbyState());       
+    assertEquals(0, runTool("-transitionToStandby", "nn1"));
+    assertTrue(nnode1.isStandbyState());
+    
+    NameNode nnode2 = cluster.getNameNode(1);
+    assertTrue(nnode2.isStandbyState());
+    assertEquals(0, runTool("-transitionToActive", "nn2"));
+    assertFalse(nnode2.isStandbyState());
+    assertEquals(0, runTool("-transitionToStandby", "nn2"));
+    assertTrue(nnode2.isStandbyState());
+  }
+    
+  /**
+   * Test failover with various options
+   */
+  @Test
+  public void testFencer() throws Exception { 
+    // Test failover with no fencer
+    assertEquals(-1, runTool("-failover", "nn1", "nn2"));
+    
+    // Test failover with fencer
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-transitionToActive", "nn1"));
+    assertEquals(0, runTool("-failover", "nn1", "nn2"));
+    
+    // Test failover with fencer and nameservice
+    assertEquals(0, runTool("-ns", "minidfs-ns", "-failover", "nn2", "nn1"));
+
+    // Test failover with fencer and forcefence option
+    assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
+      
+    // Test failover with forceactive option
+    assertEquals(0, runTool("-failover", "nn2", "nn1", "--forceactive"));
+          
+    // Test failover with not fencer and forcefence option
+    conf.unset(NodeFencer.CONF_METHODS_KEY);
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
+    
+    // Test failover with bad fencer and forcefence option
+    conf.set(NodeFencer.CONF_METHODS_KEY, "foobar!");
+    tool.setConf(conf);
+    assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
+
+    // Test failover with force fence listed before the other arguments
+    conf.set(NodeFencer.CONF_METHODS_KEY, "shell(true)");
+    tool.setConf(conf);
+    assertEquals(0, runTool("-failover", "--forcefence", "nn1", "nn2"));
+  }
+     
+  @Test
+  public void testCheckHealth() throws Exception {
+    assertEquals(0, runTool("-checkHealth", "nn1"));
+    assertEquals(0, runTool("-checkHealth", "nn2"));
+  }
+  
+  private int runTool(String ... args) throws Exception {
+    ByteArrayOutputStream errOutBytes = new ByteArrayOutputStream();
+    errOutBytes.reset();
+    LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args));
+    int ret = tool.run(args);
+    String errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8);
+    LOG.info("Output:\n" + errOutput);
+    return ret;
+  }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
index 7152e12..97be2b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
@@ -24,6 +24,7 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.StringTokenizer;
 
 import static org.junit.Assert.*;
@@ -32,6 +33,7 @@
 import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.tools.GetConf;
 import org.apache.hadoop.hdfs.tools.GetConf.Command;
@@ -72,7 +74,7 @@
     String[] values = new String[nameServiceIdCount];
     for (int i = 0; i < nameServiceIdCount; i++, portOffset++) {
       String nsID = getNameServiceId(i);
-      String specificKey = DFSUtil.getNameServiceIdKey(key, nsID);
+      String specificKey = DFSUtil.addKeySuffixes(key, nsID);
       values[i] = "nn" + i + ":" + portOffset;
       conf.set(specificKey, values[i]);
     }
@@ -80,13 +82,13 @@
   }
 
   /*
-   * Convert list of InetSocketAddress to string array with each address
-   * represented as "host:port"
+   * Convert the map returned from DFSUtil functions to an array of
+   * addresses represented as "host:port"
    */
-  private String[] toStringArray(List<InetSocketAddress> list) {
+  private String[] toStringArray(List<ConfiguredNNAddress> list) {
     String[] ret = new String[list.size()];
     for (int i = 0; i < list.size(); i++) {
-      ret[i] = NetUtils.getHostPortString(list.get(i));
+      ret[i] = NetUtils.getHostPortString(list.get(i).getAddress());
     }
     return ret;
   }
@@ -94,8 +96,8 @@
   /**
    * Using DFSUtil methods get the list of given {@code type} of address
    */
-  private List<InetSocketAddress> getAddressListFromConf(TestType type,
-      HdfsConfiguration conf) throws IOException {
+  private Map<String, Map<String, InetSocketAddress>> getAddressListFromConf(
+      TestType type, HdfsConfiguration conf) throws IOException {
     switch (type) {
     case NAMENODE:
       return DFSUtil.getNNServiceRpcAddresses(conf);
@@ -161,7 +163,7 @@
    * @param expected, expected addresses
    */
   private void getAddressListFromTool(TestType type, HdfsConfiguration conf,
-      boolean checkPort, List<InetSocketAddress> expected) throws Exception {
+      boolean checkPort, List<ConfiguredNNAddress> expected) throws Exception {
     String out = getAddressListFromTool(type, conf, expected.size() != 0);
     List<String> values = new ArrayList<String>();
     
@@ -176,7 +178,8 @@
     // Convert expected list to String[] of hosts
     int i = 0;
     String[] expectedHosts = new String[expected.size()];
-    for (InetSocketAddress addr : expected) {
+    for (ConfiguredNNAddress cnn : expected) {
+      InetSocketAddress addr = cnn.getAddress();
       if (!checkPort) {
         expectedHosts[i++] = addr.getHostName();
       }else {
@@ -191,7 +194,9 @@
   private void verifyAddresses(HdfsConfiguration conf, TestType type,
       boolean checkPort, String... expected) throws Exception {
     // Ensure DFSUtil returned the right set of addresses
-    List<InetSocketAddress> list = getAddressListFromConf(type, conf);
+    Map<String, Map<String, InetSocketAddress>> map =
+      getAddressListFromConf(type, conf);
+    List<ConfiguredNNAddress> list = DFSUtil.flattenAddressMap(map);
     String[] actual = toStringArray(list);
     Arrays.sort(actual);
     Arrays.sort(expected);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
index 966e52f..5d3272a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
@@ -31,6 +31,7 @@
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
@@ -79,7 +80,7 @@
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
 
     cluster = new MiniDFSCluster.Builder(conf)
-        .numNameNodes(nNameNodes)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
         .numDataNodes(nDataNodes)
         .build();
     cluster.waitActive();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
index 13e9683..23d1bb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
@@ -20,6 +20,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeoutException;
@@ -79,8 +80,8 @@
   public static void assertExceptionContains(String string, Throwable t) {
     String msg = t.getMessage();
     Assert.assertTrue(
-        "Unexpected exception:" + StringUtils.stringifyException(t),
-        msg.contains(string));    
+        "Expected to find '" + string + "' but got unexpected exception:"
+        + StringUtils.stringifyException(t), msg.contains(string));
   }  
 
   public static void waitFor(Supplier<Boolean> check,
@@ -109,7 +110,11 @@
     
     private final CountDownLatch fireLatch = new CountDownLatch(1);
     private final CountDownLatch waitLatch = new CountDownLatch(1);
-  
+    private final CountDownLatch resultLatch = new CountDownLatch(1);
+    
+    // Result fields set after proceed() is called.
+    private volatile Throwable thrown;
+    private volatile Object returnValue;
     
     public DelayAnswer(Log log) {
       this.LOG = log;
@@ -144,7 +149,40 @@
     }
 
     protected Object passThrough(InvocationOnMock invocation) throws Throwable {
-      return invocation.callRealMethod();
+      try {
+        Object ret = invocation.callRealMethod();
+        returnValue = ret;
+        return ret;
+      } catch (Throwable t) {
+        thrown = t;
+        throw t;
+      } finally {
+        resultLatch.countDown();
+      }
+    }
+    
+    /**
+     * After calling proceed(), this will wait until the call has
+     * completed and a result has been returned to the caller.
+     */
+    public void waitForResult() throws InterruptedException {
+      resultLatch.await();
+    }
+    
+    /**
+     * After the call has gone through, return any exception that
+     * was thrown, or null if no exception was thrown.
+     */
+    public Throwable getThrown() {
+      return thrown;
+    }
+    
+    /**
+     * After the call has gone through, return the call's return value,
+     * or null in case it was void or an exception was thrown.
+     */
+    public Object getReturnValue() {
+      return returnValue;
     }
   }
   
@@ -176,4 +214,35 @@
     }
   }
 
+  /**
+   * An Answer implementation which sleeps for a random number of milliseconds
+   * between 0 and a configurable value before delegating to the real
+   * implementation of the method. This can be useful for drawing out race
+   * conditions.
+   */
+  public static class SleepAnswer implements Answer<Object> {
+    private final int maxSleepTime;
+    private static Random r = new Random();
+    
+    public SleepAnswer(int maxSleepTime) {
+      this.maxSleepTime = maxSleepTime;
+    }
+    
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      boolean interrupted = false;
+      try {
+        Thread.sleep(r.nextInt(maxSleepTime));
+      } catch (InterruptedException ie) {
+        interrupted = true;
+      }
+      try {
+        return invocation.callRealMethod();
+      } finally {
+        if (interrupted) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
index 0101672..5099ce2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
Binary files differ
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
index 65fe23a..acc34bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
@@ -1,34 +1,34 @@
 <?xml version="1.0"?>
 <EDITS>
-  <EDITS_VERSION>-38</EDITS_VERSION>
+  <EDITS_VERSION>-40</EDITS_VERSION>
   <RECORD>
     <OPCODE>24</OPCODE>
     <DATA>
       <TRANSACTION_ID>1</TRANSACTION_ID>
     </DATA>
-    <CHECKSUM>1504643968</CHECKSUM>
+    <CHECKSUM>-2045328303</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>21</OPCODE>
     <DATA>
       <TRANSACTION_ID>2</TRANSACTION_ID>
       <KEY_ID>1</KEY_ID>
-      <KEY_EXPIRY_DATE>1304751257518</KEY_EXPIRY_DATE>
+      <KEY_EXPIRY_DATE>1331096884634</KEY_EXPIRY_DATE>
       <KEY_LENGTH>3</KEY_LENGTH>
-      <KEY_BLOB>2FhO</KEY_BLOB>
+      <KEY_BLOB>o0v1</KEY_BLOB>
     </DATA>
-    <CHECKSUM>-174778556</CHECKSUM>
+    <CHECKSUM>-1521490291</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>21</OPCODE>
     <DATA>
       <TRANSACTION_ID>3</TRANSACTION_ID>
       <KEY_ID>2</KEY_ID>
-      <KEY_EXPIRY_DATE>1304751257521</KEY_EXPIRY_DATE>
+      <KEY_EXPIRY_DATE>1331096884637</KEY_EXPIRY_DATE>
       <KEY_LENGTH>3</KEY_LENGTH>
-      <KEY_BLOB>77-r</KEY_BLOB>
+      <KEY_BLOB>3WMF</KEY_BLOB>
     </DATA>
-    <CHECKSUM>1565957291</CHECKSUM>
+    <CHECKSUM>65546244</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>10</OPCODE>
@@ -42,11 +42,10 @@
     <OPCODE>0</OPCODE>
     <DATA>
       <TRANSACTION_ID>5</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057562</MTIME>
-      <ATIME>1304060057562</ATIME>
+      <MTIME>1330405685834</MTIME>
+      <ATIME>1330405685834</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
@@ -54,20 +53,19 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-66857152_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>-1854451489</CHECKSUM>
+    <CHECKSUM>179250704</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>9</OPCODE>
     <DATA>
       <TRANSACTION_ID>6</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057572</MTIME>
-      <ATIME>1304060057562</ATIME>
+      <MTIME>1330405685848</MTIME>
+      <ATIME>1330405685834</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
@@ -76,44 +74,41 @@
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>617592855</CHECKSUM>
+    <CHECKSUM>-584136658</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>1</OPCODE>
     <DATA>
       <TRANSACTION_ID>7</TRANSACTION_ID>
-      <LENGTH>3</LENGTH>
       <SOURCE>/file_create</SOURCE>
       <DESTINATION>/file_moved</DESTINATION>
-      <TIMESTAMP>1304060057575</TIMESTAMP>
+      <TIMESTAMP>1330405685852</TIMESTAMP>
     </DATA>
-    <CHECKSUM>367100554</CHECKSUM>
+    <CHECKSUM>-1983534581</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>2</OPCODE>
     <DATA>
       <TRANSACTION_ID>8</TRANSACTION_ID>
-      <LENGTH>2</LENGTH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1304060057577</TIMESTAMP>
+      <TIMESTAMP>1330405685857</TIMESTAMP>
     </DATA>
-    <CHECKSUM>1048346698</CHECKSUM>
+    <CHECKSUM>-97648053</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>3</OPCODE>
     <DATA>
       <TRANSACTION_ID>9</TRANSACTION_ID>
-      <LENGTH>3</LENGTH>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1304060057581</TIMESTAMP>
-      <ATIME>0</ATIME>
+      <TIMESTAMP>1330405685861</TIMESTAMP>
+      <ATIME>1330405685861</ATIME>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>493</FS_PERMISSIONS>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>1207240248</CHECKSUM>
+    <CHECKSUM>-146811985</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>10</OPCODE>
@@ -127,11 +122,10 @@
     <OPCODE>0</OPCODE>
     <DATA>
       <TRANSACTION_ID>11</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057584</MTIME>
-      <ATIME>1304060057584</ATIME>
+      <MTIME>1330405685866</MTIME>
+      <ATIME>1330405685866</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
@@ -139,20 +133,19 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-66857152_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>1796314473</CHECKSUM>
+    <CHECKSUM>806955943</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>9</OPCODE>
     <DATA>
       <TRANSACTION_ID>12</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057588</MTIME>
-      <ATIME>1304060057584</ATIME>
+      <MTIME>1330405685868</MTIME>
+      <ATIME>1330405685866</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
@@ -161,7 +154,7 @@
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>1017626905</CHECKSUM>
+    <CHECKSUM>641893387</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>4</OPCODE>
@@ -170,7 +163,7 @@
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
     </DATA>
-    <CHECKSUM>1842610087</CHECKSUM>
+    <CHECKSUM>24198146</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>7</OPCODE>
@@ -195,12 +188,11 @@
     <OPCODE>13</OPCODE>
     <DATA>
       <TRANSACTION_ID>16</TRANSACTION_ID>
-      <LENGTH>3</LENGTH>
       <PATH>/file_create</PATH>
       <MTIME>1285195527000</MTIME>
       <ATIME>1285195527000</ATIME>
     </DATA>
-    <CHECKSUM>1428793678</CHECKSUM>
+    <CHECKSUM>1853168961</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>14</OPCODE>
@@ -216,13 +208,12 @@
     <OPCODE>15</OPCODE>
     <DATA>
       <TRANSACTION_ID>18</TRANSACTION_ID>
-      <LENGTH>3</LENGTH>
       <SOURCE>/file_create</SOURCE>
       <DESTINATION>/file_moved</DESTINATION>
-      <TIMESTAMP>1304060057605</TIMESTAMP>
+      <TIMESTAMP>1330405685882</TIMESTAMP>
       <RENAME_OPTIONS>AA</RENAME_OPTIONS>
     </DATA>
-    <CHECKSUM>-1155144192</CHECKSUM>
+    <CHECKSUM>-1235158297</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>10</OPCODE>
@@ -236,11 +227,10 @@
     <OPCODE>0</OPCODE>
     <DATA>
       <TRANSACTION_ID>20</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057613</MTIME>
-      <ATIME>1304060057613</ATIME>
+      <MTIME>1330405685889</MTIME>
+      <ATIME>1330405685889</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
@@ -248,125 +238,141 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-66857152_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>-428545606</CHECKSUM>
-  </RECORD>
-  <RECORD>
-    <OPCODE>9</OPCODE>
-    <DATA>
-      <TRANSACTION_ID>21</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
-      <PATH>/file_concat_target</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057694</MTIME>
-      <ATIME>1304060057613</ATIME>
-      <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>3</NUMBLOCKS>
-      <BLOCK>
-        <BLOCK_ID>3459038074990663911</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
-      </BLOCK>
-      <BLOCK>
-        <BLOCK_ID>-5555244278278879146</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
-      </BLOCK>
-      <BLOCK>
-        <BLOCK_ID>-6344128791846831740</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
-      </BLOCK>
-      <PERMISSION_STATUS>
-        <USERNAME>todd</USERNAME>
-        <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
-      </PERMISSION_STATUS>
-    </DATA>
-    <CHECKSUM>707995174</CHECKSUM>
+    <CHECKSUM>-981119572</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>10</OPCODE>
     <DATA>
-      <TRANSACTION_ID>22</TRANSACTION_ID>
+      <TRANSACTION_ID>21</TRANSACTION_ID>
       <GENERATION_STAMP>1004</GENERATION_STAMP>
     </DATA>
-    <CHECKSUM>-1500977009</CHECKSUM>
+    <CHECKSUM>-1627007926</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>0</OPCODE>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>22</TRANSACTION_ID>
+      <PATH>/file_concat_target</PATH>
+      <NUMBLOCKS>1</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7144805496741076283</BLOCK_ID>
+        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-1131701615</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
     <DATA>
       <TRANSACTION_ID>23</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
-      <PATH>/file_concat_0</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057701</MTIME>
-      <ATIME>1304060057701</ATIME>
-      <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>0</NUMBLOCKS>
-      <PERMISSION_STATUS>
-        <USERNAME>todd</USERNAME>
-        <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
-      </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-66857152_1</CLIENT_NAME>
-      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+      <GENERATION_STAMP>1005</GENERATION_STAMP>
     </DATA>
-    <CHECKSUM>-119850856</CHECKSUM>
+    <CHECKSUM>-957035430</CHECKSUM>
   </RECORD>
   <RECORD>
-    <OPCODE>9</OPCODE>
+    <OPCODE>25</OPCODE>
     <DATA>
       <TRANSACTION_ID>24</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
-      <PATH>/file_concat_0</PATH>
-      <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057737</MTIME>
-      <ATIME>1304060057701</ATIME>
-      <BLOCKSIZE>512</BLOCKSIZE>
-      <NUMBLOCKS>3</NUMBLOCKS>
+      <PATH>/file_concat_target</PATH>
+      <NUMBLOCKS>2</NUMBLOCKS>
       <BLOCK>
-        <BLOCK_ID>4671949296381030428</BLOCK_ID>
+        <BLOCK_ID>-7144805496741076283</BLOCK_ID>
         <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
         <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
-        <BLOCK_ID>-844362243522407159</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+        <BLOCK_ID>-4125931756867080767</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
       </BLOCK>
-      <BLOCK>
-        <BLOCK_ID>3476886462779656950</BLOCK_ID>
-        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
-      </BLOCK>
-      <PERMISSION_STATUS>
-        <USERNAME>todd</USERNAME>
-        <GROUPNAME>supergroup</GROUPNAME>
-        <FS_PERMISSIONS>420</FS_PERMISSIONS>
-      </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-766805874</CHECKSUM>
+    <CHECKSUM>-932985519</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>10</OPCODE>
     <DATA>
       <TRANSACTION_ID>25</TRANSACTION_ID>
-      <GENERATION_STAMP>1005</GENERATION_STAMP>
+      <GENERATION_STAMP>1006</GENERATION_STAMP>
     </DATA>
-    <CHECKSUM>238426056</CHECKSUM>
+    <CHECKSUM>-1757460878</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>26</TRANSACTION_ID>
+      <PATH>/file_concat_target</PATH>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7144805496741076283</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-4125931756867080767</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>0</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>1562413691487277050</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-154090859</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>27</TRANSACTION_ID>
+      <PATH>/file_concat_target</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1330405685978</MTIME>
+      <ATIME>1330405685889</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7144805496741076283</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-4125931756867080767</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>1562413691487277050</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1006</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>todd</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+    <CHECKSUM>-292633850</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>28</TRANSACTION_ID>
+      <GENERATION_STAMP>1007</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>-1431358549</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>0</OPCODE>
     <DATA>
-      <TRANSACTION_ID>26</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
-      <PATH>/file_concat_1</PATH>
+      <TRANSACTION_ID>29</TRANSACTION_ID>
+      <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057742</MTIME>
-      <ATIME>1304060057742</ATIME>
+      <MTIME>1330405685983</MTIME>
+      <ATIME>1330405685983</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
@@ -374,36 +380,116 @@
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-66857152_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>1156254705</CHECKSUM>
+    <CHECKSUM>-318194869</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>30</TRANSACTION_ID>
+      <GENERATION_STAMP>1008</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>156309208</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>31</TRANSACTION_ID>
+      <PATH>/file_concat_0</PATH>
+      <NUMBLOCKS>1</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>6084289468290363112</BLOCK_ID>
+        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-596016492</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>32</TRANSACTION_ID>
+      <GENERATION_STAMP>1009</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>-1734001394</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>33</TRANSACTION_ID>
+      <PATH>/file_concat_0</PATH>
+      <NUMBLOCKS>2</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>6084289468290363112</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-4219431127125026105</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>1352178323</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>34</TRANSACTION_ID>
+      <GENERATION_STAMP>1010</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>794444850</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>35</TRANSACTION_ID>
+      <PATH>/file_concat_0</PATH>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>6084289468290363112</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-4219431127125026105</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>0</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-1765119074945211374</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-1530696539</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>9</OPCODE>
     <DATA>
-      <TRANSACTION_ID>27</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
-      <PATH>/file_concat_1</PATH>
+      <TRANSACTION_ID>36</TRANSACTION_ID>
+      <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1304060057764</MTIME>
-      <ATIME>1304060057742</ATIME>
+      <MTIME>1330405686013</MTIME>
+      <ATIME>1330405685983</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>3</NUMBLOCKS>
       <BLOCK>
-        <BLOCK_ID>-754893470864399741</BLOCK_ID>
+        <BLOCK_ID>6084289468290363112</BLOCK_ID>
         <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+        <BLOCK_GENERATION_STAMP>1008</BLOCK_GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
-        <BLOCK_ID>1820875380010181049</BLOCK_ID>
+        <BLOCK_ID>-4219431127125026105</BLOCK_ID>
         <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+        <BLOCK_GENERATION_STAMP>1009</BLOCK_GENERATION_STAMP>
       </BLOCK>
       <BLOCK>
-        <BLOCK_ID>8266387560744259971</BLOCK_ID>
+        <BLOCK_ID>-1765119074945211374</BLOCK_ID>
         <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
-        <BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
+        <BLOCK_GENERATION_STAMP>1010</BLOCK_GENERATION_STAMP>
       </BLOCK>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
@@ -411,121 +497,336 @@
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>-654780301</CHECKSUM>
+    <CHECKSUM>-2043978220</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>37</TRANSACTION_ID>
+      <GENERATION_STAMP>1011</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>1010571629</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>0</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>38</TRANSACTION_ID>
+      <PATH>/file_concat_1</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1330405686017</MTIME>
+      <ATIME>1330405686017</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>0</NUMBLOCKS>
+      <PERMISSION_STATUS>
+        <USERNAME>todd</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+    </DATA>
+    <CHECKSUM>-501297097</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>39</TRANSACTION_ID>
+      <GENERATION_STAMP>1012</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>-1934711736</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>40</TRANSACTION_ID>
+      <PATH>/file_concat_1</PATH>
+      <NUMBLOCKS>1</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7448471719302683860</BLOCK_ID>
+        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-1853122907</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>41</TRANSACTION_ID>
+      <GENERATION_STAMP>1013</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>862670668</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>42</TRANSACTION_ID>
+      <PATH>/file_concat_1</PATH>
+      <NUMBLOCKS>2</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7448471719302683860</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-8051065559769974521</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-1169706939</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>43</TRANSACTION_ID>
+      <GENERATION_STAMP>1014</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>-2070661520</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>44</TRANSACTION_ID>
+      <PATH>/file_concat_1</PATH>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7448471719302683860</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-8051065559769974521</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>0</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>3808670437711973616</BLOCK_ID>
+        <BLOCK_DELTA_NUM_BYTES>-512</BLOCK_DELTA_NUM_BYTES>
+        <BLOCK_DELTA_GEN_STAMP>1</BLOCK_DELTA_GEN_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-1568093815</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>45</TRANSACTION_ID>
+      <PATH>/file_concat_1</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1330405686042</MTIME>
+      <ATIME>1330405686017</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>3</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-7448471719302683860</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1012</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>-8051065559769974521</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1013</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <BLOCK>
+        <BLOCK_ID>3808670437711973616</BLOCK_ID>
+        <BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1014</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>todd</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+    <CHECKSUM>-1640101896</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>16</OPCODE>
     <DATA>
-      <TRANSACTION_ID>28</TRANSACTION_ID>
-      <LENGTH>4</LENGTH>
+      <TRANSACTION_ID>46</TRANSACTION_ID>
       <CONCAT_TARGET>/file_concat_target</CONCAT_TARGET>
+      <LENGTH>2</LENGTH>
       <CONCAT_SOURCE>/file_concat_0</CONCAT_SOURCE>
       <CONCAT_SOURCE>/file_concat_1</CONCAT_SOURCE>
-      <TIMESTAMP>1304060057767</TIMESTAMP>
+      <TIMESTAMP>1330405686046</TIMESTAMP>
     </DATA>
-    <CHECKSUM>1273279541</CHECKSUM>
+    <CHECKSUM>2122891157</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>17</OPCODE>
     <DATA>
-      <TRANSACTION_ID>29</TRANSACTION_ID>
-      <LENGTH>4</LENGTH>
+      <TRANSACTION_ID>47</TRANSACTION_ID>
       <SOURCE>/file_symlink</SOURCE>
       <DESTINATION>/file_concat_target</DESTINATION>
-      <MTIME>1304060057770</MTIME>
-      <ATIME>1304060057770</ATIME>
+      <MTIME>1330405686051</MTIME>
+      <ATIME>1330405686051</ATIME>
       <PERMISSION_STATUS>
         <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>511</FS_PERMISSIONS>
       </PERMISSION_STATUS>
     </DATA>
-    <CHECKSUM>1385678569</CHECKSUM>
+    <CHECKSUM>-585385283</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>18</OPCODE>
     <DATA>
-      <TRANSACTION_ID>30</TRANSACTION_ID>
+      <TRANSACTION_ID>48</TRANSACTION_ID>
       <T_VERSION>0</T_VERSION>
       <T_OWNER>todd</T_OWNER>
       <T_RENEWER>JobTracker</T_RENEWER>
       <T_REAL_USER/>
-      <T_ISSUE_DATE>1304060057773</T_ISSUE_DATE>
-      <T_MAX_DATE>1304664857773</T_MAX_DATE>
+      <T_ISSUE_DATE>1330405686056</T_ISSUE_DATE>
+      <T_MAX_DATE>1331010486056</T_MAX_DATE>
       <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
       <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
-      <T_EXPIRY_TIME>1304146457773</T_EXPIRY_TIME>
+      <T_EXPIRY_TIME>1330492086056</T_EXPIRY_TIME>
     </DATA>
-    <CHECKSUM>913145699</CHECKSUM>
+    <CHECKSUM>791321007</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>19</OPCODE>
     <DATA>
-      <TRANSACTION_ID>31</TRANSACTION_ID>
+      <TRANSACTION_ID>49</TRANSACTION_ID>
       <T_VERSION>0</T_VERSION>
       <T_OWNER>todd</T_OWNER>
       <T_RENEWER>JobTracker</T_RENEWER>
       <T_REAL_USER/>
-      <T_ISSUE_DATE>1304060057773</T_ISSUE_DATE>
-      <T_MAX_DATE>1304664857773</T_MAX_DATE>
+      <T_ISSUE_DATE>1330405686056</T_ISSUE_DATE>
+      <T_MAX_DATE>1331010486056</T_MAX_DATE>
       <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
       <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
-      <T_EXPIRY_TIME>1304146457785</T_EXPIRY_TIME>
+      <T_EXPIRY_TIME>1330492086075</T_EXPIRY_TIME>
     </DATA>
-    <CHECKSUM>-1772039941</CHECKSUM>
+    <CHECKSUM>649714969</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>20</OPCODE>
     <DATA>
-      <TRANSACTION_ID>32</TRANSACTION_ID>
+      <TRANSACTION_ID>50</TRANSACTION_ID>
       <T_VERSION>0</T_VERSION>
       <T_OWNER>todd</T_OWNER>
       <T_RENEWER>JobTracker</T_RENEWER>
       <T_REAL_USER/>
-      <T_ISSUE_DATE>1304060057773</T_ISSUE_DATE>
-      <T_MAX_DATE>1304664857773</T_MAX_DATE>
+      <T_ISSUE_DATE>1330405686056</T_ISSUE_DATE>
+      <T_MAX_DATE>1331010486056</T_MAX_DATE>
       <T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
       <T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
     </DATA>
-    <CHECKSUM>1382094146</CHECKSUM>
+    <CHECKSUM>1190872628</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>51</TRANSACTION_ID>
+      <GENERATION_STAMP>1015</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>-460593521</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>0</OPCODE>
     <DATA>
-      <TRANSACTION_ID>33</TRANSACTION_ID>
-      <LENGTH>5</LENGTH>
-      <PATH>/reassign-lease-test</PATH>
+      <TRANSACTION_ID>52</TRANSACTION_ID>
+      <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1286491964741</MTIME>
-      <ATIME>1286491964741</ATIME>
+      <MTIME>1330405686084</MTIME>
+      <ATIME>1330405686084</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <NUMBLOCKS>0</NUMBLOCKS>
       <PERMISSION_STATUS>
-        <USERNAME>atm</USERNAME>
+        <USERNAME>todd</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <FS_PERMISSIONS>420</FS_PERMISSIONS>
       </PERMISSION_STATUS>
-      <CLIENT_NAME>DFSClient_871171074</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
     </DATA>
-    <CHECKSUM>1975140107</CHECKSUM>
+    <CHECKSUM>2093219037</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>53</TRANSACTION_ID>
+      <GENERATION_STAMP>1016</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>120488596</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>54</TRANSACTION_ID>
+      <PATH>/hard-lease-recovery-test</PATH>
+      <NUMBLOCKS>1</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-357061736603024522</BLOCK_ID>
+        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1016</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>2098840974</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>25</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>55</TRANSACTION_ID>
+      <PATH>/hard-lease-recovery-test</PATH>
+      <NUMBLOCKS>1</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-357061736603024522</BLOCK_ID>
+        <BLOCK_NUM_BYTES>0</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1016</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+    </DATA>
+    <CHECKSUM>-1794222801</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>10</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>56</TRANSACTION_ID>
+      <GENERATION_STAMP>1017</GENERATION_STAMP>
+    </DATA>
+    <CHECKSUM>-2123999915</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>22</OPCODE>
     <DATA>
-      <TRANSACTION_ID>34</TRANSACTION_ID>
-      <CLIENT_NAME>DFSClient_871171074</CLIENT_NAME>
-      <PATH>/reassign-lease-test</PATH>
+      <TRANSACTION_ID>57</TRANSACTION_ID>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-2143415023_1</CLIENT_NAME>
+      <PATH>/hard-lease-recovery-test</PATH>
       <CLIENT_NAME>HDFS_NameNode</CLIENT_NAME>
     </DATA>
-    <CHECKSUM>1975140107</CHECKSUM>
+    <CHECKSUM>-1841690515</CHECKSUM>
+  </RECORD>
+  <RECORD>
+    <OPCODE>9</OPCODE>
+    <DATA>
+      <TRANSACTION_ID>58</TRANSACTION_ID>
+      <PATH>/hard-lease-recovery-test</PATH>
+      <REPLICATION>1</REPLICATION>
+      <MTIME>1330405688726</MTIME>
+      <ATIME>1330405686084</ATIME>
+      <BLOCKSIZE>512</BLOCKSIZE>
+      <NUMBLOCKS>1</NUMBLOCKS>
+      <BLOCK>
+        <BLOCK_ID>-357061736603024522</BLOCK_ID>
+        <BLOCK_NUM_BYTES>11</BLOCK_NUM_BYTES>
+        <BLOCK_GENERATION_STAMP>1017</BLOCK_GENERATION_STAMP>
+      </BLOCK>
+      <PERMISSION_STATUS>
+        <USERNAME>todd</USERNAME>
+        <GROUPNAME>supergroup</GROUPNAME>
+        <FS_PERMISSIONS>420</FS_PERMISSIONS>
+      </PERMISSION_STATUS>
+    </DATA>
+    <CHECKSUM>-218102037</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>23</OPCODE>
     <DATA>
-      <TRANSACTION_ID>35</TRANSACTION_ID>
+      <TRANSACTION_ID>59</TRANSACTION_ID>
     </DATA>
-    <CHECKSUM>1975140107</CHECKSUM>
+    <CHECKSUM>-1616653774</CHECKSUM>
   </RECORD>
   <RECORD>
     <OPCODE>-1</OPCODE>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1.0-multiblock-file.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1.0-multiblock-file.tgz
new file mode 100644
index 0000000..8e327c2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1.0-multiblock-file.tgz
Binary files differ
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-policy.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-policy.xml
index 0f5310c..eb3f4bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-policy.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-policy.xml
@@ -109,5 +109,12 @@
     group list is separated by a blank. For e.g. "alice,bob users,wheel". 
     A special value of "*" means all users are allowed.</description>
   </property>
-
+  
+  <property>
+    <name>security.ha.service.protocol.acl</name>
+    <value>*</value>
+    <description>ACL for HAService protocol used by HAAdmin to manage the
+      active and stand-by states of namenode.</description>
+  </property>
+  
 </configuration>
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/scripts/saveVersion.sh b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/scripts/saveVersion.sh
index 11d7022..e644bbf 100755
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/scripts/saveVersion.sh
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/scripts/saveVersion.sh
@@ -34,7 +34,7 @@
   url="git://${hostname}${cwd}"
 elif [ -d .svn ]; then
   revision=`svn info ../ | sed -n -e 's/Last Changed Rev: \(.*\)/\1/p'`
-  url=`svn info ../ | sed -n -e 's/URL: \(.*\)/\1/p'`
+  url=`svn info ../ | sed -n -e 's/^URL: \(.*\)/\1/p'`
   # Get canonical branch (branches/X, tags/X, or trunk)
   branch=`echo $url | sed -n -e 's,.*\(branches/.*\)$,\1,p' \
                              -e 's,.*\(tags/.*\)$,\1,p' \
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm
new file mode 100644
index 0000000..c665067
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm
@@ -0,0 +1,434 @@
+~~ Licensed 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. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - High Availability
+  ---
+  ---
+  ${maven.build.timestamp}
+
+HDFS High Availability
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+* {Purpose}
+
+  This guide provides an overview of the HDFS High Availability (HA) feature and
+  how to configure and manage an HA HDFS cluster.
+ 
+  This document assumes that the reader has a general understanding of
+  general components and node types in an HDFS cluster. Please refer to the
+  HDFS Architecture guide for details.
+
+* {Background}
+
+  Prior to Hadoop 0.23.2, the NameNode was a single point of failure (SPOF) in
+  an HDFS cluster. Each cluster had a single NameNode, and if that machine or
+  process became unavailable, the cluster as a whole would be unavailable
+  until the NameNode was either restarted or brought up on a separate machine.
+  
+  This impacted the total availability of the HDFS cluster in two major ways:
+
+    * In the case of an unplanned event such as a machine crash, the cluster would
+      be unavailable until an operator restarted the NameNode.
+
+    * Planned maintenance events such as software or hardware upgrades on the
+      NameNode machine would result in windows of cluster downtime.
+  
+  The HDFS High Availability feature addresses the above problems by providing
+  the option of running two redundant NameNodes in the same cluster in an
+  Active/Passive configuration with a hot standby. This allows a fast failover to
+  a new NameNode in the case that a machine crashes, or a graceful
+  administrator-initiated failover for the purpose of planned maintenance.
+
+* {Architecture}
+
+  In a typical HA cluster, two separate machines are configured as NameNodes.
+  At any point in time, exactly one of the NameNodes is in an <Active> state,
+  and the other is in a <Standby> state. The Active NameNode is responsible
+  for all client operations in the cluster, while the Standby is simply acting
+  as a slave, maintaining enough state to provide a fast failover if
+  necessary.
+  
+  In order for the Standby node to keep its state synchronized with the Active
+  node, the current implementation requires that the two nodes both have access
+  to a directory on a shared storage device (eg an NFS mount from a NAS). This
+  restriction will likely be relaxed in future versions.
+
+  When any namespace modification is performed by the Active node, it durably
+  logs a record of the modification to an edit log file stored in the shared
+  directory.  The Standby node is constantly watching this directory for edits,
+  and as it sees the edits, it applies them to its own namespace. In the event of
+  a failover, the Standby will ensure that it has read all of the edits from the
+  shared storage before promoting itself to the Active state. This ensures that
+  the namespace state is fully synchronized before a failover occurs.
+  
+  In order to provide a fast failover, it is also necessary that the Standby node
+  have up-to-date information regarding the location of blocks in the cluster.
+  In order to achieve this, the DataNodes are configured with the location of
+  both NameNodes, and send block location information and heartbeats to both.
+  
+  It is vital for the correct operation of an HA cluster that only one of the
+  NameNodes be Active at a time. Otherwise, the namespace state would quickly
+  diverge between the two, risking data loss or other incorrect results.  In
+  order to ensure this property and prevent the so-called "split-brain scenario,"
+  the administrator must configure at least one <fencing method> for the shared
+  storage. During a failover, if it cannot be verified that the previous Active
+  node has relinquished its Active state, the fencing process is responsible for
+  cutting off the previous Active's access to the shared edits storage. This
+  prevents it from making any further edits to the namespace, allowing the new
+  Active to safely proceed with failover.
+
+  <<Note:>> Currently, only manual failover is supported. This means the HA
+  NameNodes are incapable of automatically detecting a failure of the Active
+  NameNode, and instead rely on the operator to manually initiate a failover.
+  Automatic failure detection and initiation of a failover will be implemented in
+  future versions.
+
+* {Hardware resources}
+
+  In order to deploy an HA cluster, you should prepare the following:
+
+    * <<NameNode machines>> - the machines on which you run the Active and
+    Standby NameNodes should have equivalent hardware to each other, and
+    equivalent hardware to what would be used in a non-HA cluster.
+
+    * <<Shared storage>> - you will need to have a shared directory which both
+    NameNode machines can have read/write access to. Typically this is a remote
+    filer which supports NFS and is mounted on each of the NameNode machines.
+    Currently only a single shared edits directory is supported. Thus, the
+    availability of the system is limited by the availability of this shared edits
+    directory, and therefore in order to remove all single points of failure there
+    needs to be redundancy for the shared edits directory. Specifically, multiple
+    network paths to the storage, and redundancy in the storage itself (disk,
+    network, and power). Beacuse of this, it is recommended that the shared storage
+    server be a high-quality dedicated NAS appliance rather than a simple Linux
+    server.
+  
+  Note that, in an HA cluster, the Standby NameNode also performs checkpoints of
+  the namespace state, and thus it is not necessary to run a Secondary NameNode,
+  CheckpointNode, or BackupNode in an HA cluster. In fact, to do so would be an
+  error. This also allows one who is reconfiguring a non-HA-enabled HDFS cluster
+  to be HA-enabled to reuse the hardware which they had previously dedicated to
+  the Secondary NameNode.
+
+* {Deployment}
+
+** Configuration overview
+
+  Similar to Federation configuration, HA configuration is backward compatible
+  and allows existing single NameNode configurations to work without change.
+  The new configuration is designed such that all the nodes in the cluster may
+  have the same configuration without the need for deploying different
+  configuration files to different machines based on the type of the node.
+ 
+  Like HDFS Federation, HA clusters reuse the <<<nameservice ID>>> to identify a
+  single HDFS instance that may in fact consist of multiple HA NameNodes. In
+  addition, a new abstraction called <<<NameNode ID>>> is added with HA. Each
+  distinct NameNode in the cluster has a different NameNode ID to distinguish it.
+  To support a single configuration file for all of the NameNodes, the relevant
+  configuration parameters are suffixed with the <<nameservice ID>> as well as
+  the <<NameNode ID>>.
+
+** Configuration details
+
+  To configure HA NameNodes, you must add several configuration options to your
+  <<hdfs-site.xml>> configuration file.
+
+  The order in which you set these configurations is unimportant, but the values
+  you choose for <<dfs.federation.nameservices>> and
+  <<dfs.ha.namenodes.[nameservice ID]>> will determine the keys of those that
+  follow. Thus, you should decide on these values before setting the rest of the
+  configuration options.
+
+  * <<dfs.federation.nameservices>> - the logical name for this new nameservice
+
+    Choose a logical name for this nameservice, for example "mycluster", and use
+    this logical name for the value of this config option. The name you choose is
+    arbitrary. It will be used both for configuration and as the authority
+    component of absolute HDFS paths in the cluster.
+
+    <<Note:>> If you are also using HDFS Federation, this configuration setting
+    should also include the list of other nameservices, HA or otherwise, as a
+    comma-separated list.
+
+----
+<property>
+  <name>dfs.federation.nameservices</name>
+  <value>mycluster</value>
+</property>
+----
+
+  * <<dfs.ha.namenodes.[nameservice ID]>> - unique identifiers for each NameNode in the nameservice
+
+    Configure with a list of comma-separated NameNode IDs. This will be used by
+    DataNodes to determine all the NameNodes in the cluster. For example, if you
+    used "mycluster" as the nameservice ID previously, and you wanted to use "nn1"
+    and "nn2" as the individual IDs of the NameNodes, you would configure this as
+    such:
+
+----
+<property>
+  <name>dfs.ha.namenodes.mycluster</name>
+  <value>nn1,nn2</value>
+</property>
+----
+
+    <<Note:>> Currently, only a maximum of two NameNodes may be configured per
+    nameservice.
+
+  * <<dfs.namenode.rpc-address.[nameservice ID].[name node ID]>> - the fully-qualified RPC address for each NameNode to listen on
+
+    For both of the previously-configured NameNode IDs, set the full address and
+    IPC port of the NameNode processs. Note that this results in two separate
+    configuration options. For example:
+
+----
+<property>
+  <name>dfs.namenode.rpc-address.mycluster.nn1</name>
+  <value>machine1.example.com:8020</value>
+</property>
+<property>
+  <name>dfs.namenode.rpc-address.mycluster.nn2</name>
+  <value>machine2.example.com:8020</value>
+</property>
+----
+
+    <<Note:>> You may similarly configure the "<<servicerpc-address>>" setting if
+    you so desire.
+
+  * <<dfs.namenode.http-address.[nameservice ID].[name node ID]>> - the fully-qualified HTTP address for each NameNode to listen on
+
+    Similarly to <rpc-address> above, set the addresses for both NameNodes' HTTP
+    servers to listen on. For example:
+
+----
+<property>
+  <name>dfs.namenode.http-address.mycluster.nn1</name>
+  <value>machine1.example.com:50070</value>
+</property>
+<property>
+  <name>dfs.namenode.http-address.mycluster.nn2</name>
+  <value>machine2.example.com:50070</value>
+</property>
+----
+
+    <<Note:>> If you have Hadoop's security features enabled, you should also set
+    the <https-address> similarly for each NameNode.
+
+  * <<dfs.namenode.shared.edits.dir>> - the location of the shared storage directory
+
+    This is where one configures the path to the remote shared edits directory
+    which the Standby NameNode uses to stay up-to-date with all the file system
+    changes the Active NameNode makes. <<You should only configure one of these
+    directories.>> This directory should be mounted r/w on both NameNode machines.
+    The value of this setting should be the absolute path to this directory on the
+    NameNode machines. For example:
+
+----
+<property>
+  <name>dfs.namenode.shared.edits.dir</name>
+  <value>file:///mnt/filer1/dfs/ha-name-dir-shared</value>
+</property>
+----
+
+  * <<dfs.client.failover.proxy.provider.[nameservice ID]>> - the Java class that HDFS clients use to contact the Active NameNode
+
+    Configure the name of the Java class which will be used by the DFS Client to
+    determine which NameNode is the current Active, and therefore which NameNode is
+    currently serving client requests. The only implementation which currently
+    ships with Hadoop is the <<ConfiguredFailoverProxyProvider>>, so use this
+    unless you are using a custom one. For example:
+
+----
+<property>
+  <name>dfs.client.failover.proxy.provider.mycluster</name>
+  <value>org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider</value>
+</property>
+----
+
+  * <<dfs.ha.fencing.methods>> - a list of scripts or Java classes which will be used to fence the Active NameNode during a failover
+
+    It is critical for correctness of the system that only one NameNode be in the
+    Active state at any given time. Thus, during a failover, we first ensure that
+    the Active NameNode is either in the Standby state, or the process has
+    terminated, before transitioning the other NameNode to the Active state. In
+    order to do this, you must configure at least one <<fencing method.>> These are
+    configured as a carriage-return-separated list, which will be attempted in order
+    until one indicates that fencing has succeeded. There are two methods which
+    ship with Hadoop: <shell> and <sshfence>. For information on implementing
+    your own custom fencing method, see the <org.apache.hadoop.ha.NodeFencer> class.
+
+    * <<sshfence>> - SSH to the Active NameNode and kill the process
+
+      The <sshfence> option SSHes to the target node and uses <fuser> to kill the
+      process listening on the service's TCP port. In order for this fencing option
+      to work, it must be able to SSH to the target node without providing a
+      passphrase. Thus, one must also configure the
+      <<dfs.ha.fencing.ssh.private-key-files>> option, which is a
+      comma-separated list of SSH private key files. For example:
+
+---
+<property>
+  <name>dfs.ha.fencing.methods</name>
+  <value>sshfence</value>
+</property>
+
+<property>
+  <name>dfs.ha.fencing.ssh.private-key-files</name>
+  <value>/home/exampleuser/.ssh/id_rsa</value>
+</property>
+---
+
+      Optionally, one may configure a non-standard username or port to perform the
+      SSH. One may also configure a timeout, in milliseconds, for the SSH, after
+      which this fencing method will be considered to have failed. It may be
+      configured like so:
+
+---
+<property>
+  <name>dfs.ha.fencing.methods</name>
+  <value>sshfence([[username][:port]])</value>
+</property>
+<property>
+  <name>dfs.ha.fencing.ssh.connect-timeout</name>
+  <value>
+</property>
+---
+
+    * <<shell>> - run an arbitrary shell command to fence the Active NameNode
+
+      The <shell> fencing method runs an arbitrary shell command. It may be
+      configured like so:
+
+---
+<property>
+  <name>dfs.ha.fencing.methods</name>
+  <value>shell(/path/to/my/script.sh arg1 arg2 ...)</value>
+</property>
+---
+
+      The string between '(' and ')' is passed directly to a bash shell and may not
+      include any closing parentheses.
+
+      When executed, the first argument to the configured script will be the address
+      of the NameNode to be fenced, followed by all arguments specified in the
+      configuration.
+
+      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).
+
+  * <<fs.defaultFS>> - the default path prefix used by the Hadoop FS client when none is given
+
+    Optionally, you may now configure the default path for Hadoop clients to use
+    the new HA-enabled logical URI. If you used "mycluster" as the nameservice ID
+    earlier, this will be the value of the authority portion of all of your HDFS
+    paths. This may be configured like so, in your <<core-site.xml>> file:
+
+---
+<property>
+  <name>fs.defaultFS</name>
+  <value>hdfs://mycluster</value>
+</property>
+---
+
+** Deployment details
+
+  After all of the necessary configuration options have been set, one must
+  initially synchronize the two HA NameNodes' on-disk metadata. If you are
+  setting up a fresh HDFS cluster, you should first run the format command (<hdfs
+  namenode -format>) on one of NameNodes. If you have already formatted the
+  NameNode, or are converting a non-HA-enabled cluster to be HA-enabled, you
+  should now copy over the contents of your NameNode metadata directories to
+  the other, unformatted NameNode using <scp> or a similar utility. The location
+  of the directories containing the NameNode metadata are configured via the
+  configuration options <<dfs.namenode.name.dir>> and/or
+  <<dfs.namenode.edits.dir>>. At this time, you should also ensure that the
+  shared edits dir (as configured by <<dfs.namenode.shared.edits.dir>>) includes
+  all recent edits files which are in your NameNode metadata directories.
+
+  At this point you may start both of your HA NameNodes as you normally would
+  start a NameNode.
+
+  You can visit each of the NameNodes' web pages separately by browsing to their
+  configured HTTP addresses. You should notice that next to the configured
+  address will be the HA state of the NameNode (either "standby" or "active".)
+  Whenever an HA NameNode starts, it is initially in the Standby state.
+
+** Administrative commands
+
+  Now that your HA NameNodes are configured and started, you will have access
+  to some additional commands to administer your HA HDFS cluster. Specifically,
+  you should familiarize yourself with all of the subcommands of the "<hdfs
+  haadmin>" command. Running this command without any additional arguments will
+  display the following usage information:
+
+---
+Usage: DFSHAAdmin [-ns <nameserviceId>]
+    [-transitionToActive <serviceId>]
+    [-transitionToStandby <serviceId>]
+    [-failover [--forcefence] [--forceactive] <serviceId> <serviceId>]
+    [-getServiceState <serviceId>]
+    [-checkHealth <serviceId>]
+    [-help <command>]
+---
+
+  This guide describes high-level uses of each of these subcommands. For
+  specific usage information of each subcommand, you should run "<hdfs haadmin
+  -help <command>>".
+
+  * <<transitionToActive>> and <<transitionToStandby>> - transition the state of the given NameNode to Active or Standby
+
+    These subcommands cause a given NameNode to transition to the Active or Standby
+    state, respectively. <<These commands do not attempt to perform any fencing,
+    and thus should rarely be used.>> Instead, one should almost always prefer to
+    use the "<hdfs haadmin -failover>" subcommand.
+
+  * <<failover>> - initiate a failover between two NameNodes
+
+    This subcommand causes a failover from the first provided NameNode to the
+    second. If the first NameNode is in the Standby state, this command simply
+    transitions the second to the Active state without error. If the first NameNode
+    is in the Active state, an attempt will be made to gracefully transition it to
+    the Standby state. If this fails, the fencing methods (as configured by
+    <<dfs.ha.fencing.methods>>) will be attempted in order until one
+    succeeds. Only after this process will the second NameNode be transitioned to
+    the Active state. If no fencing method succeeds, the second NameNode will not
+    be transitioned to the Active state, and an error will be returned.
+
+  * <<getServiceState>> - determine whether the given NameNode is Active or Standby
+
+    Connect to the provided NameNode to determine its current state, printing
+    either "standby" or "active" to STDOUT appropriately. This subcommand might be
+    used by cron jobs or monitoring scripts which need to behave differently based
+    on whether the NameNode is currently Active or Standby.
+
+  * <<checkHealth>> - check the health of the given NameNode
+
+    Connect to the provided NameNode to check its health. The NameNode is capable
+    of performing some diagnostics on itself, including checking if internal
+    services are running as expected. This command will return 0 if the NameNode is
+    healthy, non-zero otherwise. One might use this command for monitoring
+    purposes.
+
+    <<Note:>> This is not yet implemented, and at present will always return
+    success, unless the given NameNode is completely down.
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 93e9742..1cf5a72 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -593,6 +593,11 @@
         <version>${commons-daemon.version}</version>
       </dependency>
       <dependency>
+        <groupId>com.jcraft</groupId>
+        <artifactId>jsch</artifactId>
+        <version>0.1.42</version>
+      </dependency>
+      <dependency>
         <groupId>org.jdom</groupId>
         <artifactId>jdom</artifactId>
         <version>1.1</version>
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 27f9b7b..f992a17 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -53,6 +53,7 @@
     </menu>
     
     <menu name="HDFS" inherit="top">
+      <item name="High Availability" href="hadoop-yarn/hadoop-yarn-site/HDFSHighAvailability.html"/>
       <item name="Federation" href="hadoop-yarn/hadoop-yarn-site/Federation.html"/>
       <item name="WebHDFS REST API" href="hadoop-yarn/hadoop-yarn-site/WebHDFS.html"/>
       <item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/RetriableCommand.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/RetriableCommand.java
index 1d248f0..563372e 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/RetriableCommand.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/RetriableCommand.java
@@ -22,7 +22,9 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
 import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.util.ThreadUtil;
 
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
@@ -80,7 +82,7 @@
   public Object execute(Object... arguments) throws Exception {
     Exception latestException;
     int counter = 0;
-    do {
+    while (true) {
       try {
         return doExecute(arguments);
       } catch(Exception exception) {
@@ -88,7 +90,13 @@
         latestException = exception;
       }
       counter++;
-    } while (retryPolicy.shouldRetry(latestException, counter, 0, true).equals(RetryPolicy.RetryAction.RETRY));
+      RetryAction action = retryPolicy.shouldRetry(latestException, counter, 0, true);
+      if (action.action == RetryPolicy.RetryAction.RetryDecision.RETRY) {
+        ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis);
+      } else {
+        break;
+      }
+    }
 
     throw new IOException("Couldn't run retriable-command: " + description,
                           latestException);
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
index e5ab059..5ba5eb8 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyMapper.java
@@ -545,7 +545,12 @@
             Assert.fail("Didn't expect the file to be copied");
           } catch (AccessControlException ignore) {
           } catch (Exception e) {
-            if (e.getCause() == null || !(e.getCause() instanceof AccessControlException)) {
+            // We want to make sure the underlying cause of the exception is
+            // due to permissions error. The exception we're interested in is
+            // wrapped twice - once in RetriableCommand and again in CopyMapper
+            // itself.
+            if (e.getCause() == null || e.getCause().getCause() == null ||
+                !(e.getCause().getCause() instanceof AccessControlException)) {
               throw new RuntimeException(e);
             }
           }