You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2015/06/24 19:50:39 UTC

[01/21] hadoop git commit: YARN-3834. Scrub debug logging of tokens during resource localization. Contributed by Chris Nauroth

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 f08bf361d -> 845a71063


YARN-3834. Scrub debug logging of tokens during resource localization. Contributed by Chris Nauroth


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6c7a9d50
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6c7a9d50
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6c7a9d50

Branch: refs/heads/HDFS-7240
Commit: 6c7a9d502a633b5aca75c9798f19ce4a5729014e
Parents: c7d022b
Author: Xuan <xg...@apache.org>
Authored: Sun Jun 21 17:13:44 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Sun Jun 21 17:13:44 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../localizer/ResourceLocalizationService.java  | 29 +++++++++++++++++++-
 .../TestResourceLocalizationService.java        | 10 +++++--
 3 files changed, 39 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c7a9d50/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d89c285..b50f490 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -312,6 +312,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3148. Allow CORS related headers to passthrough in WebAppProxyServlet.
     (Varun Saxena via devaraj)
 
+    YARN-3834. Scrub debug logging of tokens during resource localization.
+    (Chris Nauroth via xgong)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c7a9d50/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index 54c31c2..d6e0903 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -51,6 +51,7 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -1208,7 +1209,7 @@ public class ResourceLocalizationService extends CompositeService
         if (LOG.isDebugEnabled()) {
           for (Token<? extends TokenIdentifier> tk : credentials
               .getAllTokens()) {
-            LOG.debug(tk.getService() + " : " + tk.encodeToUrlString());
+            LOG.debug(tk + " : " + buildTokenFingerprint(tk));
           }
         }
         if (UserGroupInformation.isSecurityEnabled()) {
@@ -1228,6 +1229,32 @@ public class ResourceLocalizationService extends CompositeService
 
   }
 
+  /**
+   * Returns a fingerprint of a token.  The fingerprint is suitable for use in
+   * logging, because it cannot be used to determine the secret.  The
+   * fingerprint is built using the first 10 bytes of a SHA-256 hash of the
+   * string encoding of the token.  The returned string contains the hex
+   * representation of each byte, delimited by a space.
+   *
+   * @param tk token
+   * @return token fingerprint
+   * @throws IOException if there is an I/O error
+   */
+  @VisibleForTesting
+  static String buildTokenFingerprint(Token<? extends TokenIdentifier> tk)
+      throws IOException {
+    char[] digest = DigestUtils.sha256Hex(tk.encodeToUrlString()).toCharArray();
+    StringBuilder fingerprint = new StringBuilder();
+    for (int i = 0; i < 10; ++i) {
+      if (i > 0) {
+        fingerprint.append(' ');
+      }
+      fingerprint.append(digest[2 * i]);
+      fingerprint.append(digest[2 * i + 1]);
+    }
+    return fingerprint.toString();
+  }
+
   static class CacheCleanup extends Thread {
 
     private final Dispatcher dispatcher;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c7a9d50/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index a02b2b0..c515506 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -2035,7 +2035,7 @@ public class TestResourceLocalizationService {
   }
 
   private static Container getMockContainer(ApplicationId appId, int id,
-      String user) {
+      String user) throws IOException {
     Container c = mock(Container.class);
     ApplicationAttemptId appAttemptId =
         BuilderUtils.newApplicationAttemptId(appId, 1);
@@ -2043,7 +2043,13 @@ public class TestResourceLocalizationService {
     when(c.getUser()).thenReturn(user);
     when(c.getContainerId()).thenReturn(cId);
     Credentials creds = new Credentials();
-    creds.addToken(new Text("tok" + id), getToken(id));
+    Token<? extends TokenIdentifier> tk = getToken(id);
+    String fingerprint = ResourceLocalizationService.buildTokenFingerprint(tk);
+    assertNotNull(fingerprint);
+    assertTrue(
+        "Expected token fingerprint of 10 hex bytes delimited by space.",
+        fingerprint.matches("^(([0-9a-f]){2} ){9}([0-9a-f]){2}$"));
+    creds.addToken(new Text("tok" + id), tk);
     when(c.getCredentials()).thenReturn(creds);
     when(c.toString()).thenReturn(cId.toString());
     return c;


[14/21] hadoop git commit: HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
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 fdbacdc..0a21886 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
@@ -62,6 +62,8 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -445,7 +447,7 @@ public class MiniDFSCluster {
     final int numNameNodes = builder.nnTopology.countNameNodes();
     LOG.info("starting cluster: numNameNodes=" + numNameNodes
         + ", numDataNodes=" + builder.numDataNodes);
-    nameNodes = new NameNodeInfo[numNameNodes];
+
     this.storagesPerDatanode = builder.storagesPerDatanode;
 
     // Duplicate the storageType setting for each DN.
@@ -515,7 +517,7 @@ public class MiniDFSCluster {
   }
 
   private Configuration conf;
-  private NameNodeInfo[] nameNodes;
+  private Multimap<String, NameNodeInfo> namenodes = ArrayListMultimap.create();
   protected int numDataNodes;
   protected final List<DataNodeProperties> dataNodes =
                          new ArrayList<DataNodeProperties>();
@@ -539,10 +541,10 @@ public class MiniDFSCluster {
    * Stores the information related to a namenode in the cluster
    */
   public static class NameNodeInfo {
-    final NameNode nameNode;
-    final Configuration conf;
-    final String nameserviceId;
-    final String nnId;
+    public NameNode nameNode;
+    Configuration conf;
+    String nameserviceId;
+    String nnId;
     StartupOption startOpt;
     NameNodeInfo(NameNode nn, String nameserviceId, String nnId,
         StartupOption startOpt, Configuration conf) {
@@ -563,7 +565,6 @@ public class MiniDFSCluster {
    * without a name node (ie when the name node is started elsewhere).
    */
   public MiniDFSCluster() {
-    nameNodes = new NameNodeInfo[0]; // No namenode in the cluster
     storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     synchronized (MiniDFSCluster.class) {
       instanceId = instanceCount++;
@@ -740,7 +741,6 @@ public class MiniDFSCluster {
                         StartupOption operation,
                         String[] racks, String hosts[],
                         long[] simulatedCapacities) throws IOException {
-    this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
     this.storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     initMiniDFSCluster(conf, numDataNodes, null, format,
                        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
@@ -814,7 +814,7 @@ public class MiniDFSCluster {
         createNameNodesAndSetConf(
             nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
             enableManagedDfsDirsRedundancy,
-            format, startOpt, clusterId, conf);
+            format, startOpt, clusterId);
       } catch (IOException ioe) {
         LOG.error("IOE creating namenodes. Permissions dump:\n" +
             createPermissionsDiagnosisString(data_dir), ioe);
@@ -871,7 +871,127 @@ public class MiniDFSCluster {
   private void createNameNodesAndSetConf(MiniDFSNNTopology nnTopology,
       boolean manageNameDfsDirs, boolean manageNameDfsSharedDirs,
       boolean enableManagedDfsDirsRedundancy, boolean format,
+      StartupOption operation, String clusterId) throws IOException {
+    // do the basic namenode configuration
+    configureNameNodes(nnTopology, federation, conf);
+
+    int nnCounter = 0;
+    int nsCounter = 0;
+    // configure each NS independently
+    for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
+      configureNameService(nameservice, nsCounter++, manageNameDfsSharedDirs,
+          manageNameDfsDirs, enableManagedDfsDirsRedundancy,
+          format, operation, clusterId, nnCounter);
+      nnCounter += nameservice.getNNs().size();
+    }
+  }
+
+  /**
+   * Do the rest of the NN configuration for things like shared edits,
+   * as well as directory formatting, etc. for a single nameservice
+   * @param nnCounter the count of the number of namenodes already configured/started. Also,
+   *                  acts as the <i>index</i> to the next NN to start (since indicies start at 0).
+   * @throws IOException
+   */
+  private void configureNameService(MiniDFSNNTopology.NSConf nameservice, int nsCounter,
+      boolean manageNameDfsSharedDirs, boolean manageNameDfsDirs, boolean
+      enableManagedDfsDirsRedundancy, boolean format,
       StartupOption operation, String clusterId,
+      final int nnCounter) throws IOException{
+    String nsId = nameservice.getId();
+    String lastDefaultFileSystem = null;
+
+    // If HA is enabled on this nameservice, enumerate all the namenodes
+    // in the configuration. Also need to set a shared edits dir
+    int numNNs = nameservice.getNNs().size();
+    if (numNNs > 1 && manageNameDfsSharedDirs) {
+      URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter + numNNs - 1);
+      conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
+      // Clean out the shared edits dir completely, including all subdirectories.
+      FileUtil.fullyDelete(new File(sharedEditsUri));
+    }
+
+    // Now format first NN and copy the storage directory from that node to the others.
+    int nnIndex = nnCounter;
+    Collection<URI> prevNNDirs = null;
+    for (NNConf nn : nameservice.getNNs()) {
+      initNameNodeConf(conf, nsId, nsCounter, nn.getNnId(), manageNameDfsDirs,
+          manageNameDfsDirs,  nnIndex);
+      Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
+      if (format) {
+        // delete the existing namespaces
+        for (URI nameDirUri : namespaceDirs) {
+          File nameDir = new File(nameDirUri);
+          if (nameDir.exists() && !FileUtil.fullyDelete(nameDir)) {
+            throw new IOException("Could not fully delete " + nameDir);
+          }
+        }
+
+        // delete the checkpoint directories, if they exist
+        Collection<URI> checkpointDirs = Util.stringCollectionAsURIs(conf
+            .getTrimmedStringCollection(DFS_NAMENODE_CHECKPOINT_DIR_KEY));
+        for (URI checkpointDirUri : checkpointDirs) {
+          File checkpointDir = new File(checkpointDirUri);
+          if (checkpointDir.exists() && !FileUtil.fullyDelete(checkpointDir)) {
+            throw new IOException("Could not fully delete " + checkpointDir);
+          }
+        }
+      }
+
+      boolean formatThisOne = format;
+      // if we are looking at not the first NN
+      if (nnIndex++ > nnCounter && format) {
+        // Don't format the second, third, etc 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 previous one.
+        formatThisOne = false;
+        assert (null != prevNNDirs);
+        copyNameDirs(prevNNDirs, namespaceDirs, conf);
+      }
+
+      if (formatThisOne) {
+        // Allow overriding clusterID for specific NNs to test
+        // misconfiguration.
+        if (nn.getClusterId() == null) {
+          StartupOption.FORMAT.setClusterId(clusterId);
+        } else {
+          StartupOption.FORMAT.setClusterId(nn.getClusterId());
+        }
+        DFSTestUtil.formatNameNode(conf);
+      }
+      prevNNDirs = namespaceDirs;
+    }
+
+    // create all the namenodes in the namespace
+    nnIndex = nnCounter;
+    for (NNConf nn : nameservice.getNNs()) {
+      initNameNodeConf(conf, nsId, nsCounter, nn.getNnId(), manageNameDfsDirs,
+          enableManagedDfsDirsRedundancy, nnIndex++);
+      NameNodeInfo info = createNameNode(conf, false, operation,
+          clusterId, nsId, nn.getNnId());
+
+      // Record the last namenode uri
+      if (info != null && info.conf != null) {
+        lastDefaultFileSystem =
+            info.conf.get(FS_DEFAULT_NAME_KEY);
+      }
+    }
+    if (!federation && lastDefaultFileSystem != null) {
+      // Set the default file system to the actual bind address of NN.
+      conf.set(FS_DEFAULT_NAME_KEY, lastDefaultFileSystem);
+    }
+  }
+
+  /**
+   * Do the basic NN configuration for the topology. Does not configure things like the shared
+   * edits directories
+   * @param nnTopology
+   * @param federation
+   * @param conf
+   * @throws IOException
+   */
+  public static void configureNameNodes(MiniDFSNNTopology nnTopology, boolean federation,
       Configuration conf) throws IOException {
     Preconditions.checkArgument(nnTopology.countNameNodes() > 0,
         "empty NN topology: no namenodes specified!");
@@ -884,22 +1004,21 @@ public class MiniDFSCluster {
       // NN is started.
       conf.set(FS_DEFAULT_NAME_KEY, "hdfs://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());
       }
     }
+
     if (!allNsIds.isEmpty()) {
       conf.set(DFS_NAMESERVICES, Joiner.on(",").join(allNsIds));
     }
-    
-    int nnCounter = 0;
+
     for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
       String nsId = nameservice.getId();
-      String lastDefaultFileSystem = null;
-      
+
       Preconditions.checkArgument(
           !federation || nsId != null,
           "if there is more than one NS, they must have names");
@@ -918,85 +1037,10 @@ public class MiniDFSCluster {
       // 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 (manageNameDfsSharedDirs) {
-          URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter+nnIds.size()-1); 
-          conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
-          // Clean out the shared edits dir completely, including all subdirectories.
-          FileUtil.fullyDelete(new File(sharedEditsUri));
-        }
-      }
-
-      // 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, 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);
-            }
-          }
-          Collection<URI> checkpointDirs = Util.stringCollectionAsURIs(conf
-              .getTrimmedStringCollection(DFS_NAMENODE_CHECKPOINT_DIR_KEY));
-          for (URI checkpointDirUri : checkpointDirs) {
-            File checkpointDir = new File(checkpointDirUri);
-            if (checkpointDir.exists() && !FileUtil.fullyDelete(checkpointDir)) {
-              throw new IOException("Could not fully delete " + checkpointDir);
-            }
-          }
-        }
-        
-        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) {
-          // Allow overriding clusterID for specific NNs to test
-          // misconfiguration.
-          if (nn.getClusterId() == null) {
-            StartupOption.FORMAT.setClusterId(clusterId);
-          } else {
-            StartupOption.FORMAT.setClusterId(nn.getClusterId());
-          }
-          DFSTestUtil.formatNameNode(conf);
-        }
-        prevNNDirs = namespaceDirs;
-      }
-
-      // Start all Namenodes
-      for (NNConf nn : nameservice.getNNs()) {
-        initNameNodeConf(conf, nsId, nn.getNnId(), manageNameDfsDirs,
-            enableManagedDfsDirsRedundancy, nnCounter);
-        createNameNode(nnCounter, conf, numDataNodes, false, operation,
-            clusterId, nsId, nn.getNnId());
-        // Record the last namenode uri
-        if (nameNodes[nnCounter] != null && nameNodes[nnCounter].conf != null) {
-          lastDefaultFileSystem =
-              nameNodes[nnCounter].conf.get(FS_DEFAULT_NAME_KEY);
-        }
-        nnCounter++;
-      }
-      if (!federation && lastDefaultFileSystem != null) {
-        // Set the default file system to the actual bind address of NN.
-        conf.set(FS_DEFAULT_NAME_KEY, lastDefaultFileSystem);
+        conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, nameservice.getId()), Joiner
+            .on(",").join(nnIds));
       }
     }
-
   }
   
   public URI getSharedEditsDir(int minNN, int maxNN) throws IOException {
@@ -1010,39 +1054,92 @@ public class MiniDFSCluster {
   }
   
   public NameNodeInfo[] getNameNodeInfos() {
-    return this.nameNodes;
+    return this.namenodes.values().toArray(new NameNodeInfo[0]);
   }
 
-  private void initNameNodeConf(Configuration conf,
-      String nameserviceId, String nnId,
-      boolean manageNameDfsDirs, boolean enableManagedDfsDirsRedundancy,
-      int nnIndex) throws IOException {
+  /**
+   * @param nsIndex index of the namespace id to check
+   * @return all the namenodes bound to the given namespace index
+   */
+  public NameNodeInfo[] getNameNodeInfos(int nsIndex) {
+    int i = 0;
+    for (String ns : this.namenodes.keys()) {
+      if (i++ == nsIndex) {
+        return this.namenodes.get(ns).toArray(new NameNodeInfo[0]);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * @param nameservice id of nameservice to read
+   * @return all the namenodes bound to the given namespace index
+   */
+  public NameNodeInfo[] getNameNodeInfos(String nameservice) {
+    for (String ns : this.namenodes.keys()) {
+      if (nameservice.equals(ns)) {
+        return this.namenodes.get(ns).toArray(new NameNodeInfo[0]);
+      }
+    }
+    return null;
+  }
+
+
+  private void initNameNodeConf(Configuration conf, String nameserviceId, int nsIndex, String nnId,
+      boolean manageNameDfsDirs, boolean enableManagedDfsDirsRedundancy, int nnIndex)
+      throws IOException {
     if (nameserviceId != null) {
       conf.set(DFS_NAMESERVICE_ID, nameserviceId);
     }
     if (nnId != null) {
       conf.set(DFS_HA_NAMENODE_ID_KEY, nnId);
     }
-    
     if (manageNameDfsDirs) {
       if (enableManagedDfsDirsRedundancy) {
-        conf.set(DFS_NAMENODE_NAME_DIR_KEY,
-            fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
-            fileAsURI(new File(base_dir, "name" + (2*nnIndex + 2))));
-        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
-            fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
-            fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
+        File[] files = getNameNodeDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_NAME_DIR_KEY, fileAsURI(files[0]) + "," + fileAsURI(files[1]));
+        files = getCheckpointDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY, fileAsURI(files[0]) + "," + fileAsURI(files[1]));
       } else {
-        conf.set(DFS_NAMENODE_NAME_DIR_KEY,
-            fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1))).
-              toString());
-        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
-            fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1))).
-              toString());
+        File[] files = getNameNodeDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_NAME_DIR_KEY, fileAsURI(files[0]).toString());
+        files = getCheckpointDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY, fileAsURI(files[0]).toString());
       }
     }
   }
 
+  private File[] getNameNodeDirectory(int nameserviceIndex, int nnIndex) {
+    return getNameNodeDirectory(base_dir, nameserviceIndex, nnIndex);
+  }
+
+  public static File[] getNameNodeDirectory(String base_dir, int nsIndex, int nnIndex) {
+    return getNameNodeDirectory(new File(base_dir), nsIndex, nnIndex);
+  }
+
+  public static File[] getNameNodeDirectory(File base_dir, int nsIndex, int nnIndex) {
+    File[] files = new File[2];
+    files[0] = new File(base_dir, "name-" + nsIndex + "-" + (2 * nnIndex + 1));
+    files[1] = new File(base_dir, "name-" + nsIndex + "-" + (2 * nnIndex + 2));
+    return files;
+  }
+
+  public File[] getCheckpointDirectory(int nsIndex, int nnIndex) {
+    return getCheckpointDirectory(base_dir, nsIndex, nnIndex);
+  }
+
+  public static File[] getCheckpointDirectory(String base_dir, int nsIndex, int nnIndex) {
+    return getCheckpointDirectory(new File(base_dir), nsIndex, nnIndex);
+  }
+
+  public static File[] getCheckpointDirectory(File base_dir, int nsIndex, int nnIndex) {
+    File[] files = new File[2];
+    files[0] = new File(base_dir, "namesecondary-" + nsIndex + "-" + (2 * nnIndex + 1));
+    files[1] = new File(base_dir, "namesecondary-" + nsIndex + "-" + (2 * nnIndex + 2));
+    return files;
+  }
+
+
   public static void copyNameDirs(Collection<URI> srcDirs, Collection<URI> dstDirs,
       Configuration dstConf) throws IOException {
     URI srcDir = Lists.newArrayList(srcDirs).get(0);
@@ -1094,12 +1191,9 @@ public class MiniDFSCluster {
             new String[] {} : new String[] {operation.getName()};
     return args;
   }
-  
-  private void createNameNode(int nnIndex, Configuration conf,
-      int numDataNodes, boolean format, StartupOption operation,
-      String clusterId, String nameserviceId,
-      String nnId)
-      throws IOException {
+
+  private NameNodeInfo createNameNode(Configuration conf, boolean format, StartupOption operation,
+      String clusterId, String nameserviceId, String nnId) throws IOException {
     // Format and clean out DataNode directories
     if (format) {
       DFSTestUtil.formatNameNode(conf);
@@ -1113,7 +1207,7 @@ public class MiniDFSCluster {
     String[] args = createArgs(operation);
     NameNode nn =  NameNode.createNameNode(args, conf);
     if (operation == StartupOption.RECOVER) {
-      return;
+      return null;
     }
     
     // After the NN has started, set back the bound ports into
@@ -1131,14 +1225,17 @@ public class MiniDFSCluster {
 
     DFSUtil.setGenericConf(conf, nameserviceId, nnId,
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
+    NameNodeInfo info = new NameNodeInfo(nn, nameserviceId, nnId,
         operation, new Configuration(conf));
+    namenodes.put(nameserviceId, info);
+
     // Restore the default fs name
     if (originalDefaultFs == null) {
       conf.set(FS_DEFAULT_NAME_KEY, "");
     } else {
       conf.set(FS_DEFAULT_NAME_KEY, originalDefaultFs);
     }
+    return info;
   }
 
   /**
@@ -1154,7 +1251,7 @@ public class MiniDFSCluster {
    */
   public URI getURI(int nnIndex) {
     String hostPort =
-        nameNodes[nnIndex].nameNode.getNameNodeAddressHostPortString();
+        getNN(nnIndex).nameNode.getNameNodeAddressHostPortString();
     URI uri = null;
     try {
       uri = new URI("hdfs://" + hostPort);
@@ -1172,9 +1269,21 @@ public class MiniDFSCluster {
    * @return Configuration of for the given namenode
    */
   public Configuration getConfiguration(int nnIndex) {
-    return nameNodes[nnIndex].conf;
+    return getNN(nnIndex).conf;
   }
 
+  private NameNodeInfo getNN(int nnIndex) {
+    int count = 0;
+    for (NameNodeInfo nn : namenodes.values()) {
+      if (count == nnIndex) {
+        return nn;
+      }
+      count++;
+    }
+    return null;
+  }
+
+
   /**
    * wait for the given namenode to get out of safemode.
    */
@@ -1593,7 +1702,7 @@ public class MiniDFSCluster {
    * @throws Exception
    */
   public void finalizeCluster(int nnIndex, Configuration conf) throws Exception {
-    finalizeNamenode(nameNodes[nnIndex].nameNode, nameNodes[nnIndex].conf);
+    finalizeNamenode(getNN(nnIndex).nameNode, getNN(nnIndex).conf);
   }
 
   /**
@@ -1604,7 +1713,7 @@ public class MiniDFSCluster {
    * @throws IllegalStateException if the Namenode is not running.
    */
   public void finalizeCluster(Configuration conf) throws Exception {
-    for (NameNodeInfo nnInfo : nameNodes) {
+    for (NameNodeInfo nnInfo : namenodes.values()) {
       if (nnInfo == null) {
         throw new IllegalStateException("Attempting to finalize "
             + "Namenode but it is not running");
@@ -1612,9 +1721,9 @@ public class MiniDFSCluster {
       finalizeNamenode(nnInfo.nameNode, nnInfo.conf);
     }
   }
-  
+
   public int getNumNameNodes() {
-    return nameNodes.length;
+    return namenodes.size();
   }
   
   /**
@@ -1644,7 +1753,7 @@ public class MiniDFSCluster {
    * Gets the NameNode for the index.  May be null.
    */
   public NameNode getNameNode(int nnIndex) {
-    return nameNodes[nnIndex].nameNode;
+    return getNN(nnIndex).nameNode;
   }
   
   /**
@@ -1653,11 +1762,11 @@ public class MiniDFSCluster {
    */
   public FSNamesystem getNamesystem() {
     checkSingleNameNode();
-    return NameNodeAdapter.getNamesystem(nameNodes[0].nameNode);
+    return NameNodeAdapter.getNamesystem(getNN(0).nameNode);
   }
-  
+
   public FSNamesystem getNamesystem(int nnIndex) {
-    return NameNodeAdapter.getNamesystem(nameNodes[nnIndex].nameNode);
+    return NameNodeAdapter.getNamesystem(getNN(nnIndex).nameNode);
   }
 
   /**
@@ -1697,14 +1806,14 @@ public class MiniDFSCluster {
    * caller supplied port is not necessarily the actual port used.
    */     
   public int getNameNodePort(int nnIndex) {
-    return nameNodes[nnIndex].nameNode.getNameNodeAddress().getPort();
+    return getNN(nnIndex).nameNode.getNameNodeAddress().getPort();
   }
 
   /**
    * @return the service rpc port used by the NameNode at the given index.
    */     
   public int getNameNodeServicePort(int nnIndex) {
-    return nameNodes[nnIndex].nameNode.getServiceRpcAddress().getPort();
+    return getNN(nnIndex).nameNode.getServiceRpcAddress().getPort();
   }
     
   /**
@@ -1745,7 +1854,7 @@ public class MiniDFSCluster {
       fileSystems.clear();
     }
     shutdownDataNodes();
-    for (NameNodeInfo nnInfo : nameNodes) {
+    for (NameNodeInfo nnInfo : namenodes.values()) {
       if (nnInfo == null) continue;
       NameNode nameNode = nnInfo.nameNode;
       if (nameNode != null) {
@@ -1781,7 +1890,7 @@ public class MiniDFSCluster {
    * Shutdown all the namenodes.
    */
   public synchronized void shutdownNameNodes() {
-    for (int i = 0; i < nameNodes.length; i++) {
+    for (int i = 0; i < namenodes.size(); i++) {
       shutdownNameNode(i);
     }
   }
@@ -1790,13 +1899,15 @@ public class MiniDFSCluster {
    * Shutdown the namenode at a given index.
    */
   public synchronized void shutdownNameNode(int nnIndex) {
-    NameNode nn = nameNodes[nnIndex].nameNode;
+    NameNodeInfo info = getNN(nnIndex);
+    NameNode nn = info.nameNode;
     if (nn != null) {
       LOG.info("Shutting down the namenode");
       nn.stop();
       nn.join();
-      Configuration conf = nameNodes[nnIndex].conf;
-      nameNodes[nnIndex] = new NameNodeInfo(null, null, null, null, conf);
+      info.nnId = null;
+      info.nameNode = null;
+      info.nameserviceId = null;
     }
   }
   
@@ -1804,7 +1915,7 @@ public class MiniDFSCluster {
    * Restart all namenodes.
    */
   public synchronized void restartNameNodes() throws IOException {
-    for (int i = 0; i < nameNodes.length; i++) {
+    for (int i = 0; i < namenodes.size(); i++) {
       restartNameNode(i, false);
     }
     waitActive();
@@ -1840,19 +1951,19 @@ public class MiniDFSCluster {
    */
   public synchronized void restartNameNode(int nnIndex, boolean waitActive,
       String... args) throws IOException {
-    String nameserviceId = nameNodes[nnIndex].nameserviceId;
-    String nnId = nameNodes[nnIndex].nnId;
-    StartupOption startOpt = nameNodes[nnIndex].startOpt;
-    Configuration conf = nameNodes[nnIndex].conf;
+    NameNodeInfo info = getNN(nnIndex);
+    StartupOption startOpt = info.startOpt;
+
     shutdownNameNode(nnIndex);
     if (args.length != 0) {
       startOpt = null;
     } else {
       args = createArgs(startOpt);
     }
-    NameNode nn = NameNode.createNameNode(args, conf);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId, startOpt,
-        conf);
+
+    NameNode nn = NameNode.createNameNode(args, info.conf);
+    info.nameNode = nn;
+    info.setStartOpt(startOpt);
     if (waitActive) {
       waitClusterUp();
       LOG.info("Restarted the namenode");
@@ -2124,7 +2235,7 @@ public class MiniDFSCluster {
    * or if waiting for safe mode is disabled.
    */
   public boolean isNameNodeUp(int nnIndex) {
-    NameNode nameNode = nameNodes[nnIndex].nameNode;
+    NameNode nameNode = getNN(nnIndex).nameNode;
     if (nameNode == null) {
       return false;
     }
@@ -2142,7 +2253,7 @@ public class MiniDFSCluster {
    * Returns true if all the NameNodes are running and is out of Safe Mode.
    */
   public boolean isClusterUp() {
-    for (int index = 0; index < nameNodes.length; index++) {
+    for (int index = 0; index < namenodes.size(); index++) {
       if (!isNameNodeUp(index)) {
         return false;
       }
@@ -2172,15 +2283,13 @@ public class MiniDFSCluster {
     checkSingleNameNode();
     return getFileSystem(0);
   }
-  
+
   /**
    * Get a client handle to the DFS cluster for the namenode at given index.
    */
   public DistributedFileSystem getFileSystem(int nnIndex) throws IOException {
-    DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(
-        getURI(nnIndex), nameNodes[nnIndex].conf);
-    fileSystems.add(dfs);
-    return dfs;
+    return (DistributedFileSystem) addFileSystem(FileSystem.get(getURI(nnIndex),
+        getNN(nnIndex).conf));
   }
 
   /**
@@ -2188,17 +2297,20 @@ public class MiniDFSCluster {
    * This simulating different threads working on different FileSystem instances.
    */
   public FileSystem getNewFileSystemInstance(int nnIndex) throws IOException {
-    FileSystem dfs = FileSystem.newInstance(getURI(nnIndex), nameNodes[nnIndex].conf);
-    fileSystems.add(dfs);
-    return dfs;
+    return addFileSystem(FileSystem.newInstance(getURI(nnIndex), getNN(nnIndex).conf));
   }
-  
+
+  private <T extends FileSystem> T addFileSystem(T fs) {
+    fileSystems.add(fs);
+    return fs;
+  }
+
   /**
    * @return a http URL
    */
   public String getHttpUri(int nnIndex) {
     return "http://"
-        + nameNodes[nnIndex].conf
+        + getNN(nnIndex).conf
             .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
   }
 
@@ -2206,14 +2318,14 @@ public class MiniDFSCluster {
    * Get the directories where the namenode stores its image.
    */
   public Collection<URI> getNameDirs(int nnIndex) {
-    return FSNamesystem.getNamespaceDirs(nameNodes[nnIndex].conf);
+    return FSNamesystem.getNamespaceDirs(getNN(nnIndex).conf);
   }
 
   /**
    * Get the directories where the namenode stores its edits.
    */
   public Collection<URI> getNameEditsDirs(int nnIndex) throws IOException {
-    return FSNamesystem.getNamespaceEditsDirs(nameNodes[nnIndex].conf);
+    return FSNamesystem.getNamespaceEditsDirs(getNN(nnIndex).conf);
   }
   
   public void transitionToActive(int nnIndex) throws IOException,
@@ -2254,11 +2366,12 @@ public class MiniDFSCluster {
 
   /** Wait until the given namenode gets registration from all the datanodes */
   public void waitActive(int nnIndex) throws IOException {
-    if (nameNodes.length == 0 || nameNodes[nnIndex] == null
-        || nameNodes[nnIndex].nameNode == null) {
+    if (namenodes.size() == 0 || getNN(nnIndex) == null || getNN(nnIndex).nameNode == null) {
       return;
     }
-    InetSocketAddress addr = nameNodes[nnIndex].nameNode.getServiceRpcAddress();
+
+    NameNodeInfo info = getNN(nnIndex);
+    InetSocketAddress addr = info.nameNode.getServiceRpcAddress();
     assert addr.getPort() != 0;
     DFSClient client = new DFSClient(addr, conf);
 
@@ -2278,7 +2391,7 @@ public class MiniDFSCluster {
    * Wait until the cluster is active and running.
    */
   public void waitActive() throws IOException {
-    for (int index = 0; index < nameNodes.length; index++) {
+    for (int index = 0; index < namenodes.size(); index++) {
       int failedCount = 0;
       while (true) {
         try {
@@ -2298,7 +2411,14 @@ public class MiniDFSCluster {
     }
     LOG.info("Cluster is active");
   }
-  
+
+  public void printNNs() {
+    for (int i = 0; i < namenodes.size(); i++) {
+      LOG.info("Have namenode " + i + ", info:" + getNN(i));
+      LOG.info(" has namenode: " + getNN(i).nameNode);
+    }
+  }
+
   private synchronized boolean shouldWait(DatanodeInfo[] dnInfo,
       InetSocketAddress addr) {
     // If a datanode failed to start, then do not wait
@@ -2696,7 +2816,7 @@ public class MiniDFSCluster {
    * namenode
    */
   private void checkSingleNameNode() {
-    if (nameNodes.length != 1) {
+    if (namenodes.size() != 1) {
       throw new IllegalArgumentException("Namenode index is needed");
     }
   }
@@ -2712,13 +2832,9 @@ public class MiniDFSCluster {
     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];
-    System.arraycopy(nameNodes, 0, newlist, 0, nameNodes.length);
-    nameNodes = newlist;
-    String nameserviceId = NAMESERVICE_ID_PREFIX + (nnIndex + 1);
-    
+    int nameServiceIndex = namenodes.keys().size();
+    String nameserviceId = NAMESERVICE_ID_PREFIX + (namenodes.keys().size() + 1);
+
     String nameserviceIds = conf.get(DFS_NAMESERVICES);
     nameserviceIds += "," + nameserviceId;
     conf.set(DFS_NAMESERVICES, nameserviceIds);
@@ -2726,9 +2842,11 @@ public class MiniDFSCluster {
     String nnId = null;
     initNameNodeAddress(conf, nameserviceId,
         new NNConf(nnId).setIpcPort(namenodePort));
-    initNameNodeConf(conf, nameserviceId, nnId, true, true, nnIndex);
-    createNameNode(nnIndex, conf, numDataNodes, true, null, null,
-        nameserviceId, nnId);
+    // figure out the current number of NNs
+    NameNodeInfo[] infos = this.getNameNodeInfos(nameserviceId);
+    int nnIndex = infos == null ? 0 : infos.length;
+    initNameNodeConf(conf, nameserviceId, nameServiceIndex, nnId, true, true, nnIndex);
+    NameNodeInfo info = createNameNode(conf, true, null, null, nameserviceId, nnId);
 
     // Refresh datanodes with the newly started namenode
     for (DataNodeProperties dn : dataNodes) {
@@ -2738,7 +2856,7 @@ public class MiniDFSCluster {
 
     // Wait for new namenode to get registrations from all the datanodes
     waitActive(nnIndex);
-    return nameNodes[nnIndex].nameNode;
+    return info.nameNode;
   }
   
   protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
----------------------------------------------------------------------
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
index a99e9c3..b9786a3 100644
--- 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
@@ -56,10 +56,20 @@ public class MiniDFSNNTopology {
    * 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")));
+    return simpleHATopology(2);
+  }
+
+  /**
+   * Set up an HA topology with a single HA nameservice.
+   * @param nnCount of namenodes to use with the nameservice
+   */
+  public static MiniDFSNNTopology simpleHATopology(int nnCount) {
+    MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf("minidfs-ns");
+    for (int i = 1; i <= nnCount; i++) {
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i));
+    }
+    MiniDFSNNTopology topology = new MiniDFSNNTopology().addNameservice(nameservice);
+    return topology;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index ad907f6..fae1024 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -303,12 +303,12 @@ public class TestDFSUpgradeFromImage {
     unpackStorage(HADOOP22_IMAGE, HADOOP_DFS_DIR_TXT);
     
     // Overwrite the md5 stored in the VERSION files
-    File baseDir = new File(MiniDFSCluster.getBaseDirectory());
+    File[] nnDirs = MiniDFSCluster.getNameNodeDirectory(MiniDFSCluster.getBaseDirectory(), 0, 0);
     FSImageTestUtil.corruptVersionFile(
-        new File(baseDir, "name1/current/VERSION"),
+        new File(nnDirs[0], "current/VERSION"),
         "imageMD5Digest", "22222222222222222222222222222222");
     FSImageTestUtil.corruptVersionFile(
-        new File(baseDir, "name2/current/VERSION"),
+        new File(nnDirs[1], "current/VERSION"),
         "imageMD5Digest", "22222222222222222222222222222222");
     
     // Attach our own log appender so we can verify output

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
index c4c890f..b50b1cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.junit.Assert.assertTrue;
 
 /**
  * This class tests rolling upgrade.
@@ -66,7 +67,7 @@ public class TestRollingUpgrade {
    */
   @Test
   public void testDFSAdminRollingUpgradeCommands() throws Exception {
-    // start a cluster 
+    // start a cluster
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     try {
@@ -97,7 +98,7 @@ public class TestRollingUpgrade {
         runCmd(dfsadmin, true, "-rollingUpgrade", "query");
 
         dfs.mkdirs(bar);
-        
+
         //finalize rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "finalize");
 
@@ -143,7 +144,7 @@ public class TestRollingUpgrade {
     String nnDirPrefix = MiniDFSCluster.getBaseDirectory() + "/nn/";
     final File nn1Dir = new File(nnDirPrefix + "image1");
     final File nn2Dir = new File(nnDirPrefix + "image2");
-    
+
     LOG.info("nn1Dir=" + nn1Dir);
     LOG.info("nn2Dir=" + nn2Dir);
 
@@ -186,9 +187,9 @@ public class TestRollingUpgrade {
 
       final RollingUpgradeInfo info1;
       {
-        final DistributedFileSystem dfs = cluster.getFileSystem(); 
+        final DistributedFileSystem dfs = cluster.getFileSystem();
         dfs.mkdirs(foo);
-  
+
         //start rolling upgrade
         dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
         info1 = dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
@@ -197,7 +198,7 @@ public class TestRollingUpgrade {
 
         //query rolling upgrade
         Assert.assertEquals(info1, dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
-  
+
         dfs.mkdirs(bar);
         cluster.shutdown();
       }
@@ -209,8 +210,8 @@ public class TestRollingUpgrade {
         .format(false)
         .manageNameDfsDirs(false)
         .build();
-      final DistributedFileSystem dfs2 = cluster2.getFileSystem(); 
-      
+      final DistributedFileSystem dfs2 = cluster2.getFileSystem();
+
       // Check that cluster2 sees the edits made on cluster1
       Assert.assertTrue(dfs2.exists(foo));
       Assert.assertTrue(dfs2.exists(bar));
@@ -260,7 +261,7 @@ public class TestRollingUpgrade {
 
   @Test
   public void testRollback() throws IOException {
-    // start a cluster 
+    // start a cluster
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     try {
@@ -305,7 +306,7 @@ public class TestRollingUpgrade {
       if(cluster != null) cluster.shutdown();
     }
   }
-  
+
   private static void startRollingUpgrade(Path foo, Path bar,
       Path file, byte[] data,
       MiniDFSCluster cluster) throws IOException {
@@ -327,7 +328,7 @@ public class TestRollingUpgrade {
     TestFileTruncate.checkBlockRecovery(file, dfs);
     AppendTestUtil.checkFullFile(dfs, file, newLength, data);
   }
-  
+
   private static void rollbackRollingUpgrade(Path foo, Path bar,
       Path file, byte[] data,
       MiniDFSCluster cluster) throws IOException {
@@ -372,22 +373,33 @@ public class TestRollingUpgrade {
     }
   }
 
-  @Test (timeout = 300000)
+  @Test(timeout = 300000)
   public void testFinalize() throws Exception {
+    testFinalize(2);
+  }
+
+  @Test(timeout = 300000)
+  public void testFinalizeWithMultipleNN() throws Exception {
+    testFinalize(3);
+  }
+
+  private void testFinalize(int nnCount) throws Exception {
     final Configuration conf = new HdfsConfiguration();
     MiniQJMHACluster cluster = null;
     final Path foo = new Path("/foo");
     final Path bar = new Path("/bar");
 
     try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
+      cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
 
-      // let NN1 tail editlog every 1s
-      dfsCluster.getConfiguration(1).setInt(
-          DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
-      dfsCluster.restartNameNode(1);
+      // let other NN tail editlog every 1s
+      for(int i=1; i < nnCount; i++) {
+        dfsCluster.getConfiguration(i).setInt(
+            DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+      }
+      dfsCluster.restartNameNodes();
 
       dfsCluster.transitionToActive(0);
       DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
@@ -425,17 +437,29 @@ public class TestRollingUpgrade {
 
   @Test (timeout = 300000)
   public void testQuery() throws Exception {
+    testQuery(2);
+  }
+
+  @Test (timeout = 300000)
+  public void testQueryWithMultipleNN() throws Exception {
+    testQuery(3);
+  }
+
+  private void testQuery(int nnCount) throws Exception{
     final Configuration conf = new Configuration();
     MiniQJMHACluster cluster = null;
     try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
+      cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
 
       dfsCluster.transitionToActive(0);
       DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
 
-      dfsCluster.shutdownNameNode(1);
+      // shutdown other NNs
+      for (int i = 1; i < nnCount; i++) {
+        dfsCluster.shutdownNameNode(i);
+      }
 
       // start rolling upgrade
       RollingUpgradeInfo info = dfs
@@ -445,13 +469,16 @@ public class TestRollingUpgrade {
       info = dfs.rollingUpgrade(RollingUpgradeAction.QUERY);
       Assert.assertFalse(info.createdRollbackImages());
 
-      dfsCluster.restartNameNode(1);
-
+      // restart other NNs
+      for (int i = 1; i < nnCount; i++) {
+        dfsCluster.restartNameNode(i);
+      }
+      // check that one of the other NNs has created the rollback image and uploaded it
       queryForPreparation(dfs);
 
       // The NN should have a copy of the fsimage in case of rollbacks.
       Assert.assertTrue(dfsCluster.getNamesystem(0).getFSImage()
-          .hasRollbackFSImage());
+              .hasRollbackFSImage());
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -487,6 +514,15 @@ public class TestRollingUpgrade {
 
   @Test(timeout = 300000)
   public void testCheckpoint() throws IOException, InterruptedException {
+    testCheckpoint(2);
+  }
+
+  @Test(timeout = 300000)
+  public void testCheckpointWithMultipleNN() throws IOException, InterruptedException {
+    testCheckpoint(3);
+  }
+
+  public void testCheckpoint(int nnCount) throws IOException, InterruptedException {
     final Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 1);
@@ -495,7 +531,7 @@ public class TestRollingUpgrade {
     final Path foo = new Path("/foo");
 
     try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
+      cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
 
@@ -513,16 +549,9 @@ public class TestRollingUpgrade {
       long txid = dfs.rollEdits();
       Assert.assertTrue(txid > 0);
 
-      int retries = 0;
-      while (++retries < 5) {
-        NNStorage storage = dfsCluster.getNamesystem(1).getFSImage()
-            .getStorage();
-        if (storage.getFsImageName(txid - 1) != null) {
-          return;
-        }
-        Thread.sleep(1000);
+      for(int i=1; i< nnCount; i++) {
+        verifyNNCheckpoint(dfsCluster, txid, i);
       }
-      Assert.fail("new checkpoint does not exist");
 
     } finally {
       if (cluster != null) {
@@ -531,6 +560,22 @@ public class TestRollingUpgrade {
     }
   }
 
+  /**
+   * Verify that the namenode at the given index has an FSImage with a TxId up to txid-1
+   */
+  private void verifyNNCheckpoint(MiniDFSCluster dfsCluster, long txid, int nnIndex) throws InterruptedException {
+    int retries = 0;
+    while (++retries < 5) {
+      NNStorage storage = dfsCluster.getNamesystem(nnIndex).getFSImage()
+              .getStorage();
+      if (storage.getFsImageName(txid - 1) != null) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    Assert.fail("new checkpoint does not exist");
+  }
+
   static void queryForPreparation(DistributedFileSystem dfs) throws IOException,
       InterruptedException {
     RollingUpgradeInfo info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
index ef4c559..470a08b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
@@ -17,43 +17,39 @@
  */
 package org.apache.hadoop.hdfs.qjournal;
 
-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.BindException;
-import java.net.URI;
-import java.util.Random;
-
 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.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 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 java.io.IOException;
+import java.net.BindException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
 
 public class MiniQJMHACluster {
   private MiniDFSCluster cluster;
   private MiniJournalCluster journalCluster;
   private final Configuration conf;
   private static final Log LOG = LogFactory.getLog(MiniQJMHACluster.class);
-  
+
   public static final String NAMESERVICE = "ns1";
-  private static final String NN1 = "nn1";
-  private static final String NN2 = "nn2";
   private static final Random RANDOM = new Random();
   private int basePort = 10000;
 
   public static class Builder {
     private final Configuration conf;
     private StartupOption startOpt = null;
+    private int numNNs = 2;
     private final MiniDFSCluster.Builder dfsBuilder;
-    
+
     public Builder(Configuration conf) {
       this.conf = conf;
       // most QJMHACluster tests don't need DataNodes, so we'll make
@@ -64,7 +60,7 @@ public class MiniQJMHACluster {
     public MiniDFSCluster.Builder getDfsBuilder() {
       return dfsBuilder;
     }
-    
+
     public MiniQJMHACluster build() throws IOException {
       return new MiniQJMHACluster(this);
     }
@@ -72,15 +68,25 @@ public class MiniQJMHACluster {
     public void startupOption(StartupOption startOpt) {
       this.startOpt = startOpt;
     }
+
+    public Builder setNumNameNodes(int nns) {
+      this.numNNs = nns;
+      return this;
+    }
+  }
+
+  public static MiniDFSNNTopology createDefaultTopology(int nns, int startingPort) {
+    MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf(NAMESERVICE);
+    for (int i = 0; i < nns; i++) {
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i).setIpcPort(startingPort++)
+          .setHttpPort(startingPort++));
+    }
+
+    return new MiniDFSNNTopology().addNameservice(nameservice);
   }
-  
+
   public static MiniDFSNNTopology createDefaultTopology(int basePort) {
-    return new MiniDFSNNTopology()
-      .addNameservice(new MiniDFSNNTopology.NSConf(NAMESERVICE).addNN(
-        new MiniDFSNNTopology.NNConf("nn1").setIpcPort(basePort)
-            .setHttpPort(basePort + 1)).addNN(
-        new MiniDFSNNTopology.NNConf("nn2").setIpcPort(basePort + 2)
-            .setHttpPort(basePort + 3)));
+    return createDefaultTopology(2, basePort);
   }
 
   private MiniQJMHACluster(Builder builder) throws IOException {
@@ -94,10 +100,10 @@ public class MiniQJMHACluster {
             .build();
         URI journalURI = journalCluster.getQuorumJournalURI(NAMESERVICE);
 
-        // start cluster with 2 NameNodes
-        MiniDFSNNTopology topology = createDefaultTopology(basePort);
+        // start cluster with specified NameNodes
+        MiniDFSNNTopology topology = createDefaultTopology(builder.numNNs, basePort);
 
-        initHAConf(journalURI, builder.conf);
+        initHAConf(journalURI, builder.conf, builder.numNNs);
 
         // First start up the NNs just to format the namespace. The MinIDFSCluster
         // has no way to just format the NameNodes without also starting them.
@@ -110,8 +116,9 @@ public class MiniQJMHACluster {
         Configuration confNN0 = cluster.getConfiguration(0);
         NameNode.initializeSharedEdits(confNN0, true);
 
-        cluster.getNameNodeInfos()[0].setStartOpt(builder.startOpt);
-        cluster.getNameNodeInfos()[1].setStartOpt(builder.startOpt);
+        for (MiniDFSCluster.NameNodeInfo nn : cluster.getNameNodeInfos()) {
+          nn.setStartOpt(builder.startOpt);
+        }
 
         // restart the cluster
         cluster.restartNameNodes();
@@ -123,31 +130,28 @@ public class MiniQJMHACluster {
       }
     }
   }
-  
-  private Configuration initHAConf(URI journalURI, Configuration conf) {
+
+  private Configuration initHAConf(URI journalURI, Configuration conf, int numNNs) {
     conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
         journalURI.toString());
-    
-    String address1 = "127.0.0.1:" + basePort;
-    String address2 = "127.0.0.1:" + (basePort + 2);
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
-        NAMESERVICE, NN1), address1);
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
-        NAMESERVICE, NN2), address2);
-    conf.set(DFSConfigKeys.DFS_NAMESERVICES, NAMESERVICE);
-    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, NAMESERVICE),
-        NN1 + "," + NN2);
-    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + NAMESERVICE,
-        ConfiguredFailoverProxyProvider.class.getName());
-    conf.set("fs.defaultFS", "hdfs://" + NAMESERVICE);
-    
+
+    List<String> nns = new ArrayList<String>(numNNs);
+    int port = basePort;
+    for (int i = 0; i < numNNs; i++) {
+      nns.add("127.0.0.1:" + port);
+      // increment by 2 each time to account for the http port in the config setting
+      port += 2;
+    }
+
+    // use standard failover configurations
+    HATestUtil.setFailoverConfigurations(conf, NAMESERVICE, nns);
     return conf;
   }
 
   public MiniDFSCluster getDfsCluster() {
     return cluster;
   }
-  
+
   public MiniJournalCluster getJournalCluster() {
     return journalCluster;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
----------------------------------------------------------------------
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 d5a9426..b203872 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
@@ -162,7 +162,7 @@ public class TestBlockToken {
   public void testWritable() throws Exception {
     TestWritable.testWritable(new BlockTokenIdentifier());
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     TestWritable.testWritable(generateTokenId(sm, block1,
         EnumSet.allOf(BlockTokenIdentifier.AccessMode.class)));
     TestWritable.testWritable(generateTokenId(sm, block2,
@@ -201,7 +201,7 @@ public class TestBlockToken {
   @Test
   public void testBlockTokenSecretManager() throws Exception {
     BlockTokenSecretManager masterHandler = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     BlockTokenSecretManager slaveHandler = new BlockTokenSecretManager(
         blockKeyUpdateInterval, blockTokenLifetime, "fake-pool", null);
     ExportedBlockKeys keys = masterHandler.exportKeys();
@@ -244,7 +244,7 @@ public class TestBlockToken {
     UserGroupInformation.setConfiguration(conf);
     
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     Token<BlockTokenIdentifier> token = sm.generateToken(block3,
         EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
 
@@ -283,7 +283,7 @@ public class TestBlockToken {
     
     Assume.assumeTrue(FD_DIR.exists());
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     Token<BlockTokenIdentifier> token = sm.generateToken(block3,
         EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
 
@@ -352,7 +352,7 @@ public class TestBlockToken {
     for (int i = 0; i < 10; i++) {
       String bpid = Integer.toString(i);
       BlockTokenSecretManager masterHandler = new BlockTokenSecretManager(
-          blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+          blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
       BlockTokenSecretManager slaveHandler = new BlockTokenSecretManager(
           blockKeyUpdateInterval, blockTokenLifetime, "fake-pool", null);
       bpMgr.addBlockPool(bpid, slaveHandler);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
----------------------------------------------------------------------
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 f01be4b..0818571 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
@@ -319,7 +319,7 @@ public class TestBackupNode {
       if(fileSys != null) fileSys.close();
       if(cluster != null) cluster.shutdown();
     }
-    File nnCurDir = new File(BASE_DIR, "name1/current/");
+    File nnCurDir = new File(MiniDFSCluster.getNameNodeDirectory(BASE_DIR, 0, 0)[0], "current/");
     File bnCurDir = new File(getBackupNodeDir(op, 1), "/current/");
 
     FSImageTestUtil.assertParallelFilesAreIdentical(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
----------------------------------------------------------------------
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 5a51cb7..7073726 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
@@ -1428,7 +1428,8 @@ public class TestCheckpoint {
       //
       secondary = startSecondaryNameNode(conf);
 
-      File secondaryDir = new File(MiniDFSCluster.getBaseDirectory(), "namesecondary1");
+      File secondaryDir = MiniDFSCluster.getCheckpointDirectory(MiniDFSCluster.getBaseDirectory(),
+        0, 0)[0];
       File secondaryCurrent = new File(secondaryDir, "current");
 
       long expectedTxIdToDownload = cluster.getNameNode().getFSImage()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
----------------------------------------------------------------------
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
index 5b72901..a736d27 100644
--- 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
@@ -42,7 +42,8 @@ public class HAStressTestHarness {
   private MiniDFSCluster cluster;
   static final int BLOCK_SIZE = 1024;
   final TestContext testCtx = new TestContext();
-  
+  private int nns = 2;
+
   public HAStressTestHarness() {
     conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -55,11 +56,19 @@ public class HAStressTestHarness {
   }
 
   /**
+   * Set the number of namenodes that should be run. This must be set before calling
+   * {@link #startCluster()}
+   */
+  public void setNumberOfNameNodes(int nns) {
+    this.nns = nns;
+  }
+
+  /**
    * Start and return the MiniDFSCluster.
    */
   public MiniDFSCluster startCluster() throws IOException {
     cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .nnTopology(MiniDFSNNTopology.simpleHATopology(nns))
       .numDataNodes(3)
       .build();
     return cluster;
@@ -99,28 +108,27 @@ public class HAStressTestHarness {
   }
 
   /**
-   * Add a thread which periodically triggers failover back and forth between
-   * the two namenodes.
+   * Add a thread which periodically triggers failover back and forth between the 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);
+        // fail over from one namenode to the next, all the way back to the original NN
+        for (int i = 0; i < nns; i++) {
+          // next node, mod nns so we wrap to the 0th NN on the last iteration
+          int next = (i + 1) % nns;
+          System.err.println("==============================\n"
+              + "[Starting] Failing over from " + i + "->" + next + "\n"
+              + "==============================");
+          cluster.transitionToStandby(i);
+          cluster.transitionToActive(next);
+          System.err.println("==============================\n"
+              + "[Completed] Failing over from " + i + "->" + next + ". Sleeping for "+
+              (msBetweenFailovers/1000) +"sec \n"
+              + "==============================");
+          Thread.sleep(msBetweenFailovers);
+        }
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
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
index c7c4a77..5543a2a 100644
--- 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
@@ -24,9 +24,14 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
 
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -67,12 +72,11 @@ public abstract class HATestUtil {
    */
   public static void waitForStandbyToCatchUp(NameNode active,
       NameNode standby) throws InterruptedException, IOException, CouldNotCatchUpException {
-    
     long activeTxId = active.getNamesystem().getFSImage().getEditLog()
       .getLastWrittenTxId();
-    
+
     active.getRpcServer().rollEditLog();
-    
+
     long start = Time.now();
     while (Time.now() - start < TestEditLogTailer.NN_LAG_TIMEOUT) {
       long nn2HighestTxId = standby.getNamesystem().getFSImage()
@@ -166,34 +170,52 @@ public abstract class HATestUtil {
   /** 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();
-    setFailoverConfigurations(conf, logicalName, nnAddr1, nnAddr2);
+    MiniDFSCluster.NameNodeInfo[] nns = cluster.getNameNodeInfos(nsIndex);
+    List<InetSocketAddress> nnAddresses = new ArrayList<InetSocketAddress>(3);
+    for (MiniDFSCluster.NameNodeInfo nn : nns) {
+      nnAddresses.add(nn.nameNode.getNameNodeAddress());
+    }
+    setFailoverConfigurations(conf, logicalName, nnAddresses);
+  }
+
+  public static void setFailoverConfigurations(Configuration conf, String logicalName,
+      InetSocketAddress ... nnAddresses){
+    setFailoverConfigurations(conf, logicalName, Arrays.asList(nnAddresses));
   }
 
   /**
    * Sets the required configurations for performing failover
    */
   public static void setFailoverConfigurations(Configuration conf,
-      String logicalName, InetSocketAddress nnAddr1,
-      InetSocketAddress nnAddr2) {
-    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);
-    
+      String logicalName, List<InetSocketAddress> nnAddresses) {
+    setFailoverConfigurations(conf, logicalName,
+        Iterables.transform(nnAddresses, new Function<InetSocketAddress, String>() {
+
+          // transform the inet address to a simple string
+          @Override
+          public String apply(InetSocketAddress addr) {
+            return "hdfs://" + addr.getHostName() + ":" + addr.getPort();
+          }
+        }));
+  }
+
+  public static void setFailoverConfigurations(Configuration conf, String logicalName,
+      Iterable<String> nnAddresses) {
+    List<String> nnids = new ArrayList<String>();
+    int i = 0;
+    for (String address : nnAddresses) {
+      String nnId = "nn" + (i + 1);
+      nnids.add(nnId);
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, logicalName, nnId), address);
+      i++;
+    }
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, logicalName);
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, logicalName),
-        nameNodeId1 + "," + nameNodeId2);
+        Joiner.on(',').join(nnids));
     conf.set(HdfsClientConfigKeys.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());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
index 7abc502..16dc766 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
@@ -46,37 +46,47 @@ import com.google.common.collect.ImmutableList;
 
 public class TestBootstrapStandby {
   private static final Log LOG = LogFactory.getLog(TestBootstrapStandby.class);
-  
+
+  private static final int maxNNCount = 3;
+  private static final int STARTING_PORT = 20000;
+
   private MiniDFSCluster cluster;
   private NameNode nn0;
-  
+
   @Before
   public void setupCluster() throws IOException {
     Configuration conf = new Configuration();
 
-    MiniDFSNNTopology topology = new MiniDFSNNTopology()
-      .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(20001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(20002)));
-    
+    // duplicate code with MiniQJMHACluster#createDefaultTopology, but don't want to cross
+    // dependencies or munge too much code to support it all correctly
+    MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf("ns1");
+    for (int i = 0; i < maxNNCount; i++) {
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i).setHttpPort(STARTING_PORT + i + 1));
+    }
+
+    MiniDFSNNTopology topology = new MiniDFSNNTopology().addNameservice(nameservice);
+
     cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(topology)
-      .numDataNodes(0)
-      .build();
+        .nnTopology(topology)
+        .numDataNodes(0)
+        .build();
     cluster.waitActive();
-    
+
     nn0 = cluster.getNameNode(0);
     cluster.transitionToActive(0);
-    cluster.shutdownNameNode(1);
+    // shutdown the other NNs
+    for (int i = 1; i < maxNNCount; i++) {
+      cluster.shutdownNameNode(i);
+    }
   }
-  
+
   @After
   public void shutdownCluster() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
-  
+
   /**
    * Test for the base success case. The primary NN
    * hasn't made any checkpoints, and we copy the fsimage_0
@@ -85,30 +95,29 @@ public class TestBootstrapStandby {
   @Test
   public void testSuccessfulBaseCase() throws Exception {
     removeStandbyNameDirs();
-    
-    try {
-      cluster.restartNameNode(1);
-      fail("Did not throw");
-    } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains(
-          "storage directory does not exist or is not accessible",
-          ioe);
+
+    // skip the first NN, its up
+    for (int index = 1; index < maxNNCount; index++) {
+      try {
+        cluster.restartNameNode(index);
+        fail("Did not throw");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "storage directory does not exist or is not accessible", ioe);
+      }
+
+      int rc = BootstrapStandby.run(new String[] { "-nonInteractive" },
+          cluster.getConfiguration(index));
+      assertEquals(0, rc);
+
+      // Should have copied over the namespace from the active
+      FSImageTestUtil.assertNNHasCheckpoints(cluster, index, ImmutableList.of(0));
     }
-    
-    int rc = BootstrapStandby.run(
-        new String[]{"-nonInteractive"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the active
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of(0));
-    FSImageTestUtil.assertNNFilesMatch(cluster);
 
-    // We should now be able to start the standby successfully.
-    cluster.restartNameNode(1);
+    // We should now be able to start the standbys successfully.
+    restartNameNodesFromIndex(1);
   }
-  
+
   /**
    * Test for downloading a checkpoint made at a later checkpoint
    * from the active.
@@ -123,21 +132,21 @@ public class TestBootstrapStandby {
     NameNodeAdapter.saveNamespace(nn0);
     NameNodeAdapter.leaveSafeMode(nn0);
     long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
-      .getFSImage().getMostRecentCheckpointTxId();
+        .getFSImage().getMostRecentCheckpointTxId();
     assertEquals(6, expectedCheckpointTxId);
 
-    int rc = BootstrapStandby.run(
-        new String[]{"-force"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the active
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of((int)expectedCheckpointTxId));
+    for (int i = 1; i < maxNNCount; i++) {
+      assertEquals(0, forceBootstrap(i));
+
+      // Should have copied over the namespace from the active
+      LOG.info("Checking namenode: " + i);
+      FSImageTestUtil.assertNNHasCheckpoints(cluster, i,
+          ImmutableList.of((int) expectedCheckpointTxId));
+    }
     FSImageTestUtil.assertNNFilesMatch(cluster);
 
     // We should now be able to start the standby successfully.
-    cluster.restartNameNode(1);
+    restartNameNodesFromIndex(1);
   }
 
   /**
@@ -147,36 +156,40 @@ public class TestBootstrapStandby {
   @Test
   public void testSharedEditsMissingLogs() throws Exception {
     removeStandbyNameDirs();
-    
+
     CheckpointSignature sig = nn0.getRpcServer().rollEditLog();
     assertEquals(3, sig.getCurSegmentTxId());
-    
+
     // Should have created edits_1-2 in shared edits dir
-    URI editsUri = cluster.getSharedEditsDir(0, 1);
+    URI editsUri = cluster.getSharedEditsDir(0, maxNNCount - 1);
     File editsDir = new File(editsUri);
-    File editsSegment = new File(new File(editsDir, "current"),
+    File currentDir = new File(editsDir, "current");
+    File editsSegment = new File(currentDir,
         NNStorage.getFinalizedEditsFileName(1, 2));
     GenericTestUtils.assertExists(editsSegment);
+    GenericTestUtils.assertExists(currentDir);
 
     // Delete the segment.
     assertTrue(editsSegment.delete());
-    
+
     // Trying to bootstrap standby should now fail since the edit
     // logs aren't available in the shared dir.
     LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(
         LogFactory.getLog(BootstrapStandby.class));
     try {
-      int rc = BootstrapStandby.run(
-          new String[]{"-force"},
-          cluster.getConfiguration(1));
-      assertEquals(BootstrapStandby.ERR_CODE_LOGS_UNAVAILABLE, rc);
+      assertEquals(BootstrapStandby.ERR_CODE_LOGS_UNAVAILABLE, forceBootstrap(1));
     } finally {
       logs.stopCapturing();
     }
     GenericTestUtils.assertMatches(logs.getOutput(),
         "FATAL.*Unable to read transaction ids 1-3 from the configured shared");
   }
-  
+
+  /**
+   * Show that bootstrapping will fail on a given NameNode if its directories already exist. Its not
+   * run across all the NN because its testing the state local on each node.
+   * @throws Exception on unexpected failure
+   */
   @Test
   public void testStandbyDirsAlreadyExist() throws Exception {
     // Should not pass since standby dirs exist, force not given
@@ -186,12 +199,9 @@ public class TestBootstrapStandby {
     assertEquals(BootstrapStandby.ERR_CODE_ALREADY_FORMATTED, rc);
 
     // Should pass with -force
-    rc = BootstrapStandby.run(
-        new String[]{"-force"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
+    assertEquals(0, forceBootstrap(1));
   }
-  
+
   /**
    * Test that, even if the other node is not active, we are able
    * to bootstrap standby from it.
@@ -199,18 +209,44 @@ public class TestBootstrapStandby {
   @Test(timeout=30000)
   public void testOtherNodeNotActive() throws Exception {
     cluster.transitionToStandby(0);
-    int rc = BootstrapStandby.run(
-        new String[]{"-force"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
+    assertSuccessfulBootstrapFromIndex(1);
   }
 
   private void removeStandbyNameDirs() {
-    for (URI u : cluster.getNameDirs(1)) {
-      assertTrue(u.getScheme().equals("file"));
-      File dir = new File(u.getPath());
-      LOG.info("Removing standby dir " + dir);
-      assertTrue(FileUtil.fullyDelete(dir));
+    for (int i = 1; i < maxNNCount; i++) {
+      for (URI u : cluster.getNameDirs(i)) {
+        assertTrue(u.getScheme().equals("file"));
+        File dir = new File(u.getPath());
+        LOG.info("Removing standby dir " + dir);
+        assertTrue(FileUtil.fullyDelete(dir));
+      }
+    }
+  }
+
+  private void restartNameNodesFromIndex(int start) throws IOException {
+    for (int i = start; i < maxNNCount; i++) {
+      // We should now be able to start the standby successfully.
+      cluster.restartNameNode(i, false);
+    }
+
+    cluster.waitClusterUp();
+    cluster.waitActive();
+  }
+
+  /**
+   * Force boot strapping on a namenode
+   * @param i index of the namenode to attempt
+   * @return exit code
+   * @throws Exception on unexpected failure
+   */
+  private int forceBootstrap(int i) throws Exception {
+    return BootstrapStandby.run(new String[] { "-force" },
+        cluster.getConfiguration(i));
+  }
+
+  private void assertSuccessfulBootstrapFromIndex(int start) throws Exception {
+    for (int i = start; i < maxNNCount; i++) {
+      assertEquals(0, forceBootstrap(i));
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
index ca8f563..db9a2de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
@@ -52,7 +52,8 @@ public class TestBootstrapStandbyWithQJM {
 
   private MiniDFSCluster cluster;
   private MiniJournalCluster jCluster;
-  
+  private int nnCount = 3;
+
   @Before
   public void setup() throws Exception {
     Configuration conf = new Configuration();
@@ -62,7 +63,8 @@ public class TestBootstrapStandbyWithQJM {
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
 
-    MiniQJMHACluster miniQjmHaCluster = new MiniQJMHACluster.Builder(conf).build();
+    MiniQJMHACluster miniQjmHaCluster =
+        new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
     cluster = miniQjmHaCluster.getDfsCluster();
     jCluster = miniQjmHaCluster.getJournalCluster();
     
@@ -90,18 +92,7 @@ public class TestBootstrapStandbyWithQJM {
   public void testBootstrapStandbyWithStandbyNN() throws Exception {
     // make the first NN in standby state
     cluster.transitionToStandby(0);
-    Configuration confNN1 = cluster.getConfiguration(1);
-    
-    // shut down nn1
-    cluster.shutdownNameNode(1);
-    
-    int rc = BootstrapStandby.run(new String[] { "-force" }, confNN1);
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the standby
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of(0));
-    FSImageTestUtil.assertNNFilesMatch(cluster);
+    bootstrapStandbys();
   }
   
   /** BootstrapStandby when the existing NN is active */
@@ -109,17 +100,23 @@ public class TestBootstrapStandbyWithQJM {
   public void testBootstrapStandbyWithActiveNN() throws Exception {
     // make the first NN in active state
     cluster.transitionToActive(0);
-    Configuration confNN1 = cluster.getConfiguration(1);
-    
-    // shut down nn1
-    cluster.shutdownNameNode(1);
-    
-    int rc = BootstrapStandby.run(new String[] { "-force" }, confNN1);
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the standby
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of(0));
+    bootstrapStandbys();
+  }
+
+  private void bootstrapStandbys() throws Exception {
+    // shutdown and bootstrap all the other nns, except the first (start 1, not 0)
+    for (int i = 1; i < nnCount; i++) {
+      Configuration otherNNConf = cluster.getConfiguration(i);
+
+      // shut down other nn
+      cluster.shutdownNameNode(i);
+
+      int rc = BootstrapStandby.run(new String[] { "-force" }, otherNNConf);
+      assertEquals(0, rc);
+
+      // Should have copied over the namespace from the standby
+      FSImageTestUtil.assertNNHasCheckpoints(cluster, i, ImmutableList.of(0));
+    }
     FSImageTestUtil.assertNNFilesMatch(cluster);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
----------------------------------------------------------------------
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
index e7cba75..9164582 100644
--- 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
@@ -107,6 +107,7 @@ public class TestDNFencingWithReplication {
   @Test
   public void testFencingStress() throws Exception {
     HAStressTestHarness harness = new HAStressTestHarness();
+    harness.setNumberOfNameNodes(3);
     harness.conf.setInt(
         DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
     harness.conf.setInt(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
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
index 8c61c92..aea4f87 100644
--- 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
@@ -113,7 +113,12 @@ public class TestEditLogTailer {
   public void testNN1TriggersLogRolls() throws Exception {
     testStandbyTriggersLogRolls(1);
   }
-  
+
+  @Test
+  public void testNN2TriggersLogRolls() throws Exception {
+    testStandbyTriggersLogRolls(2);
+  }
+
   private static void testStandbyTriggersLogRolls(int activeIndex)
       throws Exception {
     Configuration conf = new Configuration();
@@ -125,7 +130,8 @@ public class TestEditLogTailer {
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
         .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10031))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10032)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10032))
+        .addNN(new MiniDFSNNTopology.NNConf("nn3").setIpcPort(10033)));
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)
@@ -145,7 +151,7 @@ public class TestEditLogTailer {
   
   private static void waitForLogRollInSharedDir(MiniDFSCluster cluster,
       long startTxId) throws Exception {
-    URI sharedUri = cluster.getSharedEditsDir(0, 1);
+    URI sharedUri = cluster.getSharedEditsDir(0, 2);
     File sharedDir = new File(sharedUri.getPath(), "current");
     final File expectedLog = new File(sharedDir,
         NNStorage.getInProgressEditsFileName(startTxId));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
index 151e7d3..116079a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
@@ -56,10 +56,11 @@ public class TestFailoverWithBlockTokensEnabled {
   
   private static final Path TEST_PATH = new Path("/test-path");
   private static final String TEST_DATA = "very important text";
-  
+  private static final int numNNs = 3;
+
   private Configuration conf;
   private MiniDFSCluster cluster;
-  
+
   @Before
   public void startCluster() throws IOException {
     conf = new Configuration();
@@ -67,7 +68,7 @@ public class TestFailoverWithBlockTokensEnabled {
     // Set short retry timeouts so this test runs faster
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
     cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .nnTopology(MiniDFSNNTopology.simpleHATopology(numNNs))
         .numDataNodes(1)
         .build();
   }
@@ -78,33 +79,41 @@ public class TestFailoverWithBlockTokensEnabled {
       cluster.shutdown();
     }
   }
-  
+
   @Test
   public void ensureSerialNumbersNeverOverlap() {
     BlockTokenSecretManager btsm1 = cluster.getNamesystem(0).getBlockManager()
         .getBlockTokenSecretManager();
     BlockTokenSecretManager btsm2 = cluster.getNamesystem(1).getBlockManager()
         .getBlockTokenSecretManager();
-    
-    btsm1.setSerialNo(0);
-    btsm2.setSerialNo(0);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
-    
-    btsm1.setSerialNo(Integer.MAX_VALUE);
-    btsm2.setSerialNo(Integer.MAX_VALUE);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
-    
-    btsm1.setSerialNo(Integer.MIN_VALUE);
-    btsm2.setSerialNo(Integer.MIN_VALUE);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
-    
-    btsm1.setSerialNo(Integer.MAX_VALUE / 2);
-    btsm2.setSerialNo(Integer.MAX_VALUE / 2);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
+    BlockTokenSecretManager btsm3 = cluster.getNamesystem(2).getBlockManager()
+        .getBlockTokenSecretManager();
+
+    setAndCheckSerialNumber(0, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MAX_VALUE, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MIN_VALUE, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MAX_VALUE / 2, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MIN_VALUE / 2, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MAX_VALUE / 3, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MIN_VALUE / 3, btsm1, btsm2, btsm3);
+  }
+
+  private void setAndCheckSerialNumber(int serialNumber, BlockTokenSecretManager... btsms) {
+    for (BlockTokenSecretManager btsm : btsms) {
+      btsm.setSerialNo(serialNumber);
+    }
 
-    btsm1.setSerialNo(Integer.MIN_VALUE / 2);
-    btsm2.setSerialNo(Integer.MIN_VALUE / 2);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
+    for (int i = 0; i < btsms.length; i++) {
+      for (int j = 0; j < btsms.length; j++) {
+        if (j == i) {
+          continue;
+        }
+        int first = btsms[i].getSerialNoForTesting();
+        int second = btsms[j].getSerialNoForTesting();
+        assertFalse("Overlap found for set serial number (" + serialNumber + ") is " + i + ": "
+            + first + " == " + j + ": " + second, first == second);
+      }
+    }
   }
   
   @Test


[02/21] hadoop git commit: MAPREDUCE-6403. Fix typo in the usage of NNBench. Contributed by Jagadesh Kiran N.

Posted by ar...@apache.org.
MAPREDUCE-6403. Fix typo in the usage of NNBench. Contributed by Jagadesh Kiran N.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/445b1324
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/445b1324
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/445b1324

Branch: refs/heads/HDFS-7240
Commit: 445b13240cb632ee8bc718f0576f783a7350af2e
Parents: 6c7a9d5
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jun 22 15:32:39 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Jun 22 15:32:39 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/test/java/org/apache/hadoop/hdfs/NNBench.java             | 3 +--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/445b1324/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index d48fc5b..c42e410 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -492,6 +492,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6405. NullPointerException in App Attempts page.
     (Siqi Li and Gera Shegalov via devaraj)
 
+    MAPREDUCE-6403. Fix typo in the usage of NNBench.
+    (Jagadesh Kiran N via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/445b1324/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
index 6c8ec17..8a1f1ba 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
@@ -59,7 +59,6 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 
 /**
  * This program executes a specified operation that applies load to 
@@ -197,7 +196,7 @@ public class NNBench {
       "This is not mandatory>\n" +
       "\t-replicationFactorPerFile <Replication factor for the files." +
         " default is 1. This is not mandatory>\n" +
-      "\t-baseDir <base DFS path. default is /becnhmarks/NNBench. " +
+      "\t-baseDir <base DFS path. default is /benchmarks/NNBench. " +
       "This is not mandatory>\n" +
       "\t-readFileAfterOpen <true or false. if true, it reads the file and " +
       "reports the average time to read. This is valid with the open_read " +


[11/21] hadoop git commit: MAPREDUCE-6410. Fixed MapReduce JobHistory server to use the right (login) UGI to refresh log and cleaner settings. Contributed by Varun Saxena.

Posted by ar...@apache.org.
MAPREDUCE-6410. Fixed MapReduce JobHistory server to use the right (login) UGI to refresh log and cleaner settings. Contributed by Varun Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d481684c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d481684c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d481684c

Branch: refs/heads/HDFS-7240
Commit: d481684c7c9293a94f54ef622a92753531c6acc7
Parents: 41ae776
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Tue Jun 23 11:39:55 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Jun 23 11:39:55 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../mapreduce/v2/hs/server/HSAdminServer.java   | 43 ++++++++++++++-
 .../v2/hs/server/TestHSAdminServer.java         | 55 ++++++++++++++++++++
 3 files changed, 99 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d481684c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 0cefde0..5eae44e 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -548,6 +548,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6387. Serialize the recently added Task#encryptedSpillKey field at 
     the end. (Arun Suresh via kasha)
 
+    MAPREDUCE-6410. Fixed MapReduce JobHistory server to use the right (login)
+    UGI to refresh log and cleaner settings. (Varun Saxena via vinodkv)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d481684c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
index ef79bf8..3fef5e2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.hs.server;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -52,6 +53,7 @@ import org.apache.hadoop.mapreduce.v2.hs.JobHistory;
 import org.apache.hadoop.mapreduce.v2.hs.proto.HSAdminRefreshProtocolProtos.HSAdminRefreshProtocolService;
 import org.apache.hadoop.mapreduce.v2.hs.protocolPB.HSAdminRefreshProtocolServerSideTranslatorPB;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
 
 @Private
@@ -67,6 +69,8 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
   private static final String HISTORY_ADMIN_SERVER = "HSAdminServer";
   private JobHistory jobHistoryService = null;
 
+  private UserGroupInformation loginUGI;
+
   public HSAdminServer(AggregatedLogDeletionService aggLogDelService,
       JobHistory jobHistoryService) {
     super(HSAdminServer.class.getName());
@@ -125,9 +129,24 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
 
   @Override
   protected void serviceStart() throws Exception {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      loginUGI = UserGroupInformation.getLoginUser();
+    } else {
+      loginUGI = UserGroupInformation.getCurrentUser();
+    }
     clientRpcServer.start();
   }
 
+  @VisibleForTesting
+  UserGroupInformation getLoginUGI() {
+    return loginUGI;
+  }
+
+  @VisibleForTesting
+  void setLoginUGI(UserGroupInformation ugi) {
+    loginUGI = ugi;
+  }
+
   @Override
   protected void serviceStop() throws Exception {
     if (clientRpcServer != null) {
@@ -233,7 +252,17 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
   public void refreshLogRetentionSettings() throws IOException {
     UserGroupInformation user = checkAcls("refreshLogRetentionSettings");
 
-    aggLogDelService.refreshLogRetentionSettings();
+    try {
+      loginUGI.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws IOException {
+          aggLogDelService.refreshLogRetentionSettings();
+          return null;
+        }
+      });
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
 
     HSAuditLogger.logSuccess(user.getShortUserName(),
         "refreshLogRetentionSettings", "HSAdminServer");
@@ -243,7 +272,17 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
   public void refreshJobRetentionSettings() throws IOException {
     UserGroupInformation user = checkAcls("refreshJobRetentionSettings");
 
-    jobHistoryService.refreshJobRetentionSettings();
+    try {
+      loginUGI.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws IOException {
+          jobHistoryService.refreshJobRetentionSettings();
+          return null;
+        }
+      });
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
 
     HSAuditLogger.logSuccess(user.getShortUserName(),
         "refreshJobRetentionSettings", HISTORY_ADMIN_SERVER);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d481684c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
index 7ab90f0..d831347 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.mapreduce.v2.hs.server;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -45,6 +47,9 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
@@ -286,6 +291,56 @@ public class TestHSAdminServer {
     verify(jobHistoryService).refreshJobRetentionSettings();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testUGIForLogAndJobRefresh() throws Exception {
+    UserGroupInformation ugi =
+        UserGroupInformation.createUserForTesting("test", new String[] {"grp"});
+    UserGroupInformation loginUGI = spy(hsAdminServer.getLoginUGI());
+    hsAdminServer.setLoginUGI(loginUGI);
+
+    // Run refresh log retention settings with test user
+    ugi.doAs(new PrivilegedAction<Void>() {
+      @Override
+      public Void run() {
+        String[] args = new String[1];
+        args[0] = "-refreshLogRetentionSettings";
+        try {
+          hsAdminClient.run(args);
+        } catch (Exception e) {
+          fail("refreshLogRetentionSettings should have been successful");
+        }
+        return null;
+      }
+    });
+    // Verify if AggregatedLogDeletionService#refreshLogRetentionSettings was
+    // called with login UGI, instead of the UGI command was run with.
+    verify(loginUGI).doAs(any(PrivilegedExceptionAction.class));
+    verify(alds).refreshLogRetentionSettings();
+
+    // Reset for refresh job retention settings
+    reset(loginUGI);
+
+    // Run refresh job retention settings with test user
+    ugi.doAs(new PrivilegedAction<Void>() {
+      @Override
+      public Void run() {
+        String[] args = new String[1];
+        args[0] = "-refreshJobRetentionSettings";
+        try {
+          hsAdminClient.run(args);
+        } catch (Exception e) {
+          fail("refreshJobRetentionSettings should have been successful");
+        }
+        return null;
+      }
+    });
+    // Verify if JobHistory#refreshJobRetentionSettings was called with
+    // login UGI, instead of the UGI command was run with.
+    verify(loginUGI).doAs(any(PrivilegedExceptionAction.class));
+    verify(jobHistoryService).refreshJobRetentionSettings();
+  }
+
   @After
   public void cleanUp() {
     if (hsAdminServer != null)


[03/21] hadoop git commit: Move HDFS-4366 to 2.8.0 in CHANGES.txt

Posted by ar...@apache.org.
Move HDFS-4366 to 2.8.0 in CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5590e914
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5590e914
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5590e914

Branch: refs/heads/HDFS-7240
Commit: 5590e914f5889413da9eda047f64842c4b67fe85
Parents: 445b132
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Jun 22 14:18:55 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Jun 22 14:18:55 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5590e914/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2e030b9..234fcf7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -270,9 +270,6 @@ Trunk (Unreleased)
     HDFS-3934. duplicative dfs_hosts entries handled wrong. (Colin Patrick
     McCabe)
 
-    HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
-    Blocks for Lower-Priority Blocks (Derek Dagit via kihwal)
-
     HDFS-5719. FSImage#doRollback() should close prevState before return
     (Ted Yu via brandonli)
 
@@ -938,6 +935,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8337. Accessing httpfs via webhdfs doesn't work from a jar with
     kerberos. (Yongjun Zhang)
 
+    HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
+    Blocks for Lower-Priority Blocks (Derek Dagit via kihwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[17/21] hadoop git commit: MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed too early. Contributed by Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov.

Posted by ar...@apache.org.
MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed too early. Contributed by Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/72d08a0e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/72d08a0e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/72d08a0e

Branch: refs/heads/HDFS-7240
Commit: 72d08a0e41efda635baa985d55d67cb059a7c07c
Parents: 2ba6465
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 24 15:29:11 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 24 15:29:11 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                 |  4 ++++
 .../apache/hadoop/mapreduce/task/reduce/Fetcher.java |  1 +
 .../mapreduce/task/reduce/IFileWrappedMapOutput.java | 10 ++--------
 .../hadoop/mapreduce/task/reduce/LocalFetcher.java   | 15 +++++----------
 4 files changed, 12 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5eae44e..6c65032 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -505,6 +505,10 @@ Release 2.8.0 - UNRELEASED
     multibyte record delimiters well (Vinayakumar B, Rushabh Shah, and Akira
     AJISAKA via jlowe)
 
+    MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed
+    too early (Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov via
+    jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index 1e03387..fb0ac0a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -335,6 +335,7 @@ class Fetcher<K,V> extends Thread {
         try {
           failedTasks = copyMapOutput(host, input, remaining, fetchRetryEnabled);
         } catch (IOException e) {
+          IOUtils.cleanup(LOG, input);
           //
           // Setup connection again if disconnected by NM
           connection.disconnect();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
index 119db15..6051c34 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
@@ -60,13 +60,7 @@ public abstract class IFileWrappedMapOutput<K, V> extends MapOutput<K, V> {
                       long compressedLength, long decompressedLength,
                       ShuffleClientMetrics metrics,
                       Reporter reporter) throws IOException {
-    IFileInputStream iFin =
-        new IFileInputStream(input, compressedLength, conf);
-    try {
-      this.doShuffle(host, iFin, compressedLength,
-                    decompressedLength, metrics, reporter);
-    } finally {
-      iFin.close();
-    }
+    doShuffle(host, new IFileInputStream(input, compressedLength, conf),
+        compressedLength, decompressedLength, metrics, reporter);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
index de2382c..f45742f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.IndexRecord;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapOutputFile;
@@ -149,19 +150,13 @@ class LocalFetcher<K,V> extends Fetcher<K, V> {
     // now read the file, seek to the appropriate section, and send it.
     FileSystem localFs = FileSystem.getLocal(job).getRaw();
     FSDataInputStream inStream = localFs.open(mapOutputFileName);
-
-    inStream = CryptoUtils.wrapIfNecessary(job, inStream);
-
     try {
+      inStream = CryptoUtils.wrapIfNecessary(job, inStream);
       inStream.seek(ir.startOffset + CryptoUtils.cryptoPadding(job));
-      mapOutput.shuffle(LOCALHOST, inStream, compressedLength, decompressedLength, metrics, reporter);
+      mapOutput.shuffle(LOCALHOST, inStream, compressedLength,
+          decompressedLength, metrics, reporter);
     } finally {
-      try {
-        inStream.close();
-      } catch (IOException ioe) {
-        LOG.warn("IOException closing inputstream from map output: "
-            + ioe.toString());
-      }
+      IOUtils.cleanup(LOG, inStream);
     }
 
     scheduler.copySucceeded(mapTaskId, LOCALHOST, compressedLength, 0, 0,


[04/21] hadoop git commit: HDFS-8480. Fix performance and timeout issues in HDFS-7929 by using hard-links to preserve old edit logs, instead of copying them. (Zhe Zhang via Colin P. McCabe)

Posted by ar...@apache.org.
HDFS-8480. Fix performance and timeout issues in HDFS-7929 by using hard-links to preserve old edit logs, instead of copying them. (Zhe Zhang via Colin P. McCabe)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7b424f93
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7b424f93
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7b424f93

Branch: refs/heads/HDFS-7240
Commit: 7b424f938c3c306795d574792b086d84e4f06425
Parents: 5590e91
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Mon Jun 22 14:37:10 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Mon Jun 22 14:37:10 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/NNUpgradeUtil.java     | 18 +-----
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  | 58 +++++++++++++++-----
 3 files changed, 48 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b424f93/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 234fcf7..b9d9943 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -971,6 +971,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
 
   OPTIMIZATIONS
+    HDFS-8480. Fix performance and timeout issues in HDFS-7929 by using
+    hard-links to preserve old edit logs, instead of copying them. (Zhe Zhang
+    via Colin P. McCabe)
 
   BUG FIXES
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b424f93/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
index ee651fd..1f10bc4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -127,23 +128,8 @@ public abstract class NNUpgradeUtil {
 
     for (String s : fileNameList) {
       File prevFile = new File(tmpDir, s);
-      Preconditions.checkState(prevFile.canRead(),
-          "Edits log file " + s + " is not readable.");
       File newFile = new File(curDir, prevFile.getName());
-      Preconditions.checkState(newFile.createNewFile(),
-          "Cannot create new edits log file in " + curDir);
-      EditLogFileInputStream in = new EditLogFileInputStream(prevFile);
-      EditLogFileOutputStream out =
-          new EditLogFileOutputStream(conf, newFile, 512*1024);
-      FSEditLogOp logOp = in.nextValidOp();
-      while (logOp != null) {
-        out.write(logOp);
-        logOp = in.nextOp();
-      }
-      out.setReadyToFlush();
-      out.flushAndSync(true);
-      out.close();
-      in.close();
+      Files.createLink(newFile.toPath(), prevFile.toPath());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7b424f93/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
----------------------------------------------------------------------
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 544537c..8cc47c3 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
@@ -30,13 +30,16 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
-import java.util.LinkedList;
+import java.nio.file.Files;
 import java.util.List;
 import java.util.regex.Pattern;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.inotify.Event;
+import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -45,7 +48,11 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
@@ -54,6 +61,8 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
+
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 
@@ -466,31 +475,50 @@ public class TestDFSUpgrade {
     log("Normal NameNode upgrade", 1);
     File[] created =
         UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
-    List<String> beforeUpgrade = new LinkedList<>();
     for (final File createdDir : created) {
       List<String> fileNameList =
           IOUtils.listDirectory(createdDir, EditLogsFilter.INSTANCE);
-      beforeUpgrade.addAll(fileNameList);
+      for (String fileName : fileNameList) {
+        String tmpFileName = fileName + ".tmp";
+        File existingFile = new File(createdDir, fileName);
+        File tmpFile = new File(createdDir, tmpFileName);
+        Files.move(existingFile.toPath(), tmpFile.toPath());
+        File newFile = new File(createdDir, fileName);
+        Preconditions.checkState(newFile.createNewFile(),
+            "Cannot create new edits log file in " + createdDir);
+        EditLogFileInputStream in = new EditLogFileInputStream(tmpFile,
+            HdfsServerConstants.INVALID_TXID, HdfsServerConstants.INVALID_TXID,
+            false);
+        EditLogFileOutputStream out = new EditLogFileOutputStream(conf, newFile,
+            (int)tmpFile.length());
+        out.create(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION + 1);
+        FSEditLogOp logOp = in.readOp();
+        while (logOp != null) {
+          out.write(logOp);
+          logOp = in.readOp();
+        }
+        out.setReadyToFlush();
+        out.flushAndSync(true);
+        out.close();
+        Files.delete(tmpFile.toPath());
+      }
     }
 
     cluster = createCluster();
 
-    List<String> afterUpgrade = new LinkedList<>();
-    for (final File createdDir : created) {
-      List<String> fileNameList =
-          IOUtils.listDirectory(createdDir, EditLogsFilter.INSTANCE);
-      afterUpgrade.addAll(fileNameList);
-    }
-
-    for (String s : beforeUpgrade) {
-      assertTrue(afterUpgrade.contains(s));
-    }
-
+    DFSInotifyEventInputStream ieis =
+        cluster.getFileSystem().getInotifyEventStream(0);
+    EventBatch batch = ieis.poll();
+    Event[] events = batch.getEvents();
+    assertTrue("Should be able to get transactions before the upgrade.",
+        events.length > 0);
+    assertEquals(events[0].getEventType(), Event.EventType.CREATE);
+    assertEquals(((CreateEvent) events[0]).getPath(), "/TestUpgrade");
     cluster.shutdown();
     UpgradeUtilities.createEmptyDirs(nameNodeDirs);
   }
 
-  private static enum EditLogsFilter implements FilenameFilter {
+  private enum EditLogsFilter implements FilenameFilter {
     INSTANCE;
 
     @Override


[06/21] hadoop git commit: MAPREDUCE-5948. org.apache.hadoop.mapred.LineRecordReader does not handle multibyte record delimiters well. Contributed by Vinayakumar B, Rushabh Shah, and Akira AJISAKA

Posted by ar...@apache.org.
MAPREDUCE-5948. org.apache.hadoop.mapred.LineRecordReader does not handle multibyte record delimiters well. Contributed by Vinayakumar B, Rushabh Shah, and Akira AJISAKA


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/077250d8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/077250d8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/077250d8

Branch: refs/heads/HDFS-7240
Commit: 077250d8d7b4b757543a39a6ce8bb6e3be356c6f
Parents: 11ac848
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Jun 22 21:59:20 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Jun 22 21:59:20 2015 +0000

----------------------------------------------------------------------
 .../java/org/apache/hadoop/util/LineReader.java |   8 ++
 hadoop-mapreduce-project/CHANGES.txt            |   4 +
 .../apache/hadoop/mapred/LineRecordReader.java  |   4 +-
 .../mapreduce/lib/input/LineRecordReader.java   |   3 +-
 .../lib/input/UncompressedSplitLineReader.java  | 125 +++++++++++++++++++
 .../hadoop/mapred/TestLineRecordReader.java     |  77 +++++++++++-
 .../lib/input/TestLineRecordReader.java         |  79 ++++++++++--
 7 files changed, 286 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
index 3188cb5..1d1b569 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java
@@ -369,4 +369,12 @@ public class LineReader implements Closeable {
   public int readLine(Text str) throws IOException {
     return readLine(str, Integer.MAX_VALUE, Integer.MAX_VALUE);
   }
+
+  protected int getBufferPosn() {
+    return bufferPosn;
+  }
+
+  protected int getBufferSize() {
+    return bufferSize;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index bae2674..da1a2f3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -498,6 +498,10 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6403. Fix typo in the usage of NNBench.
     (Jagadesh Kiran N via aajisaka)
 
+    MAPREDUCE-5948. org.apache.hadoop.mapred.LineRecordReader does not handle
+    multibyte record delimiters well (Vinayakumar B, Rushabh Shah, and Akira
+    AJISAKA via jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
index 45263c4..9802697 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.io.compress.SplitCompressionInputStream;
 import org.apache.hadoop.io.compress.SplittableCompressionCodec;
 import org.apache.hadoop.mapreduce.lib.input.CompressedSplitLineReader;
 import org.apache.hadoop.mapreduce.lib.input.SplitLineReader;
+import org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.Log;
 
@@ -131,7 +132,8 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
       }
     } else {
       fileIn.seek(start);
-      in = new SplitLineReader(fileIn, job, recordDelimiter);
+      in = new UncompressedSplitLineReader(
+          fileIn, job, recordDelimiter, split.getLength());
       filePosition = fileIn;
     }
     // If this is not the first split, we always throw away first record

http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
index 5af8f43..9e1ca2a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java
@@ -112,7 +112,8 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
       }
     } else {
       fileIn.seek(start);
-      in = new SplitLineReader(fileIn, job, this.recordDelimiterBytes);
+      in = new UncompressedSplitLineReader(
+          fileIn, job, this.recordDelimiterBytes, split.getLength());
       filePosition = fileIn;
     }
     // If this is not the first split, we always throw away first record

http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
new file mode 100644
index 0000000..52fb7b0
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/UncompressedSplitLineReader.java
@@ -0,0 +1,125 @@
+/**
+ * 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.mapreduce.lib.input;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.io.Text;
+
+/**
+ * SplitLineReader for uncompressed files.
+ * This class can split the file correctly even if the delimiter is multi-bytes.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class UncompressedSplitLineReader extends SplitLineReader {
+  private boolean needAdditionalRecord = false;
+  private long splitLength;
+  /** Total bytes read from the input stream. */
+  private long totalBytesRead = 0;
+  private boolean finished = false;
+  private boolean usingCRLF;
+  private int unusedBytes = 0;
+  private int lastBytesRead = 0;
+
+  public UncompressedSplitLineReader(FSDataInputStream in, Configuration conf,
+      byte[] recordDelimiterBytes, long splitLength) throws IOException {
+    super(in, conf, recordDelimiterBytes);
+    this.splitLength = splitLength;
+    usingCRLF = (recordDelimiterBytes == null);
+  }
+
+  @Override
+  protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
+      throws IOException {
+    int maxBytesToRead = buffer.length;
+    if (totalBytesRead < splitLength) {
+      maxBytesToRead = Math.min(maxBytesToRead,
+                                (int)(splitLength - totalBytesRead));
+    }
+    int bytesRead = in.read(buffer, 0, maxBytesToRead);
+    lastBytesRead = bytesRead;
+
+    // If the split ended in the middle of a record delimiter then we need
+    // to read one additional record, as the consumer of the next split will
+    // not recognize the partial delimiter as a record.
+    // However if using the default delimiter and the next character is a
+    // linefeed then next split will treat it as a delimiter all by itself
+    // and the additional record read should not be performed.
+    if (totalBytesRead == splitLength && inDelimiter && bytesRead > 0) {
+      if (usingCRLF) {
+        needAdditionalRecord = (buffer[0] != '\n');
+      } else {
+        needAdditionalRecord = true;
+      }
+    }
+    if (bytesRead > 0) {
+      totalBytesRead += bytesRead;
+    }
+    return bytesRead;
+  }
+
+  @Override
+  public int readLine(Text str, int maxLineLength, int maxBytesToConsume)
+      throws IOException {
+    long bytesRead = 0;
+    if (!finished) {
+      // only allow at most one more record to be read after the stream
+      // reports the split ended
+      if (totalBytesRead > splitLength) {
+        finished = true;
+      }
+      bytesRead = totalBytesRead;
+      int bytesConsumed = super.readLine(str, maxLineLength, maxBytesToConsume);
+      bytesRead = totalBytesRead - bytesRead;
+
+      // No records left.
+      if (bytesConsumed == 0 && bytesRead == 0) {
+        return 0;
+      }
+
+      int bufferSize = getBufferSize();
+
+      // Add the remaining buffer size not used for the last call
+      // of fillBuffer method.
+      if (lastBytesRead <= 0) {
+        bytesRead += bufferSize;
+      } else if (bytesRead > 0) {
+        bytesRead += bufferSize - lastBytesRead;
+      }
+
+      // Adjust the size of the buffer not used for this record.
+      // The size is carried over for the next calculation.
+      bytesRead += unusedBytes;
+      unusedBytes = bufferSize - getBufferPosn();
+      bytesRead -= unusedBytes;
+    }
+    return (int) bytesRead;
+  }
+
+  @Override
+  public boolean needAdditionalRecordAfterSplit() {
+    return !finished && needAdditionalRecord;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
index cbbbeaa..a5c9933 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestLineRecordReader.java
@@ -25,13 +25,17 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
@@ -41,6 +45,9 @@ import org.apache.hadoop.io.compress.Decompressor;
 import org.junit.Test;
 
 public class TestLineRecordReader {
+  private static Path workDir = new Path(new Path(System.getProperty(
+      "test.build.data", "target"), "data"), "TestTextInputFormat");
+  private static Path inputDir = new Path(workDir, "input");
 
   private void testSplitRecords(String testFileName, long firstSplitLength)
       throws IOException {
@@ -50,15 +57,27 @@ public class TestLineRecordReader {
     long testFileSize = testFile.length();
     Path testFilePath = new Path(testFile.getAbsolutePath());
     Configuration conf = new Configuration();
+    testSplitRecordsForFile(conf, firstSplitLength, testFileSize, testFilePath);
+  }
+
+  private void testSplitRecordsForFile(Configuration conf,
+      long firstSplitLength, long testFileSize, Path testFilePath)
+      throws IOException {
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.
         LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-    assertTrue("unexpected test data at " + testFile,
+    assertTrue("unexpected test data at " + testFilePath,
         testFileSize > firstSplitLength);
 
+    String delimiter = conf.get("textinputformat.record.delimiter");
+    byte[] recordDelimiterBytes = null;
+    if (null != delimiter) {
+      recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8);
+    }
     // read the data without splitting to count the records
     FileSplit split = new FileSplit(testFilePath, 0, testFileSize,
         (String[])null);
-    LineRecordReader reader = new LineRecordReader(conf, split);
+    LineRecordReader reader = new LineRecordReader(conf, split,
+        recordDelimiterBytes);
     LongWritable key = new LongWritable();
     Text value = new Text();
     int numRecordsNoSplits = 0;
@@ -69,7 +88,7 @@ public class TestLineRecordReader {
 
     // count the records in the first split
     split = new FileSplit(testFilePath, 0, firstSplitLength, (String[])null);
-    reader = new LineRecordReader(conf, split);
+    reader = new LineRecordReader(conf, split, recordDelimiterBytes);
     int numRecordsFirstSplit = 0;
     while (reader.next(key,  value)) {
       ++numRecordsFirstSplit;
@@ -79,14 +98,14 @@ public class TestLineRecordReader {
     // count the records in the second split
     split = new FileSplit(testFilePath, firstSplitLength,
         testFileSize - firstSplitLength, (String[])null);
-    reader = new LineRecordReader(conf, split);
+    reader = new LineRecordReader(conf, split, recordDelimiterBytes);
     int numRecordsRemainingSplits = 0;
     while (reader.next(key,  value)) {
       ++numRecordsRemainingSplits;
     }
     reader.close();
 
-    assertEquals("Unexpected number of records in bzip2 compressed split",
+    assertEquals("Unexpected number of records in split",
         numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits);
   }
 
@@ -290,4 +309,52 @@ public class TestLineRecordReader {
     }
     assertEquals(10, decompressors.size());
   }
+
+  /**
+   * Writes the input test file
+   *
+   * @param conf
+   * @return Path of the file created
+   * @throws IOException
+   */
+  private Path createInputFile(Configuration conf, String data)
+      throws IOException {
+    FileSystem localFs = FileSystem.getLocal(conf);
+    Path file = new Path(inputDir, "test.txt");
+    Writer writer = new OutputStreamWriter(localFs.create(file));
+    try {
+      writer.write(data);
+    } finally {
+      writer.close();
+    }
+    return file;
+  }
+
+  @Test
+  public void testUncompressedInput() throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "abc+++def+++ghi+++"
+        + "jkl+++mno+++pqr+++stu+++vw +++xyz";
+    Path inputFile = createInputFile(conf, inputData);
+    conf.set("textinputformat.record.delimiter", "+++");
+    for(int bufferSize = 1; bufferSize <= inputData.length(); bufferSize++) {
+      for(int splitSize = 1; splitSize < inputData.length(); splitSize++) {
+        conf.setInt("io.file.buffer.size", bufferSize);
+        testSplitRecordsForFile(conf, splitSize, inputData.length(), inputFile);
+      }
+    }
+  }
+
+  @Test
+  public void testUncompressedInputContainingCRLF() throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "a\r\nb\rc\nd\r\n";
+    Path inputFile = createInputFile(conf, inputData);
+    for(int bufferSize = 1; bufferSize <= inputData.length(); bufferSize++) {
+      for(int splitSize = 1; splitSize < inputData.length(); splitSize++) {
+        conf.setInt("io.file.buffer.size", bufferSize);
+        testSplitRecordsForFile(conf, splitSize, inputData.length(), inputFile);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/077250d8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
index 8b385a0..3c1f28f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestLineRecordReader.java
@@ -25,13 +25,17 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CodecPool;
@@ -42,6 +46,9 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Test;
 
 public class TestLineRecordReader {
+  private static Path workDir = new Path(new Path(System.getProperty(
+      "test.build.data", "target"), "data"), "TestTextInputFormat");
+  private static Path inputDir = new Path(workDir, "input");
 
   private void testSplitRecords(String testFileName, long firstSplitLength)
       throws IOException {
@@ -51,17 +58,28 @@ public class TestLineRecordReader {
     long testFileSize = testFile.length();
     Path testFilePath = new Path(testFile.getAbsolutePath());
     Configuration conf = new Configuration();
+    testSplitRecordsForFile(conf, firstSplitLength, testFileSize, testFilePath);
+  }
+
+  private void testSplitRecordsForFile(Configuration conf,
+      long firstSplitLength, long testFileSize, Path testFilePath)
+      throws IOException {
     conf.setInt(org.apache.hadoop.mapreduce.lib.input.
         LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-    assertTrue("unexpected test data at " + testFile,
+    assertTrue("unexpected test data at " + testFilePath,
         testFileSize > firstSplitLength);
 
+    String delimiter = conf.get("textinputformat.record.delimiter");
+    byte[] recordDelimiterBytes = null;
+    if (null != delimiter) {
+      recordDelimiterBytes = delimiter.getBytes(Charsets.UTF_8);
+    }
     TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
 
     // read the data without splitting to count the records
     FileSplit split = new FileSplit(testFilePath, 0, testFileSize,
         (String[])null);
-    LineRecordReader reader = new LineRecordReader();
+    LineRecordReader reader = new LineRecordReader(recordDelimiterBytes);
     reader.initialize(split, context);
     int numRecordsNoSplits = 0;
     while (reader.nextKeyValue()) {
@@ -71,7 +89,7 @@ public class TestLineRecordReader {
 
     // count the records in the first split
     split = new FileSplit(testFilePath, 0, firstSplitLength, (String[])null);
-    reader = new LineRecordReader();
+    reader = new LineRecordReader(recordDelimiterBytes);
     reader.initialize(split, context);
     int numRecordsFirstSplit = 0;
     while (reader.nextKeyValue()) {
@@ -82,16 +100,15 @@ public class TestLineRecordReader {
     // count the records in the second split
     split = new FileSplit(testFilePath, firstSplitLength,
         testFileSize - firstSplitLength, (String[])null);
-    reader = new LineRecordReader();
+    reader = new LineRecordReader(recordDelimiterBytes);
     reader.initialize(split, context);
     int numRecordsRemainingSplits = 0;
     while (reader.nextKeyValue()) {
       ++numRecordsRemainingSplits;
     }
     reader.close();
-
-    assertEquals("Unexpected number of records in bzip2 compressed split",
-        numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits);
+    assertEquals("Unexpected number of records in split ", numRecordsNoSplits,
+        numRecordsFirstSplit + numRecordsRemainingSplits);
   }
 
   @Test
@@ -276,4 +293,52 @@ public class TestLineRecordReader {
     }
     assertEquals(10, decompressors.size());
   }
+
+  /**
+   * Writes the input test file
+   *
+   * @param conf
+   * @return Path of the file created
+   * @throws IOException
+   */
+  private Path createInputFile(Configuration conf, String data)
+      throws IOException {
+    FileSystem localFs = FileSystem.getLocal(conf);
+    Path file = new Path(inputDir, "test.txt");
+    Writer writer = new OutputStreamWriter(localFs.create(file));
+    try {
+      writer.write(data);
+    } finally {
+      writer.close();
+    }
+    return file;
+  }
+
+  @Test
+  public void testUncompressedInput() throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "abc+++def+++ghi+++"
+        + "jkl+++mno+++pqr+++stu+++vw +++xyz";
+    Path inputFile = createInputFile(conf, inputData);
+    conf.set("textinputformat.record.delimiter", "+++");
+    for(int bufferSize = 1; bufferSize <= inputData.length(); bufferSize++) {
+      for(int splitSize = 1; splitSize < inputData.length(); splitSize++) {
+        conf.setInt("io.file.buffer.size", bufferSize);
+        testSplitRecordsForFile(conf, splitSize, inputData.length(), inputFile);
+      }
+    }
+  }
+
+  @Test
+  public void testUncompressedInputContainingCRLF() throws Exception {
+    Configuration conf = new Configuration();
+    String inputData = "a\r\nb\rc\nd\r\n";
+    Path inputFile = createInputFile(conf, inputData);
+    for(int bufferSize = 1; bufferSize <= inputData.length(); bufferSize++) {
+      for(int splitSize = 1; splitSize < inputData.length(); splitSize++) {
+        conf.setInt("io.file.buffer.size", bufferSize);
+        testSplitRecordsForFile(conf, splitSize, inputData.length(), inputFile);
+      }
+    }
+  }
 }


[09/21] hadoop git commit: YARN-3835. hadoop-yarn-server-resourcemanager test package bundles core-site.xml, yarn-site.xml (vamsee via rkanter)

Posted by ar...@apache.org.
YARN-3835. hadoop-yarn-server-resourcemanager test package bundles core-site.xml, yarn-site.xml (vamsee via rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/99271b76
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/99271b76
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/99271b76

Branch: refs/heads/HDFS-7240
Commit: 99271b762129d78c86f3c9733a24c77962b0b3f7
Parents: 5ebf281
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Jun 22 18:02:27 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Jun 22 18:02:27 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../hadoop-yarn-server-resourcemanager/pom.xml                   | 4 ++++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/99271b76/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c3cf9d8..f235338 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -545,6 +545,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3802. Two RMNodes for the same NodeId are used in RM sometimes
     after NM is reconnected. (zhihai xu via xgong)
 
+    YARN-3835. hadoop-yarn-server-resourcemanager test package bundles
+    core-site.xml, yarn-site.xml (vamsee via rkanter)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/99271b76/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 4960f95..49a0bdb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -258,6 +258,10 @@
                   <mainClass>org.apache.hadoop.test.YarnTestDriver</mainClass>
                 </manifest>
               </archive>
+              <excludes>
+                <exclude>**/core-site.xml</exclude>
+                <exclude>**/yarn-site.xml</exclude>
+              </excludes>
             </configuration>
           </execution>
         </executions>


[10/21] hadoop git commit: MAPREDUCE-6406. Update FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT to match mapred-default.xml. Contributed by Ray Chiang.

Posted by ar...@apache.org.
MAPREDUCE-6406. Update
FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT to match
mapred-default.xml. Contributed by Ray Chiang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/41ae7768
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/41ae7768
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/41ae7768

Branch: refs/heads/HDFS-7240
Commit: 41ae7768ebb8637c09c7f9733bcfe9dc43dbce69
Parents: 99271b7
Author: Devaraj K <de...@apache.org>
Authored: Tue Jun 23 10:23:44 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Tue Jun 23 10:23:44 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java   | 2 +-
 .../src/main/resources/mapred-default.xml                         | 2 +-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ae7768/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index da1a2f3..0cefde0 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -200,6 +200,9 @@ Trunk (Unreleased)
     MAPREDUCE-6396. TestPipeApplication fails by NullPointerException.
     (Brahma Reddy Battula via aajisaka)
 
+    MAPREDUCE-6406. Update FileOutputCommitter.FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT
+    to match mapred-default.xml. (Ray Chiang via devaraj)
+
   BREAKDOWN OF MAPREDUCE-2841 (NATIVE TASK) SUBTASKS
 
     MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ae7768/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
index 6e5d0a1..1e5d71a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
@@ -63,7 +63,7 @@ public class FileOutputCommitter extends OutputCommitter {
       "mapreduce.fileoutputcommitter.marksuccessfuljobs";
   public static final String FILEOUTPUTCOMMITTER_ALGORITHM_VERSION =
       "mapreduce.fileoutputcommitter.algorithm.version";
-  public static final int FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT = 1;
+  public static final int FILEOUTPUTCOMMITTER_ALGORITHM_VERSION_DEFAULT = 2;
   private Path outputPath = null;
   private Path workPath = null;
   private final int algorithmVersion;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ae7768/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 3789429..ba63c02 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1326,7 +1326,7 @@
   <value>2</value>
   <description>The file output committer algorithm version
   valid algorithm version number: 1 or 2
-  default to 1, which is the original algorithm
+  default to 2, which is the original algorithm
 
   In algorithm version 1,
 


[20/21] hadoop git commit: YARN-3790. usedResource from rootQueue metrics may get stale data for FS scheduler after recovering the container (Zhihai Xu via rohithsharmaks)

Posted by ar...@apache.org.
YARN-3790. usedResource from rootQueue metrics may get stale data for FS scheduler after recovering the container (Zhihai Xu via rohithsharmaks)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dd4b387d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dd4b387d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dd4b387d

Branch: refs/heads/HDFS-7240
Commit: dd4b387d96abc66ddebb569b3775b18b19aed027
Parents: 8d58512
Author: rohithsharmaks <ro...@apache.org>
Authored: Wed Jun 24 23:00:14 2015 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Wed Jun 24 23:00:14 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                          |  3 +++
 .../resourcemanager/scheduler/fair/FairScheduler.java    | 11 +++++++----
 2 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4b387d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a5fc86b..9547f0f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -705,6 +705,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3832. Resource Localization fails on a cluster due to existing cache
     directories (Brahma Reddy Battula via jlowe)
 
+    YARN-3790. usedResource from rootQueue metrics may get stale data for FS
+    scheduler after recovering the container (Zhihai Xu via rohithsharmaks)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4b387d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 2ed3b2a..cbc10e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
@@ -841,11 +842,11 @@ public class FairScheduler extends
         + " with event: " + event);
   }
 
-  private synchronized void addNode(RMNode node) {
+  private synchronized void addNode(List<NMContainerStatus> containerReports,
+      RMNode node) {
     FSSchedulerNode schedulerNode = new FSSchedulerNode(node, usePortForNodeName);
     nodes.put(node.getNodeID(), schedulerNode);
     Resources.addTo(clusterResource, node.getTotalCapability());
-    updateRootQueueMetrics();
     updateMaximumAllocation(schedulerNode, true);
 
     triggerUpdate();
@@ -854,6 +855,9 @@ public class FairScheduler extends
     queueMgr.getRootQueue().recomputeSteadyShares();
     LOG.info("Added node " + node.getNodeAddress() +
         " cluster capacity: " + clusterResource);
+
+    recoverContainersOnNode(containerReports, node);
+    updateRootQueueMetrics();
   }
 
   private synchronized void removeNode(RMNode rmNode) {
@@ -1147,8 +1151,7 @@ public class FairScheduler extends
         throw new RuntimeException("Unexpected event type: " + event);
       }
       NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent)event;
-      addNode(nodeAddedEvent.getAddedRMNode());
-      recoverContainersOnNode(nodeAddedEvent.getContainerReports(),
+      addNode(nodeAddedEvent.getContainerReports(),
           nodeAddedEvent.getAddedRMNode());
       break;
     case NODE_REMOVED:


[18/21] hadoop git commit: YARN-3809. Failed to launch new attempts because ApplicationMasterLauncher's threads all hang. Contributed by Jun Gong

Posted by ar...@apache.org.
YARN-3809. Failed to launch new attempts because ApplicationMasterLauncher's threads all hang. Contributed by Jun Gong


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2a20dd9b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2a20dd9b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2a20dd9b

Branch: refs/heads/HDFS-7240
Commit: 2a20dd9b61ba3833460cbda0e8c3e8b6366fc3ab
Parents: 72d08a0
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 24 16:23:48 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 24 16:23:48 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/conf/YarnConfiguration.java     | 10 +++++++
 .../src/main/resources/yarn-default.xml         | 12 ++++++++
 .../amlauncher/ApplicationMasterLauncher.java   | 30 ++++++++++++++++++--
 4 files changed, 52 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f235338..7ecdee3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -699,6 +699,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3842. NMProxy should retry on NMNotYetReadyException. 
     (Robert Kanter via kasha)
 
+    YARN-3809. Failed to launch new attempts because
+    ApplicationMasterLauncher's threads all hang (Jun Gong via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 5d75a21..6b660f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -143,6 +143,16 @@ public class YarnConfiguration extends Configuration {
     RM_PREFIX + "client.thread-count";
   public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 50;
 
+  /** Number of threads used to launch/cleanup AM.*/
+  public static final String RM_AMLAUNCHER_THREAD_COUNT =
+      RM_PREFIX + "amlauncher.thread-count";
+  public static final int DEFAULT_RM_AMLAUNCHER_THREAD_COUNT = 50;
+
+  /** Retry times to connect with NM.*/
+  public static final String RM_NODEMANAGER_CONNECT_RETIRES =
+      RM_PREFIX + "nodemanager-connect-retries";
+  public static final int DEFAULT_RM_NODEMANAGER_CONNECT_RETIRES = 10;
+
   /** The Kerberos principal for the resource manager.*/
   public static final String RM_PRINCIPAL =
     RM_PREFIX + "principal";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index d94157c..621198c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -77,6 +77,18 @@
   </property>
 
   <property>
+    <description>Number of threads used to launch/cleanup AM.</description>
+    <name>yarn.resourcemanager.amlauncher.thread-count</name>
+    <value>50</value>
+  </property>
+
+  <property>
+    <description>Retry times to connect with NM.</description>
+    <name>yarn.resourcemanager.nodemanager-connect-retries</name>
+    <value>10</value>
+  </property>
+
+  <property>
     <description>The expiry interval for application master reporting.</description>
     <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
     <value>600000</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
index 5fc39fd..f606e45 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
@@ -19,12 +19,17 @@
 package org.apache.hadoop.yarn.server.resourcemanager.amlauncher;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 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.service.AbstractService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -34,7 +39,7 @@ public class ApplicationMasterLauncher extends AbstractService implements
     EventHandler<AMLauncherEvent> {
   private static final Log LOG = LogFactory.getLog(
       ApplicationMasterLauncher.class);
-  private final ThreadPoolExecutor launcherPool;
+  private ThreadPoolExecutor launcherPool;
   private LauncherThread launcherHandlingThread;
   
   private final BlockingQueue<Runnable> masterEvents
@@ -45,12 +50,31 @@ public class ApplicationMasterLauncher extends AbstractService implements
   public ApplicationMasterLauncher(RMContext context) {
     super(ApplicationMasterLauncher.class.getName());
     this.context = context;
-    this.launcherPool = new ThreadPoolExecutor(10, 10, 1, 
-        TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
     this.launcherHandlingThread = new LauncherThread();
   }
   
   @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    int threadCount = conf.getInt(
+        YarnConfiguration.RM_AMLAUNCHER_THREAD_COUNT,
+        YarnConfiguration.DEFAULT_RM_AMLAUNCHER_THREAD_COUNT);
+    ThreadFactory tf = new ThreadFactoryBuilder()
+        .setNameFormat("ApplicationMasterLauncher #%d")
+        .build();
+    launcherPool = new ThreadPoolExecutor(threadCount, threadCount, 1,
+        TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
+    launcherPool.setThreadFactory(tf);
+
+    Configuration newConf = new YarnConfiguration(conf);
+    newConf.setInt(CommonConfigurationKeysPublic.
+            IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        conf.getInt(YarnConfiguration.RM_NODEMANAGER_CONNECT_RETIRES,
+            YarnConfiguration.DEFAULT_RM_NODEMANAGER_CONNECT_RETIRES));
+    setConfig(newConf);
+    super.serviceInit(newConf);
+  }
+
+  @Override
   protected void serviceStart() throws Exception {
     launcherHandlingThread.start();
     super.serviceStart();


[12/21] hadoop git commit: HDFS-6564. Use slf4j instead of common-logging in hdfs-client. Contributed by Rakesh R.

Posted by ar...@apache.org.
HDFS-6564. Use slf4j instead of common-logging in hdfs-client. Contributed by Rakesh R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/122cad6a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/122cad6a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/122cad6a

Branch: refs/heads/HDFS-7240
Commit: 122cad6aec5839d8d515c5008425ecb34f2fa56b
Parents: d481684
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Jun 23 11:40:21 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Jun 23 11:41:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  | 10 ++++++++
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  7 +++--
 .../hdfs/protocol/BlockStoragePolicy.java       | 12 ++++-----
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |  3 ---
 .../org/apache/hadoop/hdfs/web/TokenAspect.java | 16 ++++++------
 .../hadoop/hdfs/web/URLConnectionFactory.java   | 13 +++++-----
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 27 ++++++++++----------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 8 files changed, 49 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index 33c2ed9..1b45095 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -35,6 +35,16 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index eda135e..0e72b98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -317,10 +317,9 @@ public class DFSUtilClient {
       if (address != null) {
         InetSocketAddress isa = NetUtils.createSocketAddr(address);
         if (isa.isUnresolved()) {
-          LOG.warn("Namenode for " + nsId +
-                       " remains unresolved for ID " + nnId +
-                   ".  Check your hdfs-site.xml file to " +
-                   "ensure namenodes are configured properly.");
+          LOG.warn("Namenode for {} remains unresolved for ID {}. Check your "
+              + "hdfs-site.xml file to ensure namenodes are configured "
+              + "properly.", nsId, nnId);
         }
         ret.put(nnId, isa);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
index 2624960..0225009 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
@@ -158,13 +158,11 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
     // remove excess storage types after fallback replacement.
     diff(storageTypes, excess, null);
     if (storageTypes.size() < expectedSize) {
-      LOG.warn("Failed to place enough replicas: expected size is " + expectedSize
-          + " but only " + storageTypes.size() + " storage types can be selected "
-          + "(replication=" + replication
-          + ", selected=" + storageTypes
-          + ", unavailable=" + unavailables
-          + ", removed=" + removed
-          + ", policy=" + this + ")");
+      LOG.warn("Failed to place enough replicas: expected size is {}"
+          + " but only {} storage types can be selected (replication={},"
+          + " selected={}, unavailable={}" + ", removed={}" + ", policy={}"
+          + ")", expectedSize, storageTypes.size(), replication, storageTypes,
+          unavailables, removed, this);
     }
     return storageTypes;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
index 61bbe38..41ec2f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
@@ -24,8 +24,6 @@ import javax.annotation.Nullable;
 
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
-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.fs.InvalidRequestException;
@@ -41,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class CachePoolInfo {
-  public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
 
   /**
    * Indicates that the pool does not have a maximum relative expiry.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
index bc3eb4b..a864d37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 
-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.fs.DelegationTokenRenewer;
@@ -37,6 +35,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -115,11 +115,11 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   private final DTSelecorByKind dtSelector;
   private final T fs;
   private boolean hasInitedToken;
-  private final Log LOG;
+  private final Logger LOG;
   private final Text serviceName;
 
   TokenAspect(T fs, final Text serviceName, final Text kind) {
-    this.LOG = LogFactory.getLog(fs.getClass());
+    this.LOG = LoggerFactory.getLogger(fs.getClass());
     this.fs = fs;
     this.dtSelector = new DTSelecorByKind(kind);
     this.serviceName = serviceName;
@@ -134,8 +134,8 @@ final class TokenAspect<T extends FileSystem & Renewable> {
       if (token != null) {
         fs.setDelegationToken(token);
         addRenewAction(fs);
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Created new DT for " + token.getService());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Created new DT for {}", token.getService());
         }
       }
       hasInitedToken = true;
@@ -149,8 +149,8 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   synchronized void initDelegationToken(UserGroupInformation ugi) {
     Token<?> token = selectDelegationToken(ugi);
     if (token != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Found existing DT for " + token.getService());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found existing DT for {}", token.getService());
       }
       fs.setDelegationToken(token);
       hasInitedToken = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index e330adf..a5e02f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
@@ -28,8 +28,6 @@ import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.SSLSocketFactory;
 
-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;
@@ -38,6 +36,8 @@ import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -47,7 +47,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.LimitedPrivate({ "HDFS" })
 @InterfaceStability.Unstable
 public class URLConnectionFactory {
-  private static final Log LOG = LogFactory.getLog(URLConnectionFactory.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(URLConnectionFactory.class);
 
   /**
    * Timeout for socket connects and reads
@@ -154,16 +155,14 @@ public class URLConnectionFactory {
       throws IOException, AuthenticationException {
     if (isSpnego) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("open AuthenticatedURL connection" + url);
+        LOG.debug("open AuthenticatedURL connection {}", url);
       }
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
       return new AuthenticatedURL(new KerberosUgiAuthenticator(),
           connConfigurator).openConnection(url, authToken);
     } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("open URL connection");
-      }
+      LOG.debug("open URL connection");
       URLConnection connection = url.openConnection();
       if (connection instanceof HttpURLConnection) {
         connConfigurator.configure((HttpURLConnection) connection);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index d902738..2650dca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -36,8 +36,6 @@ import java.util.StringTokenizer;
 
 import javax.ws.rs.core.MediaType;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -81,6 +79,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelect
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -89,7 +89,8 @@ import com.google.common.collect.Lists;
 /** A FileSystem for HDFS over the web. */
 public class WebHdfsFileSystem extends FileSystem
     implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
-  public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
+  public static final Logger LOG = LoggerFactory
+      .getLogger(WebHdfsFileSystem.class);
   /** WebHdfs version. */
   public static final int VERSION = 1;
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
@@ -221,14 +222,14 @@ public class WebHdfsFileSystem extends FileSystem
       // to get another token to match hdfs/rpc behavior
       if (token != null) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("Using UGI token: " + token);
+          LOG.debug("Using UGI token: {}", token);
         }
         canRefreshDelegationToken = false;
       } else {
         token = getDelegationToken(null);
         if (token != null) {
           if(LOG.isDebugEnabled()) {
-            LOG.debug("Fetched new token: " + token);
+            LOG.debug("Fetched new token: {}", token);
           }
         } else { // security is disabled
           canRefreshDelegationToken = false;
@@ -245,7 +246,7 @@ public class WebHdfsFileSystem extends FileSystem
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
       if(LOG.isDebugEnabled()) {
-        LOG.debug("Replaced expired token: " + token);
+        LOG.debug("Replaced expired token: {}", token);
       }
       setDelegationToken(token);
       replaced = (token != null);
@@ -430,7 +431,7 @@ public class WebHdfsFileSystem extends FileSystem
     final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("url=" + url);
+      LOG.trace("url={}", url);
     }
     return url;
   }
@@ -467,7 +468,7 @@ public class WebHdfsFileSystem extends FileSystem
         + Param.toSortedString("&", parameters);
     final URL url = getNamenodeURL(path, query);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("url=" + url);
+      LOG.trace("url={}", url);
     }
     return url;
   }
@@ -658,9 +659,9 @@ public class WebHdfsFileSystem extends FileSystem
               a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
 
           if (isRetry || isFailoverAndRetry) {
-            LOG.info("Retrying connect to namenode: " + nnAddr
-                + ". Already tried " + retry + " time(s); retry policy is "
-                + retryPolicy + ", delay " + a.delayMillis + "ms.");
+            LOG.info("Retrying connect to namenode: {}. Already tried {}"
+                + " time(s); retry policy is {}, delay {}ms.", nnAddr, retry,
+                retryPolicy, a.delayMillis);
 
             if (isFailoverAndRetry) {
               resetStateToFailOver();
@@ -757,7 +758,7 @@ public class WebHdfsFileSystem extends FileSystem
         final IOException ioe =
             new IOException("Response decoding failure: "+e.toString(), e);
         if (LOG.isDebugEnabled()) {
-          LOG.debug(ioe);
+          LOG.debug("Response decoding failure: {}", e.toString(), e);
         }
         throw ioe;
       } finally {
@@ -1212,7 +1213,7 @@ public class WebHdfsFileSystem extends FileSystem
       }
     } catch (IOException ioe) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Token cancel failed: " + ioe);
+        LOG.debug("Token cancel failed: ", ioe);
       }
     } finally {
       super.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 4bd0e8b..981ca55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -658,6 +658,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8192. Eviction should key off used locked memory instead of
     ram disk free space. (Arpit Agarwal)
 
+    HDFS-6564. Use slf4j instead of common-logging in hdfs-client.
+    (Rakesh R via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[08/21] hadoop git commit: YARN-3842. NMProxy should retry on NMNotYetReadyException. (Robert Kanter via kasha)

Posted by ar...@apache.org.
YARN-3842. NMProxy should retry on NMNotYetReadyException. (Robert Kanter via kasha)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5ebf2817
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5ebf2817
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5ebf2817

Branch: refs/heads/HDFS-7240
Commit: 5ebf2817e58e1be8214dc1916a694a912075aa0a
Parents: fac4e04
Author: Karthik Kambatla <ka...@apache.org>
Authored: Mon Jun 22 17:45:41 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Mon Jun 22 17:47:47 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../apache/hadoop/yarn/client/ServerProxy.java  |  2 ++
 .../containermanager/TestNMProxy.java           | 20 +++++++++++++++++++-
 3 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ebf2817/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b50f490..c3cf9d8 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -693,6 +693,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl
     (Varun Saxena via xgong)
 
+    YARN-3842. NMProxy should retry on NMNotYetReadyException. 
+    (Robert Kanter via kasha)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ebf2817/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
index 6024560..e9bcf8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ServerProxy.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 
 import com.google.common.base.Preconditions;
@@ -74,6 +75,7 @@ public class ServerProxy {
     exceptionToPolicyMap.put(UnknownHostException.class, retryPolicy);
     exceptionToPolicyMap.put(RetriableException.class, retryPolicy);
     exceptionToPolicyMap.put(SocketException.class, retryPolicy);
+    exceptionToPolicyMap.put(NMNotYetReadyException.class, retryPolicy);
 
     return RetryPolicies.retryByException(RetryPolicies.TRY_ONCE_THEN_FAIL,
       exceptionToPolicyMap);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ebf2817/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
index 67f540c..0b372be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestNMProxy.java
@@ -54,6 +54,7 @@ public class TestNMProxy extends BaseContainerManagerTest {
   }
 
   int retryCount = 0;
+  boolean shouldThrowNMNotYetReadyException = false;
 
   @Before
   public void setUp() throws Exception {
@@ -72,7 +73,15 @@ public class TestNMProxy extends BaseContainerManagerTest {
           StartContainersRequest requests) throws YarnException, IOException {
         if (retryCount < 5) {
           retryCount++;
-          throw new java.net.ConnectException("start container exception");
+          if (shouldThrowNMNotYetReadyException) {
+            // This causes super to throw an NMNotYetReadyException
+            containerManager.setBlockNewContainerRequests(true);
+          } else {
+            throw new java.net.ConnectException("start container exception");
+          }
+        } else {
+          // This stops super from throwing an NMNotYetReadyException
+          containerManager.setBlockNewContainerRequests(false);
         }
         return super.startContainers(requests);
       }
@@ -126,16 +135,25 @@ public class TestNMProxy extends BaseContainerManagerTest {
         NMProxy.createNMProxy(conf, ContainerManagementProtocol.class, ugi,
           YarnRPC.create(conf), address);
 
+    retryCount = 0;
+    shouldThrowNMNotYetReadyException = false;
     proxy.startContainers(allRequests);
     Assert.assertEquals(5, retryCount);
 
     retryCount = 0;
+    shouldThrowNMNotYetReadyException = false;
     proxy.stopContainers(Records.newRecord(StopContainersRequest.class));
     Assert.assertEquals(5, retryCount);
 
     retryCount = 0;
+    shouldThrowNMNotYetReadyException = false;
     proxy.getContainerStatuses(Records
       .newRecord(GetContainerStatusesRequest.class));
     Assert.assertEquals(5, retryCount);
+
+    retryCount = 0;
+    shouldThrowNMNotYetReadyException = true;
+    proxy.startContainers(allRequests);
+    Assert.assertEquals(5, retryCount);
   }
 }


[21/21] hadoop git commit: Merge remote-tracking branch 'apache-commit/trunk' into ozone

Posted by ar...@apache.org.
Merge remote-tracking branch 'apache-commit/trunk' into ozone


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/845a7106
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/845a7106
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/845a7106

Branch: refs/heads/HDFS-7240
Commit: 845a71063178bff240e14c6943bdcec94649e6dc
Parents: f08bf36 dd4b387
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Jun 24 10:45:56 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Wed Jun 24 10:45:56 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ha/ZKFailoverController.java  |  61 ++-
 .../java/org/apache/hadoop/util/LineReader.java |   8 +
 .../org/apache/hadoop/ha/MiniZKFCCluster.java   |  93 +++-
 .../hadoop/ha/TestZKFailoverController.java     |  32 ++
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |  10 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   7 +-
 .../hdfs/protocol/BlockStoragePolicy.java       |  12 +-
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |   3 -
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  10 +
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |  16 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |  13 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  60 ++-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  21 +-
 .../bkjournal/TestBookKeeperHACheckpoints.java  |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |  36 +-
 .../token/block/BlockTokenSecretManager.java    |  40 +-
 .../server/blockmanagement/BlockManager.java    |  43 +-
 .../hdfs/server/namenode/CheckpointConf.java    |  14 +-
 .../hdfs/server/namenode/ImageServlet.java      |  88 +++-
 .../hdfs/server/namenode/NNUpgradeUtil.java     |  18 +-
 .../server/namenode/NameNodeHttpServer.java     |   7 +-
 .../hdfs/server/namenode/TransferFsImage.java   |  47 +-
 .../server/namenode/ha/BootstrapStandby.java    |  94 ++--
 .../hdfs/server/namenode/ha/EditLogTailer.java  | 162 +++++--
 .../server/namenode/ha/RemoteNameNodeInfo.java  | 133 ++++++
 .../server/namenode/ha/StandbyCheckpointer.java | 182 +++++---
 .../web/resources/NamenodeWebHdfsMethods.java   |  12 +-
 .../hdfs/tools/DFSZKFailoverController.java     |  13 +
 .../src/main/resources/hdfs-default.xml         |  20 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 464 ++++++++++++-------
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |  18 +-
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |  58 ++-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |   6 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 109 +++--
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |  94 ++--
 .../security/token/block/TestBlockToken.java    |  10 +-
 .../hdfs/server/namenode/TestBackupNode.java    |   2 +-
 .../hdfs/server/namenode/TestCheckpoint.java    |   3 +-
 .../server/namenode/ha/HAStressTestHarness.java |  46 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |  60 ++-
 .../namenode/ha/TestBootstrapStandby.java       | 176 ++++---
 .../ha/TestBootstrapStandbyWithQJM.java         |  47 +-
 .../ha/TestDNFencingWithReplication.java        |   1 +
 .../server/namenode/ha/TestEditLogTailer.java   |  12 +-
 .../ha/TestFailoverWithBlockTokensEnabled.java  |  55 ++-
 .../server/namenode/ha/TestHAConfiguration.java |  49 +-
 .../namenode/ha/TestPipelinesFailover.java      | 110 +++--
 .../namenode/ha/TestRemoteNameNodeInfo.java     |  61 +++
 .../namenode/ha/TestSeveralNameNodes.java       | 179 +++++++
 .../namenode/ha/TestStandbyCheckpoints.java     | 106 +++--
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |  59 +++
 .../hdfs/web/TestWebHdfsFileSystemContract.java |   2 +-
 .../hadoop/test/MiniDFSClusterManager.java      |   4 +
 .../src/test/resources/hadoop-0.23-reserved.tgz | Bin 4558 -> 5590 bytes
 .../src/test/resources/hadoop-1-reserved.tgz    | Bin 2572 -> 3348 bytes
 .../src/test/resources/hadoop-2-reserved.tgz    | Bin 2838 -> 3465 bytes
 .../src/test/resources/hadoop-22-dfs-dir.tgz    | Bin 318180 -> 413239 bytes
 .../src/test/resources/hadoop1-bbw.tgz          | Bin 40234 -> 43294 bytes
 .../src/test/resources/log4j.properties         |   2 +-
 hadoop-mapreduce-project/CHANGES.txt            |  20 +
 .../mapreduce/v2/app/webapp/JobBlock.java       |   2 +
 .../mapreduce/v2/app/webapp/dao/JobInfo.java    |   6 +
 .../v2/app/webapp/TestAMWebServicesJobs.java    |   2 +-
 .../apache/hadoop/mapred/LineRecordReader.java  |   4 +-
 .../mapreduce/lib/input/LineRecordReader.java   |   3 +-
 .../lib/input/UncompressedSplitLineReader.java  | 125 +++++
 .../lib/output/FileOutputCommitter.java         |   2 +-
 .../hadoop/mapreduce/task/reduce/Fetcher.java   |   1 +
 .../task/reduce/IFileWrappedMapOutput.java      |  10 +-
 .../mapreduce/task/reduce/LocalFetcher.java     |  15 +-
 .../src/main/resources/mapred-default.xml       |   2 +-
 .../hadoop/mapred/TestLineRecordReader.java     |  77 ++-
 .../lib/input/TestLineRecordReader.java         |  79 +++-
 .../mapreduce/v2/hs/server/HSAdminServer.java   |  43 +-
 .../v2/hs/server/TestHSAdminServer.java         |  55 +++
 .../java/org/apache/hadoop/hdfs/NNBench.java    |   3 +-
 hadoop-yarn-project/CHANGES.txt                 |  18 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +
 .../apache/hadoop/yarn/client/ServerProxy.java  |   2 +
 .../src/main/resources/yarn-default.xml         |  12 +
 .../localizer/ResourceLocalizationService.java  |  31 +-
 .../containermanager/TestNMProxy.java           |  20 +-
 .../TestResourceLocalizationService.java        |  10 +-
 .../hadoop-yarn-server-resourcemanager/pom.xml  |   4 +
 .../amlauncher/ApplicationMasterLauncher.java   |  30 +-
 .../scheduler/fair/FairScheduler.java           |  11 +-
 87 files changed, 2716 insertions(+), 880 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/845a7106/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 6598892,7d3a678..4b760b7
--- 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
@@@ -17,14 -17,34 +17,38 @@@
   */
  package org.apache.hadoop.hdfs.server.blockmanagement;
  
+ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+ import static org.apache.hadoop.util.ExitUtil.terminate;
+ 
+ import java.io.IOException;
+ import java.io.PrintWriter;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Queue;
+ import java.util.Set;
+ import java.util.TreeMap;
+ import java.util.TreeSet;
+ import java.util.concurrent.ThreadLocalRandom;
+ import java.util.concurrent.atomic.AtomicLong;
+ 
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Preconditions;
 +import com.google.common.collect.Sets;
  import org.apache.hadoop.HadoopIllegalArgumentException;
  import org.apache.hadoop.classification.InterfaceAudience;
  import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FileEncryptionInfo;
  import org.apache.hadoop.fs.StorageType;
+ import org.apache.hadoop.hdfs.DFSUtilClient;
+ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
  import org.apache.hadoop.hdfs.DFSConfigKeys;
  import org.apache.hadoop.hdfs.DFSUtil;
  import org.apache.hadoop.hdfs.HAUtil;


[13/21] hadoop git commit: HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
----------------------------------------------------------------------
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
index c4a2988..62643ae 100644
--- 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
@@ -23,10 +23,12 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.util.Collection;
 
+import com.google.common.base.Joiner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -58,19 +60,23 @@ public class TestHAConfiguration {
     }
   }
 
-  private Configuration getHAConf(String nsId, String host1, String host2) {
+  private Configuration getHAConf(String nsId, String ... hosts) {
     Configuration conf = new Configuration();
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, nsId);
-    conf.set(DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, nsId),
-        "nn1,nn2");    
     conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
+
+    String[] nnids = new String[hosts.length];
+    for (int i = 0; i < hosts.length; i++) {
+      String nnid = "nn" + (i + 1);
+      nnids[i] = nnid;
+      conf.set(DFSUtil.addKeySuffixes(
+              DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnid),
+          hosts[i] + ":12345");
+    }
+
     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");
+            DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, nsId),
+        Joiner.on(',').join(nnids));
     return conf;
   }
 
@@ -87,11 +93,28 @@ public class TestHAConfiguration {
     // 0.0.0.0, it should substitute the address from the RPC configuration
     // above.
     StandbyCheckpointer checkpointer = new StandbyCheckpointer(conf, fsn);
-    assertEquals(new URL("http", "1.2.3.2",
-        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""),
-        checkpointer.getActiveNNAddress());
+    assertAddressMatches("1.2.3.2", checkpointer.getActiveNNAddresses().get(0));
+
+    //test when there are three NNs
+    // Use non-local addresses to avoid host address matching
+    conf = getHAConf("ns1", "1.2.3.1", "1.2.3.2", "1.2.3.3");
+
+    // This is done by the NN before the StandbyCheckpointer is created
+    NameNode.initializeGenericKeys(conf, "ns1", "nn1");
+
+    checkpointer = new StandbyCheckpointer(conf, fsn);
+    assertEquals("Got an unexpected number of possible active NNs", 2, checkpointer
+        .getActiveNNAddresses().size());
+    assertEquals(new URL("http", "1.2.3.2", DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""),
+        checkpointer.getActiveNNAddresses().get(0));
+    assertAddressMatches("1.2.3.2", checkpointer.getActiveNNAddresses().get(0));
+    assertAddressMatches("1.2.3.3", checkpointer.getActiveNNAddresses().get(1));
   }
-  
+
+  private void assertAddressMatches(String address, URL url) throws MalformedURLException {
+    assertEquals(new URL("http", address, DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""), url);
+  }
+
   /**
    * Tests that the namenode edits dirs and shared edits dirs are gotten with
    * duplicates removed

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
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
index 76a62ff..3da37f5 100644
--- 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
@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -81,24 +82,33 @@ public class TestPipelinesFailover {
   
   private static final int STRESS_NUM_THREADS = 25;
   private static final int STRESS_RUNTIME = 40000;
-  
+
+  private static final int NN_COUNT = 3;
+  private static final long FAILOVER_SEED = System.currentTimeMillis();
+  private static final Random failoverRandom = new Random(FAILOVER_SEED);
+  static{
+    // log the failover seed so we can reproduce the test exactly
+    LOG.info("Using random seed: " + FAILOVER_SEED
+        + " for selecting active target NN during failover");
+  }
+
   enum TestScenario {
     GRACEFUL_FAILOVER {
       @Override
-      void run(MiniDFSCluster cluster) throws IOException {
-        cluster.transitionToStandby(0);
-        cluster.transitionToActive(1);
+      void run(MiniDFSCluster cluster, int previousActive, int activeIndex) throws IOException {
+        cluster.transitionToStandby(previousActive);
+        cluster.transitionToActive(activeIndex);
       }
     },
     ORIGINAL_ACTIVE_CRASHED {
       @Override
-      void run(MiniDFSCluster cluster) throws IOException {
-        cluster.restartNameNode(0);
-        cluster.transitionToActive(1);
+      void run(MiniDFSCluster cluster, int previousActive, int activeIndex) throws IOException {
+        cluster.restartNameNode(previousActive);
+        cluster.transitionToActive(activeIndex);
       }
     };
 
-    abstract void run(MiniDFSCluster cluster) throws IOException;
+    abstract void run(MiniDFSCluster cluster, int previousActive, int activeIndex) throws IOException;
   }
   
   enum MethodToTestIdempotence {
@@ -135,10 +145,7 @@ public class TestPipelinesFailover {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
     
     FSDataOutputStream stm = null;
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
-      .numDataNodes(3)
-      .build();
+    MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       int sizeWritten = 0;
       
@@ -157,15 +164,15 @@ public class TestPipelinesFailover {
       // Make sure all of the blocks are written out before failover.
       stm.hflush();
 
-      LOG.info("Failing over to NN 1");
-      scenario.run(cluster);
+      LOG.info("Failing over to another NN");
+      int activeIndex = failover(cluster, scenario);
 
       // NOTE: explicitly do *not* make any further metadata calls
       // to the NN here. The next IPC call should be to allocate the next
       // block. Any other call would notice the failover and not test
       // idempotence of the operation (HDFS-3031)
       
-      FSNamesystem ns1 = cluster.getNameNode(1).getNamesystem();
+      FSNamesystem ns1 = cluster.getNameNode(activeIndex).getNamesystem();
       BlockManagerTestUtil.updateState(ns1.getBlockManager());
       assertEquals(0, ns1.getPendingReplicationBlocks());
       assertEquals(0, ns1.getCorruptReplicaBlocks());
@@ -213,10 +220,7 @@ public class TestPipelinesFailover {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     
     FSDataOutputStream stm = null;
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
-      .numDataNodes(5)
-      .build();
+    MiniDFSCluster cluster = newMiniCluster(conf, 5);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -232,8 +236,7 @@ public class TestPipelinesFailover {
       // Make sure all the blocks are written before failover
       stm.hflush();
 
-      LOG.info("Failing over to NN 1");
-      scenario.run(cluster);
+      int nextActive = failover(cluster, scenario);
 
       assertTrue(fs.exists(TEST_PATH));
       
@@ -242,9 +245,9 @@ public class TestPipelinesFailover {
       // 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(1);
+
+      LOG.info("Failing back from NN " + nextActive + " to NN 0");
+      cluster.transitionToStandby(nextActive);
       cluster.transitionToActive(0);
       
       cluster.stopDataNode(1);
@@ -262,7 +265,7 @@ public class TestPipelinesFailover {
       cluster.shutdown();
     }
   }
-  
+
   /**
    * Tests lease recovery if a client crashes. This approximates the
    * use case of HBase WALs being recovered after a NN failover.
@@ -275,10 +278,7 @@ public class TestPipelinesFailover {
     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();
+    final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -329,10 +329,7 @@ public class TestPipelinesFailover {
     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();
+    final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -406,7 +403,20 @@ public class TestPipelinesFailover {
       cluster.shutdown();
     }
   }
-  
+
+  /**
+   * Create a MiniCluster with the specified base configuration and the specified number of
+   * DataNodes. Helper method to ensure that the we use the same number of NNs across all the tests.
+   * @return mini cluster ready to use
+   * @throws IOException cluster cannot be started
+   */
+  private MiniDFSCluster newMiniCluster(Configuration conf, int dnCount) throws IOException {
+    return new MiniDFSCluster.Builder(conf)
+             .nnTopology(MiniDFSNNTopology.simpleHATopology(NN_COUNT))
+             .numDataNodes(dnCount)
+             .build();
+  }
+
   /**
    * Stress test for pipeline/lease recovery. Starts a number of
    * threads, each of which creates a file and has another client
@@ -485,6 +495,38 @@ public class TestPipelinesFailover {
   }
 
   /**
+   * Fail-over using the given scenario, assuming NN0 is currently active
+   * @param cluster cluster on which to run the scenario
+   * @param scenario failure scenario to run
+   * @return the index of the new active NN
+   * @throws IOException
+   */
+  private int failover(MiniDFSCluster cluster, TestScenario scenario) throws IOException {
+    return failover(cluster, scenario, 0);
+  }
+
+  /**
+   * Do a fail-over with the given scenario.
+   * @param cluster cluster on which to run the scenario
+   * @param scenario failure scenario to run
+   * @param activeIndex index of the currently active node
+   * @throws IOException on failure
+   * @return the index of the new active NN
+   */
+  private int failover(MiniDFSCluster cluster, TestScenario scenario, int activeIndex)
+      throws IOException {
+    // get index of the next node that should be active, ensuring its not the same as the currently
+    // active node
+    int nextActive = failoverRandom.nextInt(NN_COUNT);
+    if (nextActive == activeIndex) {
+      nextActive = (nextActive + 1) % NN_COUNT;
+    }
+    LOG.info("Failing over to a standby NN:" + nextActive + " from NN " + activeIndex);
+    scenario.run(cluster, activeIndex, nextActive);
+    return nextActive;
+  }
+
+  /**
    * 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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java
new file mode 100644
index 0000000..cb2a4fc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java
@@ -0,0 +1,61 @@
+/**
+ * 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.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test that we correctly obtain remote namenode information
+ */
+public class TestRemoteNameNodeInfo {
+
+  @Test
+  public void testParseMultipleNameNodes() throws Exception {
+    // start with an empty configuration
+    Configuration conf = new Configuration(false);
+
+    // add in keys for each of the NNs
+    String nameservice = "ns1";
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+        .addNameservice(new MiniDFSNNTopology.NSConf(nameservice)
+            .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001))
+            .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002))
+            .addNN(new MiniDFSNNTopology.NNConf("nn3").setIpcPort(10003)));
+
+    // add the configurations of the NNs to the passed conf, so we can parse it back out
+    MiniDFSCluster.configureNameNodes(topology, false, conf);
+
+    // set the 'local' one as nn1
+    conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
+
+    List<RemoteNameNodeInfo> nns = RemoteNameNodeInfo.getRemoteNameNodes(conf);
+
+    // make sure it matches when we pass in the nameservice
+    List<RemoteNameNodeInfo> nns2 = RemoteNameNodeInfo.getRemoteNameNodes(conf,
+        nameservice);
+    assertEquals(nns, nns2);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java
new file mode 100644
index 0000000..dbe8070
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java
@@ -0,0 +1,179 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+import org.junit.Test;
+
+/**
+ * Test that we can start several and run with namenodes on the same minicluster
+ */
+public class TestSeveralNameNodes {
+
+  private static final Log LOG = LogFactory.getLog(TestSeveralNameNodes.class);
+
+  /** ms between failovers between NNs */
+  private static final int TIME_BETWEEN_FAILOVERS = 200;
+  private static final int NUM_NAMENODES = 3;
+  private static final int NUM_THREADS = 3;
+  private static final int LIST_LENGTH = 50;
+  /** ms for length of test */
+  private static final long RUNTIME = 100000;
+
+  @Test
+  public void testCircularLinkedListWrites() throws Exception {
+    HAStressTestHarness harness = new HAStressTestHarness();
+    // setup the harness
+    harness.setNumberOfNameNodes(NUM_NAMENODES);
+    harness.addFailoverThread(TIME_BETWEEN_FAILOVERS);
+
+    final MiniDFSCluster cluster = harness.startCluster();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+
+      // setup the a circular writer
+      FileSystem fs = harness.getFailoverFs();
+      TestContext context = harness.testCtx;
+      List<CircularWriter> writers = new ArrayList<CircularWriter>();
+      for (int i = 0; i < NUM_THREADS; i++) {
+        Path p = new Path("/test-" + i);
+        fs.mkdirs(p);
+        CircularWriter writer = new CircularWriter(context, LIST_LENGTH, fs, p);
+        writers.add(writer);
+        context.addThread(writer);
+      }
+      harness.startThreads();
+
+      // wait for all the writer threads to finish, or that we exceed the time
+      long start = System.currentTimeMillis();
+      while ((System.currentTimeMillis() - start) < RUNTIME) {
+        for (int i = 0; i < writers.size(); i++) {
+          CircularWriter writer = writers.get(i);
+          // remove the writer from the ones to check
+          if (writer.done.await(10, TimeUnit.MILLISECONDS)) {
+            writers.remove(i--);
+          }
+        }
+      }
+      assertEquals(
+          "Some writers didn't complete in expected runtime! Current writer state:"
+              + writers, 0,
+          writers.size());
+
+      harness.stopThreads();
+    } finally {
+      System.err.println("===========================\n\n\n\n");
+      harness.shutdown();
+    }
+  }
+
+  private static class CircularWriter extends RepeatingTestThread {
+
+    private final int maxLength;
+    private final Path dir;
+    private final FileSystem fs;
+    private int currentListIndex = 0;
+    private CountDownLatch done = new CountDownLatch(1);
+
+    public CircularWriter(TestContext context, int listLength, FileSystem fs,
+        Path parentDir) {
+      super(context);
+      this.fs = fs;
+      this.maxLength = listLength;
+      this.dir = parentDir;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder("Circular Writer:\n");
+      builder.append("\t directory: " + dir + "\n");
+      builder.append("\t target length: " + maxLength + "\n");
+      // might be a little racy, but we just want a close count
+      builder.append("\t current item: " + currentListIndex + "\n");
+      builder.append("\t done: " + (done.getCount() == 0) + "\n");
+      return builder.toString();
+    }
+
+    @Override
+    public void doAnAction() throws Exception {
+      if (currentListIndex == maxLength) {
+        checkList();
+        this.stopTestThread();
+        done.countDown();
+      } else {
+        writeList();
+      }
+    }
+
+    /**
+     * Make sure we can traverse the entire linked list
+     */
+    private void checkList() throws IOException {
+      for (int i = 0; i < maxLength; i++) {
+        Path nextFile = getNextFile(i);
+        if (!fs.exists(nextFile)) {
+          throw new RuntimeException("Next file " + nextFile
+              + " for list does not exist!");
+        }
+        // read the next file name
+        FSDataInputStream in = fs.open(nextFile);
+        nextFile = getNextFile(in.read());
+        in.close();
+      }
+
+    }
+
+    private void cleanup() throws IOException {
+      if (!fs.delete(dir, true)) {
+        throw new RuntimeException("Didn't correctly delete " + dir);
+      }
+      if (!fs.mkdirs(dir)) {
+        throw new RuntimeException("Didn't correctly make directory " + dir);
+      }
+    }
+
+    private void writeList() throws IOException {
+      Path nextPath = getNextFile(currentListIndex++);
+      LOG.info("Writing next file: " + nextPath);
+      FSDataOutputStream file = fs.create(nextPath);
+      file.write(currentListIndex);
+      file.close();
+    }
+
+    private Path getNextFile(int i) {
+      return new Path(dir, Integer.toString(i));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
----------------------------------------------------------------------
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
index 33af0e2..cd32502 100644
--- 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
@@ -63,8 +63,9 @@ import static org.junit.Assert.*;
 
 public class TestStandbyCheckpoints {
   private static final int NUM_DIRS_IN_LOG = 200000;
+  protected static int NUM_NNS = 3;
   protected MiniDFSCluster cluster;
-  protected NameNode nn0, nn1;
+  protected NameNode[] nns = new NameNode[NUM_NNS];
   protected FileSystem fs;
   private final Random random = new Random();
   protected File tmpOivImgDir;
@@ -88,7 +89,8 @@ public class TestStandbyCheckpoints {
         MiniDFSNNTopology topology = new MiniDFSNNTopology()
             .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
                 .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
-                .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
+                .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1))
+                .addNN(new MiniDFSNNTopology.NNConf("nn3").setHttpPort(basePort + 2)));
 
         cluster = new MiniDFSCluster.Builder(conf)
             .nnTopology(topology)
@@ -96,8 +98,8 @@ public class TestStandbyCheckpoints {
             .build();
         cluster.waitActive();
 
-        nn0 = cluster.getNameNode(0);
-        nn1 = cluster.getNameNode(1);
+        setNNs();
+
         fs = HATestUtil.configureFailoverFs(cluster, conf);
 
         cluster.transitionToActive(0);
@@ -110,6 +112,12 @@ public class TestStandbyCheckpoints {
     }
   }
 
+  protected void setNNs(){
+    for (int i = 0; i < NUM_NNS; i++) {
+      nns[i] = cluster.getNameNode(i);
+    }
+  }
+
   protected Configuration setupCommonConfig() {
     tmpOivImgDir = Files.createTempDir();
 
@@ -136,10 +144,10 @@ public class TestStandbyCheckpoints {
 
   @Test(timeout = 300000)
   public void testSBNCheckpoints() throws Exception {
-    JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nn1);
-    
+    JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nns[1]);
+
     doEdits(0, 10);
-    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    HATestUtil.waitForStandbyToCatchUp(nns[0], nns[1]);
     // 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(12));
@@ -147,10 +155,9 @@ public class TestStandbyCheckpoints {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        if(tmpOivImgDir.list().length > 0) {
+        if (tmpOivImgDir.list().length > 0) {
           return true;
-        }
-        else {
+        } else {
           return false;
         }
       }
@@ -189,9 +196,9 @@ public class TestStandbyCheckpoints {
     HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
     HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
     
-    assertEquals(12, nn0.getNamesystem().getFSImage()
+    assertEquals(12, nns[0].getNamesystem().getFSImage()
         .getMostRecentCheckpointTxId());
-    assertEquals(12, nn1.getNamesystem().getFSImage()
+    assertEquals(12, nns[1].getNamesystem().getFSImage()
         .getMostRecentCheckpointTxId());
     
     List<File> dirs = Lists.newArrayList();
@@ -214,17 +221,17 @@ public class TestStandbyCheckpoints {
     cluster.getConfiguration(1).setInt(
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 0);
     cluster.restartNameNode(1);
-    nn1 = cluster.getNameNode(1);
- 
-    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
-    
+    nns[1] = cluster.getNameNode(1);
+
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nns[1]);
+
     // 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);
+    HATestUtil.waitForStandbyToCatchUp(nns[0], nns[1]);
     Thread.sleep(2000);
     
     // We should make exactly one checkpoint at this new txid. 
@@ -259,7 +266,7 @@ public class TestStandbyCheckpoints {
     cluster.getConfiguration(1).setInt(
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 0);
     cluster.restartNameNode(1);
-    nn1 = cluster.getNameNode(1);
+    nns[1] = cluster.getNameNode(1);
 
     cluster.transitionToActive(0);    
     
@@ -284,31 +291,42 @@ public class TestStandbyCheckpoints {
   @Test(timeout=60000)
   public void testCheckpointCancellationDuringUpload() throws Exception {
     // don't compress, we want a big image
-    cluster.getConfiguration(0).setBoolean(
-        DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
-    cluster.getConfiguration(1).setBoolean(
-        DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
+    for (int i = 0; i < NUM_NNS; i++) {
+      cluster.getConfiguration(i).setBoolean(
+          DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
+    }
+
     // Throttle SBN upload to make it hang during upload to ANN
-    cluster.getConfiguration(1).setLong(
-        DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, 100);
-    cluster.restartNameNode(0);
-    cluster.restartNameNode(1);
-    nn0 = cluster.getNameNode(0);
-    nn1 = cluster.getNameNode(1);
+    for (int i = 1; i < NUM_NNS; i++) {
+      cluster.getConfiguration(i).setLong(
+          DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, 100);
+    }
+    for (int i = 0; i < NUM_NNS; i++) {
+      cluster.restartNameNode(i);
+    }
+
+    // update references to each of the nns
+    setNNs();
 
     cluster.transitionToActive(0);
 
     doEdits(0, 100);
-    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
-    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(104));
+
+    for (int i = 1; i < NUM_NNS; i++) {
+      HATestUtil.waitForStandbyToCatchUp(nns[0], nns[i]);
+      HATestUtil.waitForCheckpoint(cluster, i, ImmutableList.of(104));
+    }
+
     cluster.transitionToStandby(0);
     cluster.transitionToActive(1);
 
+
     // Wait to make sure background TransferFsImageUpload thread was cancelled.
     // This needs to be done before the next test in the suite starts, so that a
     // file descriptor is not held open during the next cluster init.
     cluster.shutdown();
     cluster = null;
+
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
@@ -325,7 +343,7 @@ public class TestStandbyCheckpoints {
     }, 1000, 30000);
 
     // Assert that former active did not accept the canceled checkpoint file.
-    assertEquals(0, nn0.getFSImage().getMostRecentCheckpointTxId());
+    assertEquals(0, nns[0].getFSImage().getMostRecentCheckpointTxId());
   }
   
   /**
@@ -337,7 +355,7 @@ public class TestStandbyCheckpoints {
   public void testStandbyExceptionThrownDuringCheckpoint() throws Exception {
     
     // Set it up so that we know when the SBN checkpoint starts and ends.
-    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nns[1]);
     DelayAnswer answerer = new DelayAnswer(LOG);
     Mockito.doAnswer(answerer).when(spyImage1)
         .saveNamespace(Mockito.any(FSNamesystem.class),
@@ -345,7 +363,7 @@ public class TestStandbyCheckpoints {
 
     // Perform some edits and wait for a checkpoint to start on the SBN.
     doEdits(0, 1000);
-    nn0.getRpcServer().rollEditLog();
+    nns[0].getRpcServer().rollEditLog();
     answerer.waitForCall();
     assertTrue("SBN is not performing checkpoint but it should be.",
         answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
@@ -355,7 +373,7 @@ public class TestStandbyCheckpoints {
     ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
     try {
       // Perform an RPC to the SBN and make sure it throws a StandbyException.
-      nn1.getRpcServer().getFileInfo("/");
+      nns[1].getRpcServer().getFileInfo("/");
       fail("Should have thrown StandbyException, but instead succeeded.");
     } catch (StandbyException se) {
       GenericTestUtils.assertExceptionContains("is not supported", se);
@@ -382,7 +400,7 @@ public class TestStandbyCheckpoints {
   public void testReadsAllowedDuringCheckpoint() throws Exception {
     
     // Set it up so that we know when the SBN checkpoint starts and ends.
-    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nns[1]);
     DelayAnswer answerer = new DelayAnswer(LOG);
     Mockito.doAnswer(answerer).when(spyImage1)
         .saveNamespace(Mockito.any(FSNamesystem.class),
@@ -391,7 +409,7 @@ public class TestStandbyCheckpoints {
     
     // Perform some edits and wait for a checkpoint to start on the SBN.
     doEdits(0, 1000);
-    nn0.getRpcServer().rollEditLog();
+    nns[0].getRpcServer().rollEditLog();
     answerer.waitForCall();
     assertTrue("SBN is not performing checkpoint but it should be.",
         answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
@@ -405,7 +423,7 @@ public class TestStandbyCheckpoints {
       @Override
       public void run() {
         try {
-          nn1.getRpcServer().restoreFailedStorage("false");
+          nns[1].getRpcServer().restoreFailedStorage("false");
         } catch (IOException e) {
           e.printStackTrace();
         }
@@ -415,16 +433,16 @@ public class TestStandbyCheckpoints {
     
     // Make sure that our thread is waiting for the lock.
     ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
-    
-    assertFalse(nn1.getNamesystem().getFsLockForTests().hasQueuedThreads());
-    assertFalse(nn1.getNamesystem().getFsLockForTests().isWriteLocked());
-    assertTrue(nn1.getNamesystem().getCpLockForTests().hasQueuedThreads());
-    
+
+    assertFalse(nns[1].getNamesystem().getFsLockForTests().hasQueuedThreads());
+    assertFalse(nns[1].getNamesystem().getFsLockForTests().isWriteLocked());
+    assertTrue(nns[1].getNamesystem().getCpLockForTests().hasQueuedThreads());
+
     // Get /jmx of the standby NN web UI, which will cause the FSNS read lock to
     // be taken.
     String pageContents = DFSTestUtil.urlGet(new URL("http://" +
-        nn1.getHttpAddress().getHostName() + ":" +
-        nn1.getHttpAddress().getPort() + "/jmx"));
+        nns[1].getHttpAddress().getHostName() + ":" +
+        nns[1].getHttpAddress().getPort() + "/jmx"));
     assertTrue(pageContents.contains("NumLiveDataNodes"));
     
     // Make sure that the checkpoint is still going on, implying that the client

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz
index 0f53f2a..abc7bbd 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz
index 737ad2d..b3f8b9d 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz
index 3cb2ee6..2256fba 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz
index b69741c..c4959b4 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz
index 2574f8b..e7d3fbd 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
index ef3e249..c671ccc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
@@ -20,4 +20,4 @@ log4j.rootLogger=info,stdout
 log4j.threshold=ALL
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n


[15/21] hadoop git commit: HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.

Posted by ar...@apache.org.
HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/49dfad94
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/49dfad94
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/49dfad94

Branch: refs/heads/HDFS-7240
Commit: 49dfad942970459297f72632ed8dfd353e0c86de
Parents: 122cad6
Author: Aaron T. Myers <at...@apache.org>
Authored: Tue Jun 23 17:26:11 2015 -0700
Committer: Aaron T. Myers <at...@apache.org>
Committed: Tue Jun 23 17:26:11 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ha/ZKFailoverController.java  |  61 ++-
 .../org/apache/hadoop/ha/MiniZKFCCluster.java   |  93 +++-
 .../hadoop/ha/TestZKFailoverController.java     |  32 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../bkjournal/TestBookKeeperHACheckpoints.java  |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |  36 +-
 .../token/block/BlockTokenSecretManager.java    |  40 +-
 .../server/blockmanagement/BlockManager.java    |  21 +-
 .../hdfs/server/namenode/CheckpointConf.java    |  14 +-
 .../hdfs/server/namenode/ImageServlet.java      |  88 +++-
 .../server/namenode/NameNodeHttpServer.java     |   7 +-
 .../hdfs/server/namenode/TransferFsImage.java   |  47 +-
 .../server/namenode/ha/BootstrapStandby.java    |  94 ++--
 .../hdfs/server/namenode/ha/EditLogTailer.java  | 162 +++++--
 .../server/namenode/ha/RemoteNameNodeInfo.java  | 133 ++++++
 .../server/namenode/ha/StandbyCheckpointer.java | 182 +++++---
 .../hdfs/tools/DFSZKFailoverController.java     |  13 +
 .../src/main/resources/hdfs-default.xml         |  20 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 464 ++++++++++++-------
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |  18 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |   6 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 109 +++--
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |  94 ++--
 .../security/token/block/TestBlockToken.java    |  10 +-
 .../hdfs/server/namenode/TestBackupNode.java    |   2 +-
 .../hdfs/server/namenode/TestCheckpoint.java    |   3 +-
 .../server/namenode/ha/HAStressTestHarness.java |  46 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |  60 ++-
 .../namenode/ha/TestBootstrapStandby.java       | 176 ++++---
 .../ha/TestBootstrapStandbyWithQJM.java         |  47 +-
 .../ha/TestDNFencingWithReplication.java        |   1 +
 .../server/namenode/ha/TestEditLogTailer.java   |  12 +-
 .../ha/TestFailoverWithBlockTokensEnabled.java  |  55 ++-
 .../server/namenode/ha/TestHAConfiguration.java |  49 +-
 .../namenode/ha/TestPipelinesFailover.java      | 110 +++--
 .../namenode/ha/TestRemoteNameNodeInfo.java     |  61 +++
 .../namenode/ha/TestSeveralNameNodes.java       | 179 +++++++
 .../namenode/ha/TestStandbyCheckpoints.java     | 106 +++--
 .../src/test/resources/hadoop-0.23-reserved.tgz | Bin 4558 -> 5590 bytes
 .../src/test/resources/hadoop-1-reserved.tgz    | Bin 2572 -> 3348 bytes
 .../src/test/resources/hadoop-2-reserved.tgz    | Bin 2838 -> 3465 bytes
 .../src/test/resources/hadoop-22-dfs-dir.tgz    | Bin 318180 -> 413239 bytes
 .../src/test/resources/hadoop1-bbw.tgz          | Bin 40234 -> 43294 bytes
 .../src/test/resources/log4j.properties         |   2 +-
 45 files changed, 1926 insertions(+), 740 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index b1f5920..30ec8f2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.Executors;
@@ -141,6 +142,7 @@ public abstract class ZKFailoverController {
       throws AccessControlException, IOException;
   protected abstract InetSocketAddress getRpcAddressToBindTo();
   protected abstract PolicyProvider getPolicyProvider();
+  protected abstract List<HAServiceTarget> getAllOtherNodes();
 
   /**
    * Return the name of a znode inside the configured parent znode in which
@@ -616,9 +618,11 @@ public abstract class ZKFailoverController {
    * Coordinate a graceful failover. This proceeds in several phases:
    * 1) Pre-flight checks: ensure that the local node is healthy, and
    * thus a candidate for failover.
-   * 2) Determine the current active node. If it is the local node, no
+   * 2a) Determine the current active node. If it is the local node, no
    * need to failover - return success.
-   * 3) Ask that node to yield from the election for a number of seconds.
+   * 2b) Get the other nodes
+   * 3a) Ask the other nodes to yield from election for a number of seconds
+   * 3b) Ask the active node to yield from the election for a number of seconds.
    * 4) Allow the normal election path to run in other threads. Wait until
    * we either become unhealthy or we see an election attempt recorded by
    * the normal code path.
@@ -648,12 +652,27 @@ public abstract class ZKFailoverController {
           "No need to failover. Returning success.");
       return;
     }
-    
-    // Phase 3: ask the old active to yield from the election.
-    LOG.info("Asking " + oldActive + " to cede its active state for " +
-        timeout + "ms");
-    ZKFCProtocol oldZkfc = oldActive.getZKFCProxy(conf, timeout);
-    oldZkfc.cedeActive(timeout);
+
+    // Phase 2b: get the other nodes
+    List<HAServiceTarget> otherNodes = getAllOtherNodes();
+    List<ZKFCProtocol> otherZkfcs = new ArrayList<ZKFCProtocol>(otherNodes.size());
+
+    // Phase 3: ask the other nodes to yield from the election.
+    HAServiceTarget activeNode = null;
+    for (HAServiceTarget remote : otherNodes) {
+      // same location, same node - may not always be == equality
+      if (remote.getAddress().equals(oldActive.getAddress())) {
+        activeNode = remote;
+        continue;
+      }
+      otherZkfcs.add(cedeRemoteActive(remote, timeout));
+    }
+
+    assert
+      activeNode != null : "Active node does not match any known remote node";
+
+    // Phase 3b: ask the old active to yield
+    otherZkfcs.add(cedeRemoteActive(activeNode, timeout));
 
     // Phase 4: wait for the normal election to make the local node
     // active.
@@ -676,8 +695,10 @@ public abstract class ZKFailoverController {
     // Phase 5. At this point, we made some attempt to become active. So we
     // can tell the old active to rejoin if it wants. This allows a quick
     // fail-back if we immediately crash.
-    oldZkfc.cedeActive(-1);
-    
+    for (ZKFCProtocol zkfc : otherZkfcs) {
+      zkfc.cedeActive(-1);
+    }
+
     if (attempt.succeeded) {
       LOG.info("Successfully became active. " + attempt.status);
     } else {
@@ -688,6 +709,23 @@ public abstract class ZKFailoverController {
   }
 
   /**
+   * Ask the remote zkfc to cede its active status and wait for the specified
+   * timeout before attempting to claim leader status.
+   * @param remote node to ask
+   * @param timeout amount of time to cede
+   * @return the {@link ZKFCProtocol} used to talk to the ndoe
+   * @throws IOException
+   */
+  private ZKFCProtocol cedeRemoteActive(HAServiceTarget remote, int timeout)
+    throws IOException {
+    LOG.info("Asking " + remote + " to cede its active state for "
+               + timeout + "ms");
+    ZKFCProtocol oldZkfc = remote.getZKFCProxy(conf, timeout);
+    oldZkfc.cedeActive(timeout);
+    return oldZkfc;
+  }
+
+  /**
    * Ensure that the local node is in a healthy state, and thus
    * eligible for graceful failover.
    * @throws ServiceFailedException if the node is unhealthy
@@ -777,7 +815,8 @@ public abstract class ZKFailoverController {
           break;
           
         default:
-          throw new IllegalArgumentException("Unhandled state:" + lastHealthState);
+          throw new IllegalArgumentException("Unhandled state:"
+                                               + lastHealthState);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
index 5aee611..b496bf9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 
 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;
@@ -49,7 +51,7 @@ public class MiniZKFCCluster {
   private final TestContext ctx;
   private final ZooKeeperServer zks;
 
-  private DummyHAService svcs[];
+  private List<DummyHAService> svcs;
   private DummyZKFCThread thrs[];
   private Configuration conf;
   
@@ -63,38 +65,67 @@ public class MiniZKFCCluster {
     conf.setInt(CommonConfigurationKeys.HA_HM_CHECK_INTERVAL_KEY, 50);
     conf.setInt(CommonConfigurationKeys.HA_HM_CONNECT_RETRY_INTERVAL_KEY, 50);
     conf.setInt(CommonConfigurationKeys.HA_HM_SLEEP_AFTER_DISCONNECT_KEY, 50);
-    svcs = new DummyHAService[2];
-    svcs[0] = new DummyHAService(HAServiceState.INITIALIZING,
-        new InetSocketAddress("svc1", 1234));
-    svcs[0].setSharedResource(sharedResource);
-    svcs[1] = new DummyHAService(HAServiceState.INITIALIZING,
-        new InetSocketAddress("svc2", 1234));
-    svcs[1].setSharedResource(sharedResource);
-    
+    svcs = new ArrayList<DummyHAService>(2);
+    // remove any existing instances we are keeping track of
+    DummyHAService.instances.clear();
+
+    for (int i = 0; i < 2; i++) {
+      addSvcs(svcs, i);
+    }
+
     this.ctx = new TestContext();
     this.zks = zks;
   }
-  
+
+  private void addSvcs(List<DummyHAService> svcs, int i) {
+    svcs.add(new DummyHAService(HAServiceState.INITIALIZING, new InetSocketAddress("svc" + (i + 1),
+        1234)));
+    svcs.get(i).setSharedResource(sharedResource);
+  }
+
   /**
    * Set up two services and their failover controllers. svc1 is started
    * first, so that it enters ACTIVE state, and then svc2 is started,
    * which enters STANDBY
    */
   public void start() throws Exception {
+    start(2);
+  }
+
+  /**
+   * Set up the specified number of services and their failover controllers. svc1 is
+   * started first, so that it enters ACTIVE state, and then svc2...svcN is started, which enters
+   * STANDBY.
+   * <p>
+   * Adds any extra svc needed beyond the first two before starting the rest of the cluster.
+   * @param count number of zkfcs to start
+   */
+  public void start(int count) throws Exception {
+    // setup the expected number of zkfcs, if we need to add more. This seemed the least invasive
+    // way to add the services - otherwise its a large test rewrite or changing a lot of assumptions
+    if (count > 2) {
+      for (int i = 2; i < count; i++) {
+        addSvcs(svcs, i);
+      }
+    }
+
     // Format the base dir, should succeed
-    thrs = new DummyZKFCThread[2];
-    thrs[0] = new DummyZKFCThread(ctx, svcs[0]);
+    thrs = new DummyZKFCThread[count];
+    thrs[0] = new DummyZKFCThread(ctx, svcs.get(0));
     assertEquals(0, thrs[0].zkfc.run(new String[]{"-formatZK"}));
     ctx.addThread(thrs[0]);
     thrs[0].start();
     
     LOG.info("Waiting for svc0 to enter active state");
     waitForHAState(0, HAServiceState.ACTIVE);
-    
-    LOG.info("Adding svc1");
-    thrs[1] = new DummyZKFCThread(ctx, svcs[1]);
-    thrs[1].start();
-    waitForHAState(1, HAServiceState.STANDBY);
+
+    // add the remaining zkfc
+    for (int i = 1; i < count; i++) {
+      LOG.info("Adding svc" + i);
+      thrs[i] = new DummyZKFCThread(ctx, svcs.get(i));
+      thrs[i].start();
+      waitForHAState(i, HAServiceState.STANDBY);
+    }
   }
   
   /**
@@ -122,7 +153,7 @@ public class MiniZKFCCluster {
   }
   
   public DummyHAService getService(int i) {
-    return svcs[i];
+    return svcs.get(i);
   }
 
   public ActiveStandbyElector getElector(int i) {
@@ -134,23 +165,23 @@ public class MiniZKFCCluster {
   }
   
   public void setHealthy(int idx, boolean healthy) {
-    svcs[idx].isHealthy = healthy;
+    svcs.get(idx).isHealthy = healthy;
   }
 
   public void setFailToBecomeActive(int idx, boolean doFail) {
-    svcs[idx].failToBecomeActive = doFail;
+    svcs.get(idx).failToBecomeActive = doFail;
   }
 
   public void setFailToBecomeStandby(int idx, boolean doFail) {
-    svcs[idx].failToBecomeStandby = doFail;
+    svcs.get(idx).failToBecomeStandby = doFail;
   }
   
   public void setFailToFence(int idx, boolean doFail) {
-    svcs[idx].failToFence = doFail;
+    svcs.get(idx).failToFence = doFail;
   }
   
   public void setUnreachable(int idx, boolean unreachable) {
-    svcs[idx].actUnreachable = unreachable;
+    svcs.get(idx).actUnreachable = unreachable;
   }
 
   /**
@@ -204,7 +235,7 @@ public class MiniZKFCCluster {
     byte[] data = zks.getZKDatabase().getData(
         DummyZKFC.LOCK_ZNODE, stat, null);
     
-    assertArrayEquals(Ints.toByteArray(svcs[idx].index), data);
+    assertArrayEquals(Ints.toByteArray(svcs.get(idx).index), data);
     long session = stat.getEphemeralOwner();
     LOG.info("Expiring svc " + idx + "'s zookeeper session " + session);
     zks.closeSession(session);
@@ -218,7 +249,7 @@ public class MiniZKFCCluster {
    */
   public void waitForActiveLockHolder(Integer idx)
       throws Exception {
-    DummyHAService svc = idx == null ? null : svcs[idx];
+    DummyHAService svc = idx == null ? null : svcs.get(idx);
     ActiveStandbyElectorTestUtil.waitForActiveLockData(ctx, zks,
         DummyZKFC.SCOPED_PARENT_ZNODE,
         (idx == null) ? null : Ints.toByteArray(svc.index));
@@ -320,5 +351,17 @@ public class MiniZKFCCluster {
     protected PolicyProvider getPolicyProvider() {
       return null;
     }
+
+    @Override
+    protected List<HAServiceTarget> getAllOtherNodes() {
+      List<HAServiceTarget> services = new ArrayList<HAServiceTarget>(
+          DummyHAService.instances.size());
+      for (DummyHAService service : DummyHAService.instances) {
+        if (service != this.localTarget) {
+          services.add(service);
+        }
+      }
+      return services;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
index d8271c5..b8d9ce4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
@@ -605,6 +605,38 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
     }
   }
 
+  @Test(timeout = 25000)
+  public void testGracefulFailoverMultipleZKfcs() throws Exception {
+    try {
+      cluster.start(3);
+
+      cluster.waitForActiveLockHolder(0);
+
+      // failover to first
+      cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
+      cluster.waitForActiveLockHolder(1);
+
+      // failover to second
+      cluster.getService(2).getZKFCProxy(conf, 5000).gracefulFailover();
+      cluster.waitForActiveLockHolder(2);
+
+      // failover back to original
+      cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
+      cluster.waitForActiveLockHolder(0);
+
+      Thread.sleep(10000); // allow to quiesce
+
+      assertEquals(0, cluster.getService(0).fenceCount);
+      assertEquals(0, cluster.getService(1).fenceCount);
+      assertEquals(0, cluster.getService(2).fenceCount);
+      assertEquals(2, cluster.getService(0).activeTransitionCount);
+      assertEquals(1, cluster.getService(1).activeTransitionCount);
+      assertEquals(1, cluster.getService(2).activeTransitionCount);
+    } finally {
+      cluster.stop();
+    }
+  }
+
   private int runFC(DummyHAService target, String ... args) throws Exception {
     DummyZKFC zkfc = new DummyZKFC(conf, target);
     return zkfc.run(args);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 981ca55..09a6891 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -38,6 +38,8 @@ Trunk (Unreleased)
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
+    HDFS-6440. Support more than 2 NameNodes. (Jesse Yates via atm)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
index b74cd7f..ed53512 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
@@ -32,6 +32,10 @@ import org.junit.BeforeClass;
  * using a bookkeeper journal manager as the shared directory
  */
 public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
+  //overwrite the nn count
+ static{
+   TestStandbyCheckpoints.NUM_NNS = 2;
+ }
   private static BKJMUtil bkutil = null;
   static int numBookies = 3;
   static int journalCount = 0;
@@ -57,8 +61,7 @@ public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
       .build();
     cluster.waitActive();
 
-    nn0 = cluster.getNameNode(0);
-    nn1 = cluster.getNameNode(1);
+    setNNs();
     fs = HATestUtil.configureFailoverFs(cluster, conf);
 
     cluster.transitionToActive(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
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 30540a9..ebd668f 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
@@ -132,6 +132,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50090";
   public static final String  DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY = "dfs.namenode.secondary.https-address";
   public static final String  DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:50091";
+  public static final String  DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_KEY = "dfs.namenode.checkpoint.check.quiet-multiplier";
+  public static final double  DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_DEFAULT = 1.5;
   public static final String  DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY = "dfs.namenode.checkpoint.check.period";
   public static final long    DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT = 60;
   public static final String  DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";
@@ -544,6 +546,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   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
+  public static final String DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY = "dfs.ha.tail-edits.namenode-retries";
+  public static final int DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT = 3;
   public static final String DFS_HA_LOGROLL_RPC_TIMEOUT_KEY = "dfs.ha.log-roll.rpc.timeout";
   public static final int DFS_HA_LOGROLL_RPC_TIMEOUT_DEFAULT = 20000; // 20s
   public static final String DFS_HA_FENCE_METHODS_KEY = "dfs.ha.fencing.methods";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
----------------------------------------------------------------------
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
index c967c69..686a0b7 100644
--- 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
@@ -143,7 +143,7 @@ public class HAUtil {
    * @param conf the configuration of this node
    * @return the NN ID of the other node in this nameservice
    */
-  public static String getNameNodeIdOfOtherNode(Configuration conf, String nsId) {
+  public static List<String> getNameNodeIdOfOtherNodes(Configuration conf, String nsId) {
     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, " +
@@ -157,20 +157,20 @@ public class HAUtil {
         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.checkArgument(nnIds.size() >= 2,
+        "Expected at least 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;
-    return nnSet.get(0);
+    ArrayList<String> namenodes = Lists.newArrayList(nnIds);
+    namenodes.remove(myNNId);
+    assert namenodes.size() >= 1;
+    return namenodes;
   }
 
   /**
@@ -180,16 +180,20 @@ public class HAUtil {
    * @param myConf the configuration of this node
    * @return the configuration of the other node in an HA setup
    */
-  public static Configuration getConfForOtherNode(
+  public static List<Configuration> getConfForOtherNodes(
       Configuration myConf) {
     
     String nsId = DFSUtil.getNamenodeNameServiceId(myConf);
-    String otherNn = getNameNodeIdOfOtherNode(myConf, nsId);
-    
-    // Look up the address of the active NN.
-    Configuration confForOtherNode = new Configuration(myConf);
-    NameNode.initializeGenericKeys(confForOtherNode, nsId, otherNn);
-    return confForOtherNode;
+    List<String> otherNn = getNameNodeIdOfOtherNodes(myConf, nsId);
+
+    // Look up the address of the other NNs
+    List<Configuration> confs = new ArrayList<Configuration>(otherNn.size());
+    for (String nn : otherNn) {
+      Configuration confForOtherNode = new Configuration(myConf);
+      NameNode.initializeGenericKeys(confForOtherNode, nsId, nn);
+      confs.add(confForOtherNode);
+    }
+    return confs;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
index b103c1a..53da44c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
@@ -52,17 +52,11 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class BlockTokenSecretManager extends
     SecretManager<BlockTokenIdentifier> {
-  public static final Log LOG = LogFactory
-      .getLog(BlockTokenSecretManager.class);
-  
-  // We use these in an HA setup to ensure that the pair of NNs produce block
-  // token serial numbers that are in different ranges.
-  private static final int LOW_MASK  = ~(1 << 31);
-  
+  public static final Log LOG = LogFactory.getLog(BlockTokenSecretManager.class);
+
   public static final Token<BlockTokenIdentifier> DUMMY_TOKEN = new Token<BlockTokenIdentifier>();
 
   private final boolean isMaster;
-  private int nnIndex;
   
   /**
    * keyUpdateInterval is the interval that NN updates its block keys. It should
@@ -77,21 +71,22 @@ public class BlockTokenSecretManager extends
   private final Map<Integer, BlockKey> allKeys;
   private String blockPoolId;
   private final String encryptionAlgorithm;
-  
+
+  private final int intRange;
+  private final int nnRangeStart;
+
   private final SecureRandom nonceGenerator = new SecureRandom();
 
-  ;
-  
   /**
    * Constructor for slaves.
-   * 
+   *
    * @param keyUpdateInterval how often a new key will be generated
    * @param tokenLifetime how long an individual token is valid
    */
   public BlockTokenSecretManager(long keyUpdateInterval,
       long tokenLifetime, String blockPoolId, String encryptionAlgorithm) {
     this(false, keyUpdateInterval, tokenLifetime, blockPoolId,
-        encryptionAlgorithm);
+        encryptionAlgorithm, 0, 1);
   }
   
   /**
@@ -99,23 +94,25 @@ public class BlockTokenSecretManager extends
    * 
    * @param keyUpdateInterval how often a new key will be generated
    * @param tokenLifetime how long an individual token is valid
-   * @param nnIndex namenode index
+   * @param nnIndex namenode index of the namenode for which we are creating the manager
    * @param blockPoolId block pool ID
    * @param encryptionAlgorithm encryption algorithm to use
+   * @param numNNs number of namenodes possible
    */
   public BlockTokenSecretManager(long keyUpdateInterval,
-      long tokenLifetime, int nnIndex, String blockPoolId,
+      long tokenLifetime, int nnIndex, int numNNs,  String blockPoolId,
       String encryptionAlgorithm) {
-    this(true, keyUpdateInterval, tokenLifetime, blockPoolId,
-        encryptionAlgorithm);
-    Preconditions.checkArgument(nnIndex == 0 || nnIndex == 1);
-    this.nnIndex = nnIndex;
+    this(true, keyUpdateInterval, tokenLifetime, blockPoolId, encryptionAlgorithm, nnIndex, numNNs);
+    Preconditions.checkArgument(nnIndex >= 0);
+    Preconditions.checkArgument(numNNs > 0);
     setSerialNo(new SecureRandom().nextInt());
     generateKeys();
   }
   
   private BlockTokenSecretManager(boolean isMaster, long keyUpdateInterval,
-      long tokenLifetime, String blockPoolId, String encryptionAlgorithm) {
+      long tokenLifetime, String blockPoolId, String encryptionAlgorithm, int nnIndex, int numNNs) {
+    this.intRange = Integer.MAX_VALUE / numNNs;
+    this.nnRangeStart = intRange * nnIndex;
     this.isMaster = isMaster;
     this.keyUpdateInterval = keyUpdateInterval;
     this.tokenLifetime = tokenLifetime;
@@ -127,7 +124,8 @@ public class BlockTokenSecretManager extends
   
   @VisibleForTesting
   public synchronized void setSerialNo(int serialNo) {
-    this.serialNo = (serialNo & LOW_MASK) | (nnIndex << 31);
+    // we mod the serial number by the range and then add that times the index
+    this.serialNo = (serialNo % intRange) + (nnRangeStart);
   }
   
   public void setBlockPoolId(String blockPoolId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
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 824801f..7d3a678 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
@@ -42,6 +43,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -399,14 +401,21 @@ public class BlockManager {
     boolean isHaEnabled = HAUtil.isHAEnabled(conf, nsId);
 
     if (isHaEnabled) {
-      String thisNnId = HAUtil.getNameNodeId(conf, nsId);
-      String otherNnId = HAUtil.getNameNodeIdOfOtherNode(conf, nsId);
-      return new BlockTokenSecretManager(updateMin*60*1000L,
-          lifetimeMin*60*1000L, thisNnId.compareTo(otherNnId) < 0 ? 0 : 1, null,
-          encryptionAlgorithm);
+      // figure out which index we are of the nns
+      Collection<String> nnIds = DFSUtilClient.getNameNodeIds(conf, nsId);
+      String nnId = HAUtil.getNameNodeId(conf, nsId);
+      int nnIndex = 0;
+      for (String id : nnIds) {
+        if (id.equals(nnId)) {
+          break;
+        }
+        nnIndex++;
+      }
+      return new BlockTokenSecretManager(updateMin * 60 * 1000L,
+          lifetimeMin * 60 * 1000L, nnIndex, nnIds.size(), null, encryptionAlgorithm);
     } else {
       return new BlockTokenSecretManager(updateMin*60*1000L,
-          lifetimeMin*60*1000L, 0, null, encryptionAlgorithm);
+          lifetimeMin*60*1000L, 0, 1, null, encryptionAlgorithm);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
----------------------------------------------------------------------
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
index b1636bc..c30730b 100644
--- 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
@@ -44,7 +44,13 @@ public class CheckpointConf {
 
   /** The output dir for legacy OIV image */
   private final String legacyOivImageDir;
-  
+
+  /**
+  * multiplier on the checkpoint period to allow other nodes to do the checkpointing, when not the
+  * 'primary' checkpoint node
+  */
+  private double quietMultiplier;
+
   public CheckpointConf(Configuration conf) {
     checkpointCheckPeriod = conf.getLong(
         DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
@@ -57,6 +63,8 @@ public class CheckpointConf {
     maxRetriesOnMergeError = conf.getInt(DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY,
                                   DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_DEFAULT);
     legacyOivImageDir = conf.get(DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY);
+    quietMultiplier = conf.getDouble(DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_KEY,
+      DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_DEFAULT);
     warnForDeprecatedConfigs(conf);
   }
   
@@ -91,4 +99,8 @@ public class CheckpointConf {
   public String getLegacyOivImageDir() {
     return legacyOivImageDir;
   }
+
+  public double getQuietPeriod() {
+    return this.checkpointPeriod * this.quietMultiplier;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
index c565eb5..9dc20b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
@@ -30,6 +30,7 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.commons.logging.Log;
@@ -81,6 +82,9 @@ public class ImageServlet extends HttpServlet {
   private static final String LATEST_FSIMAGE_VALUE = "latest";
   private static final String IMAGE_FILE_TYPE = "imageFile";
 
+  private SortedSet<ImageUploadRequest> currentlyDownloadingCheckpoints = Collections
+      .<ImageUploadRequest> synchronizedSortedSet(new TreeSet<ImageUploadRequest>());
+
   @Override
   public void doGet(final HttpServletRequest request,
       final HttpServletResponse response) throws ServletException, IOException {
@@ -253,10 +257,12 @@ public class ImageServlet extends HttpServlet {
     }
 
     if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf))) {
-      Configuration otherNnConf = HAUtil.getConfForOtherNode(conf);
-      validRequestors.add(SecurityUtil.getServerPrincipal(otherNnConf
-          .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-          NameNode.getAddress(otherNnConf).getHostName()));
+      List<Configuration> otherNnConfs = HAUtil.getConfForOtherNodes(conf);
+      for (Configuration otherNnConf : otherNnConfs) {
+        validRequestors.add(SecurityUtil.getServerPrincipal(otherNnConf
+                .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
+            NameNode.getAddress(otherNnConf).getHostName()));
+      }
     }
 
     for (String v : validRequestors) {
@@ -420,7 +426,6 @@ public class ImageServlet extends HttpServlet {
   /**
    * Set the required parameters for uploading image
    * 
-   * @param httpMethod instance of method to set the parameters
    * @param storage colon separated storageInfo string
    * @param txid txid of the image
    * @param imageFileSize size of the imagefile to be uploaded
@@ -459,12 +464,37 @@ public class ImageServlet extends HttpServlet {
 
             @Override
             public Void run() throws Exception {
+              // if its not the active NN, then we need to notify the caller it was was the wrong
+              // target (regardless of the fact that we got the image)
+              HAServiceProtocol.HAServiceState state = NameNodeHttpServer
+                  .getNameNodeStateFromContext(getServletContext());
+              if (state != HAServiceProtocol.HAServiceState.ACTIVE) {
+                // we need a different response type here so the client can differentiate this
+                // from the failure to upload due to (1) security, or (2) other checkpoints already
+                // present
+                response.sendError(HttpServletResponse.SC_EXPECTATION_FAILED,
+                    "Nameode "+request.getLocalAddr()+" is currently not in a state which can "
+                        + "accept uploads of new fsimages. State: "+state);
+                return null;
+              }
 
               final long txid = parsedParams.getTxId();
+              String remoteAddr = request.getRemoteAddr();
+              ImageUploadRequest imageRequest = new ImageUploadRequest(txid, remoteAddr);
 
               final NameNodeFile nnf = parsedParams.getNameNodeFile();
 
-              if (!nnImage.addToCheckpointing(txid)) {
+              // if the node is attempting to upload an older transaction, we ignore it
+              SortedSet<ImageUploadRequest> larger = currentlyDownloadingCheckpoints.tailSet(imageRequest);
+              if (larger.size() > 0) {
+                response.sendError(HttpServletResponse.SC_CONFLICT,
+                    "Another checkpointer is already in the process of uploading a" +
+                        " checkpoint made up to transaction ID " + larger.last());
+                return null;
+              }
+
+              //make sure no one else has started uploading one
+              if (!currentlyDownloadingCheckpoints.add(imageRequest)) {
                 response.sendError(HttpServletResponse.SC_CONFLICT,
                     "Either current namenode is checkpointing or another"
                         + " checkpointer is already in the process of "
@@ -499,6 +529,10 @@ public class ImageServlet extends HttpServlet {
                   // remove some old ones.
                   nnImage.purgeOldStorage(nnf);
                 } finally {
+                  // remove the request once we've processed it, or it threw an error, so we
+                  // aren't using it either
+                  currentlyDownloadingCheckpoints.remove(imageRequest);
+
                   stream.close();
                 }
               } finally {
@@ -555,4 +589,46 @@ public class ImageServlet extends HttpServlet {
       return nnf;
     }
   }
+
+  private static class ImageUploadRequest implements Comparable<ImageUploadRequest> {
+
+    private final long txId;
+    private final String address;
+
+    public ImageUploadRequest(long txid, String remoteAddr) {
+      this.txId = txid;
+      this.address = remoteAddr;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      ImageUploadRequest that = (ImageUploadRequest) o;
+
+      if (txId != that.txId) return false;
+      if (!address.equals(that.address)) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = (int) (txId ^ (txId >>> 32));
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+
+    @Override public int compareTo(ImageUploadRequest other) {
+      return Long.compare(txId, other.txId);
+    }
+
+    @Override public String toString() {
+      return "ImageRequest{" +
+          "txId=" + txId +
+          ", address='" + address + '\'' +
+          '}';
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
index 09b6b80..6bd9868 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
@@ -27,6 +27,7 @@ import javax.servlet.ServletContext;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -272,4 +273,8 @@ public class NameNodeHttpServer {
       ServletContext context) {
     return (StartupProgress)context.getAttribute(STARTUP_PROGRESS_ATTRIBUTE_KEY);
   }
-}
+
+  public static HAServiceProtocol.HAServiceState getNameNodeStateFromContext(ServletContext context) {
+    return getNameNodeFromContext(context).getServiceState();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
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 9783cca..afecf99 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
@@ -70,7 +70,33 @@ import org.mortbay.jetty.EofException;
  */
 @InterfaceAudience.Private
 public class TransferFsImage {
-  
+
+  public enum TransferResult{
+    SUCCESS(HttpServletResponse.SC_OK, false),
+    AUTHENTICATION_FAILURE(HttpServletResponse.SC_FORBIDDEN, true),
+    NOT_ACTIVE_NAMENODE_FAILURE(HttpServletResponse.SC_EXPECTATION_FAILED, false),
+    OLD_TRANSACTION_ID_FAILURE(HttpServletResponse.SC_CONFLICT, false),
+    UNEXPECTED_FAILURE(-1, true);
+
+    private final int response;
+    private final boolean shouldReThrowException;
+
+    private TransferResult(int response, boolean rethrow) {
+      this.response = response;
+      this.shouldReThrowException = rethrow;
+    }
+
+    public static TransferResult getResultForCode(int code){
+      TransferResult ret = UNEXPECTED_FAILURE;
+      for(TransferResult result:TransferResult.values()){
+        if(result.response == code){
+          return result;
+        }
+      }
+      return ret;
+    }
+  }
+
   public final static String CONTENT_LENGTH = "Content-Length";
   public final static String FILE_LENGTH = "File-Length";
   public final static String MD5_HEADER = "X-MD5-Digest";
@@ -198,9 +224,9 @@ public class TransferFsImage {
    * @param txid the transaction ID of the image to be uploaded
    * @throws IOException if there is an I/O error
    */
-  public static void uploadImageFromStorage(URL fsName, Configuration conf,
+  public static TransferResult uploadImageFromStorage(URL fsName, Configuration conf,
       NNStorage storage, NameNodeFile nnf, long txid) throws IOException {
-    uploadImageFromStorage(fsName, conf, storage, nnf, txid, null);
+    return uploadImageFromStorage(fsName, conf, storage, nnf, txid, null);
   }
 
   /**
@@ -215,7 +241,7 @@ public class TransferFsImage {
    * @param canceler optional canceler to check for abort of upload
    * @throws IOException if there is an I/O error or cancellation
    */
-  public static void uploadImageFromStorage(URL fsName, Configuration conf,
+  public static TransferResult uploadImageFromStorage(URL fsName, Configuration conf,
       NNStorage storage, NameNodeFile nnf, long txid, Canceler canceler)
       throws IOException {
     URL url = new URL(fsName, ImageServlet.PATH_SPEC);
@@ -223,21 +249,18 @@ public class TransferFsImage {
     try {
       uploadImage(url, conf, storage, nnf, txid, canceler);
     } catch (HttpPutFailedException 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 {
+      // translate the error code to a result, which is a bit more obvious in usage
+      TransferResult result = TransferResult.getResultForCode(e.getResponseCode());
+      if (result.shouldReThrowException) {
         throw e;
       }
+      return result;
     }
     double xferSec = Math.max(
         ((float) (Time.monotonicNow() - startTime)) / 1000.0, 0.001);
     LOG.info("Uploaded image with txid " + txid + " to namenode at " + fsName
         + " in " + xferSec + " seconds");
+    return TransferResult.SUCCESS;
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
index 88d9a6a..c22d7f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
@@ -23,8 +23,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIP
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.net.URL;
 import java.security.PrivilegedAction;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -77,10 +77,8 @@ public class BootstrapStandby implements Tool, Configurable {
   private static final Log LOG = LogFactory.getLog(BootstrapStandby.class);
   private String nsId;
   private String nnId;
-  private String otherNNId;
+  private List<RemoteNameNodeInfo> remoteNNs;
 
-  private URL otherHttpAddr;
-  private InetSocketAddress otherIpcAddr;
   private Collection<URI> dirsToFormat;
   private List<URI> editUrisToFormat;
   private List<URI> sharedEditsUris;
@@ -139,8 +137,8 @@ public class BootstrapStandby implements Tool, Configurable {
     System.err.println("Usage: " + this.getClass().getSimpleName() +
         " [-force] [-nonInteractive] [-skipSharedEditsCheck]");
   }
-  
-  private NamenodeProtocol createNNProtocolProxy()
+
+  private NamenodeProtocol createNNProtocolProxy(InetSocketAddress otherIpcAddr)
       throws IOException {
     return NameNodeProxies.createNonHAProxy(getConf(),
         otherIpcAddr, NamenodeProtocol.class,
@@ -149,18 +147,36 @@ public class BootstrapStandby implements Tool, Configurable {
   }
   
   private int doRun() throws IOException {
-    NamenodeProtocol proxy = createNNProtocolProxy();
-    NamespaceInfo nsInfo;
-    boolean isUpgradeFinalized;
-    try {
-      nsInfo = proxy.versionRequest();
-      isUpgradeFinalized = proxy.isUpgradeFinalized();
-    } catch (IOException ioe) {
-      LOG.fatal("Unable to fetch namespace information from active NN at " +
-          otherIpcAddr + ": " + ioe.getMessage());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Full exception trace", ioe);
+    // find the active NN
+    NamenodeProtocol proxy = null;
+    NamespaceInfo nsInfo = null;
+    boolean isUpgradeFinalized = false;
+    RemoteNameNodeInfo proxyInfo = null;
+    for (int i = 0; i < remoteNNs.size(); i++) {
+      proxyInfo = remoteNNs.get(i);
+      InetSocketAddress otherIpcAddress = proxyInfo.getIpcAddress();
+      proxy = createNNProtocolProxy(otherIpcAddress);
+      try {
+        // Get the namespace from any active NN. If you just formatted the primary NN and are
+        // bootstrapping the other NNs from that layout, it will only contact the single NN.
+        // However, if there cluster is already running and you are adding a NN later (e.g.
+        // replacing a failed NN), then this will bootstrap from any node in the cluster.
+        nsInfo = proxy.versionRequest();
+        isUpgradeFinalized = proxy.isUpgradeFinalized();
+        break;
+      } catch (IOException ioe) {
+        LOG.warn("Unable to fetch namespace information from remote NN at " + otherIpcAddress
+            + ": " + ioe.getMessage());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Full exception trace", ioe);
+        }
       }
+    }
+
+    if (nsInfo == null) {
+      LOG.fatal(
+          "Unable to fetch namespace information from any remote NN. Possible NameNodes: "
+              + remoteNNs);
       return ERR_CODE_FAILED_CONNECT;
     }
 
@@ -175,9 +191,9 @@ public class BootstrapStandby implements Tool, Configurable {
         "=====================================================\n" +
         "About to bootstrap Standby ID " + nnId + " from:\n" +
         "           Nameservice ID: " + nsId + "\n" +
-        "        Other Namenode ID: " + otherNNId + "\n" +
-        "  Other NN's HTTP address: " + otherHttpAddr + "\n" +
-        "  Other NN's IPC  address: " + otherIpcAddr + "\n" +
+        "        Other Namenode ID: " + proxyInfo.getNameNodeID() + "\n" +
+        "  Other NN's HTTP address: " + proxyInfo.getHttpAddress() + "\n" +
+        "  Other NN's IPC  address: " + proxyInfo.getIpcAddress() + "\n" +
         "             Namespace ID: " + nsInfo.getNamespaceID() + "\n" +
         "            Block pool ID: " + nsInfo.getBlockPoolID() + "\n" +
         "               Cluster ID: " + nsInfo.getClusterID() + "\n" +
@@ -201,7 +217,7 @@ public class BootstrapStandby implements Tool, Configurable {
     }
 
     // download the fsimage from active namenode
-    int download = downloadImage(storage, proxy);
+    int download = downloadImage(storage, proxy, proxyInfo);
     if (download != 0) {
       return download;
     }
@@ -292,7 +308,7 @@ public class BootstrapStandby implements Tool, Configurable {
     }
   }
 
-  private int downloadImage(NNStorage storage, NamenodeProtocol proxy)
+  private int downloadImage(NNStorage storage, NamenodeProtocol proxy, RemoteNameNodeInfo proxyInfo)
       throws IOException {
     // Load the newly formatted image, using all of the directories
     // (including shared edits)
@@ -316,7 +332,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
       // Download that checkpoint into our storage directories.
       MD5Hash hash = TransferFsImage.downloadImageToStorage(
-          otherHttpAddr, imageTxId, storage, true);
+        proxyInfo.getHttpAddress(), imageTxId, storage, true);
       image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId,
           hash);
     } catch (IOException ioe) {
@@ -385,18 +401,26 @@ public class BootstrapStandby implements Tool, Configurable {
       throw new HadoopIllegalArgumentException(
         "Shared edits storage is not enabled for this namenode.");
     }
-    
-    Configuration otherNode = HAUtil.getConfForOtherNode(conf);
-    otherNNId = HAUtil.getNameNodeId(otherNode, nsId);
-    otherIpcAddr = NameNode.getServiceAddress(otherNode, true);
-    Preconditions.checkArgument(otherIpcAddr.getPort() != 0 &&
-        !otherIpcAddr.getAddress().isAnyLocalAddress(),
-        "Could not determine valid IPC address for other NameNode (%s)" +
-        ", got: %s", otherNNId, otherIpcAddr);
-
-    final String scheme = DFSUtil.getHttpClientScheme(conf);
-    otherHttpAddr = DFSUtil.getInfoServerWithDefaultHost(
-        otherIpcAddr.getHostName(), otherNode, scheme).toURL();
+
+
+    remoteNNs = RemoteNameNodeInfo.getRemoteNameNodes(conf, nsId);
+    // validate the configured NNs
+    List<RemoteNameNodeInfo> remove = new ArrayList<RemoteNameNodeInfo>(remoteNNs.size());
+    for (RemoteNameNodeInfo info : remoteNNs) {
+      InetSocketAddress address = info.getIpcAddress();
+      LOG.info("Found nn: " + info.getNameNodeID() + ", ipc: " + info.getIpcAddress());
+      if (address.getPort() == 0 || address.getAddress().isAnyLocalAddress()) {
+        LOG.error("Could not determine valid IPC address for other NameNode ("
+            + info.getNameNodeID() + ") , got: " + address);
+        remove.add(info);
+      }
+    }
+
+    // remove any invalid nns
+    remoteNNs.removeAll(remove);
+
+    // make sure we have at least one left to read
+    Preconditions.checkArgument(!remoteNNs.isEmpty(), "Could not find any valid namenodes!");
 
     dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
     editUrisToFormat = FSNamesystem.getNamespaceEditsDirs(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
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
index 38aa358..cfca77c 100644
--- 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
@@ -23,7 +23,13 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.Callable;
 
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -67,10 +73,10 @@ public class EditLogTailer {
   
   private final Configuration conf;
   private final FSNamesystem namesystem;
+  private final Iterator<RemoteNameNodeInfo> nnLookup;
   private FSEditLog editLog;
 
-  private InetSocketAddress activeAddr;
-  private NamenodeProtocol cachedActiveProxy = null;
+  private RemoteNameNodeInfo currentNN;
 
   /**
    * The last transaction ID at which an edit log roll was initiated.
@@ -100,7 +106,17 @@ public class EditLogTailer {
    * available to be read from.
    */
   private final long sleepTimeMs;
-  
+
+  private final int nnCount;
+  private NamenodeProtocol cachedActiveProxy = null;
+  // count of the number of NNs we have attempted in the current lookup loop
+  private int nnLoopCount = 0;
+
+  /**
+   * maximum number of retries we should give each of the remote namenodes before giving up
+   */
+  private int maxRetries;
+
   public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
     this.tailerThread = new EditLogTailerThread();
     this.conf = conf;
@@ -111,12 +127,24 @@ public class EditLogTailer {
 
     logRollPeriodMs = conf.getInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
         DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT) * 1000;
+    List<RemoteNameNodeInfo> nns = Collections.emptyList();
     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 " +
+      try {
+        nns = RemoteNameNodeInfo.getRemoteNameNodes(conf);
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Remote NameNodes not correctly configured!", e);
+      }
+
+      for (RemoteNameNodeInfo info : nns) {
+        // overwrite the socket address, if we need to
+        InetSocketAddress ipc = NameNode.getServiceAddress(info.getConfiguration(), true);
+        // sanity check the ipc address
+        Preconditions.checkArgument(ipc.getPort() > 0,
+            "Active NameNode must have an IPC port configured. " + "Got address '%s'", ipc);
+        info.setIpcAddress(ipc);
+      }
+
+      LOG.info("Will roll logs on active node every " +
           (logRollPeriodMs / 1000) + " seconds.");
     } else {
       LOG.info("Not going to trigger log rolls on active node because " +
@@ -125,29 +153,24 @@ public class EditLogTailer {
     
     sleepTimeMs = conf.getInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY,
         DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT) * 1000;
-    
+
+    maxRetries = conf.getInt(DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY,
+      DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT);
+    if (maxRetries <= 0) {
+      LOG.error("Specified a non-positive number of retries for the number of retries for the " +
+          "namenode connection when manipulating the edit log (" +
+          DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY + "), setting to default: " +
+          DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT);
+      maxRetries = DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT;
+    }
+
+    nnCount = nns.size();
+    // setup the iterator to endlessly loop the nns
+    this.nnLookup = Iterators.cycle(nns);
+
     LOG.debug("logRollPeriodMs=" + logRollPeriodMs +
         " sleepTime=" + sleepTimeMs);
   }
-  
-  private InetSocketAddress getActiveNodeAddress() {
-    Configuration activeConf = HAUtil.getConfForOtherNode(conf);
-    return NameNode.getServiceAddress(activeConf, true);
-  }
-  
-  private NamenodeProtocol getActiveNodeProxy() throws IOException {
-    if (cachedActiveProxy == null) {
-      int rpcTimeout = conf.getInt(
-          DFSConfigKeys.DFS_HA_LOGROLL_RPC_TIMEOUT_KEY,
-          DFSConfigKeys.DFS_HA_LOGROLL_RPC_TIMEOUT_DEFAULT);
-      NamenodeProtocolPB proxy = RPC.waitForProxy(NamenodeProtocolPB.class,
-          RPC.getProtocolVersion(NamenodeProtocolPB.class), activeAddr, conf,
-          rpcTimeout, Long.MAX_VALUE);
-      cachedActiveProxy = new NamenodeProtocolTranslatorPB(proxy);
-    }
-    assert cachedActiveProxy != null;
-    return cachedActiveProxy;
-  }
 
   public void start() {
     tailerThread.start();
@@ -270,9 +293,15 @@ public class EditLogTailer {
    * Trigger the active node to roll its logs.
    */
   private void triggerActiveLogRoll() {
-    LOG.info("Triggering log roll on remote NameNode " + activeAddr);
+    LOG.info("Triggering log roll on remote NameNode");
     try {
-      getActiveNodeProxy().rollEditLog();
+      new MultipleNameNodeProxy<Void>() {
+        @Override
+        protected Void doWork() throws IOException {
+          cachedActiveProxy.rollEditLog();
+          return null;
+        }
+      }.call();
       lastRollTriggerTxId = lastLoadedTxnId;
     } catch (IOException ioe) {
       if (ioe instanceof RemoteException) {
@@ -362,5 +391,76 @@ public class EditLogTailer {
       }
     }
   }
+  /**
+   * Manage the 'active namenode proxy'. This cannot just be the a single proxy since we could
+   * failover across a number of NameNodes, rather than just between an active and a standby.
+   * <p>
+   * We - lazily - get a proxy to one of the configured namenodes and attempt to make the request
+   * against it. If it doesn't succeed, either because the proxy failed to be created or the request
+   * failed, we try the next NN in the list. We try this up to the configuration maximum number of
+   * retries before throwing up our hands. A working proxy is retained across attempts since we
+   * expect the active NameNode to switch rarely.
+   * <p>
+   * This mechanism is <b>very bad</b> for cases where we care about being <i>fast</i>; it just
+   * blindly goes and tries namenodes.
+   */
+  private abstract class MultipleNameNodeProxy<T> implements Callable<T> {
+
+    /**
+     * Do the actual work to the remote namenode via the {@link #cachedActiveProxy}.
+     * @return the result of the work, if there is one
+     * @throws IOException if the actions done to the proxy throw an exception.
+     */
+    protected abstract T doWork() throws IOException;
+
+    public T call() throws IOException {
+      while ((cachedActiveProxy = getActiveNodeProxy()) != null) {
+        try {
+          T ret = doWork();
+          // reset the loop count on success
+          nnLoopCount = 0;
+          return ret;
+        } catch (RemoteException e) {
+          Throwable cause = e.unwrapRemoteException(StandbyException.class);
+          // if its not a standby exception, then we need to re-throw it, something bad has happened
+          if (cause == e) {
+            throw e;
+          } else {
+            // it is a standby exception, so we try the other NN
+            LOG.warn("Failed to reach remote node: " + currentNN
+                + ", retrying with remaining remote NNs");
+            cachedActiveProxy = null;
+            // this NN isn't responding to requests, try the next one
+            nnLoopCount++;
+          }
+        }
+      }
+      throw new IOException("Cannot find any valid remote NN to service request!");
+    }
 
-}
+    private NamenodeProtocol getActiveNodeProxy() throws IOException {
+      if (cachedActiveProxy == null) {
+        while (true) {
+          // if we have reached the max loop count, quit by returning null
+          if ((nnLoopCount / nnCount) >= maxRetries) {
+            return null;
+          }
+
+          currentNN = nnLookup.next();
+          try {
+            NamenodeProtocolPB proxy = RPC.waitForProxy(NamenodeProtocolPB.class,
+                RPC.getProtocolVersion(NamenodeProtocolPB.class), currentNN.getIpcAddress(), conf);
+            cachedActiveProxy = new NamenodeProtocolTranslatorPB(proxy);
+            break;
+          } catch (IOException e) {
+            LOG.info("Failed to reach " + currentNN, e);
+            // couldn't even reach this NN, try the next one
+            nnLoopCount++;
+          }
+        }
+      }
+      assert cachedActiveProxy != null;
+      return cachedActiveProxy;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
new file mode 100644
index 0000000..9a51190
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.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 java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Information about a single remote NameNode
+ */
+public class RemoteNameNodeInfo {
+
+  public static List<RemoteNameNodeInfo> getRemoteNameNodes(Configuration conf) throws IOException {
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    return getRemoteNameNodes(conf, nsId);
+  }
+
+  public static List<RemoteNameNodeInfo> getRemoteNameNodes(Configuration conf, String nsId)
+      throws IOException {
+    // there is only a single NN configured (and no federation) so we don't have any more NNs
+    if (nsId == null) {
+      return Collections.emptyList();
+    }
+    List<Configuration> otherNodes = HAUtil.getConfForOtherNodes(conf);
+    List<RemoteNameNodeInfo> nns = new ArrayList<RemoteNameNodeInfo>();
+
+    for (Configuration otherNode : otherNodes) {
+      String otherNNId = HAUtil.getNameNodeId(otherNode, nsId);
+      // don't do any validation here as in some cases, it can be overwritten later
+      InetSocketAddress otherIpcAddr = NameNode.getServiceAddress(otherNode, true);
+
+
+      final String scheme = DFSUtil.getHttpClientScheme(conf);
+      URL otherHttpAddr = DFSUtil.getInfoServerWithDefaultHost(otherIpcAddr.getHostName(),
+          otherNode, scheme).toURL();
+
+      nns.add(new RemoteNameNodeInfo(otherNode, otherNNId, otherIpcAddr, otherHttpAddr));
+    }
+    return nns;
+  }
+
+  private final Configuration conf;
+  private final String nnId;
+  private InetSocketAddress ipcAddress;
+  private final URL httpAddress;
+
+  private RemoteNameNodeInfo(Configuration conf, String nnId, InetSocketAddress ipcAddress,
+      URL httpAddress) {
+    this.conf = conf;
+    this.nnId = nnId;
+    this.ipcAddress = ipcAddress;
+    this.httpAddress = httpAddress;
+  }
+
+  public InetSocketAddress getIpcAddress() {
+    return this.ipcAddress;
+  }
+
+  public String getNameNodeID() {
+    return this.nnId;
+  }
+
+  public URL getHttpAddress() {
+    return this.httpAddress;
+  }
+
+  public Configuration getConfiguration() {
+    return this.conf;
+  }
+
+  public void setIpcAddress(InetSocketAddress ipc) {
+    this.ipcAddress = ipc;
+  }
+
+  @Override
+  public String toString() {
+    return "RemoteNameNodeInfo [nnId=" + nnId + ", ipcAddress=" + ipcAddress
+        + ", httpAddress=" + httpAddress + "]";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    RemoteNameNodeInfo that = (RemoteNameNodeInfo) o;
+
+    if (!nnId.equals(that.nnId)) return false;
+    if (!ipcAddress.equals(that.ipcAddress)) return false;
+    // convert to the standard strings since URL.equals does address resolution, which is a
+    // blocking call and a a FindBugs issue.
+    String httpString = httpAddress.toString();
+    String thatHttpString  = that.httpAddress.toString();
+    return httpString.equals(thatHttpString);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = nnId.hashCode();
+    result = 31 * result + ipcAddress.hashCode();
+    // toString rather than hashCode b/c Url.hashCode is a blocking call.
+    result = 31 * result + httpAddress.toString().hashCode();
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
----------------------------------------------------------------------
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
index 1e40368..f5ecbec 100644
--- 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
@@ -23,12 +23,10 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedAction;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -45,6 +43,7 @@ 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.hdfs.util.Canceler;
+import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -68,12 +67,13 @@ public class StandbyCheckpointer {
   private long lastCheckpointTime;
   private final CheckpointerThread thread;
   private final ThreadFactory uploadThreadFactory;
-  private URL activeNNAddress;
+  private List<URL> activeNNAddresses;
   private URL myNNAddress;
 
   private final Object cancelLock = new Object();
   private Canceler canceler;
-  
+  private boolean isPrimaryCheckPointer = true;
+
   // Keep track of how many checkpoints were canceled.
   // This is for use in tests.
   private static int canceledCount = 0;
@@ -100,14 +100,21 @@ public class StandbyCheckpointer {
     myNNAddress = getHttpAddress(conf);
 
     // Look up the active node's address
-    Configuration confForActive = HAUtil.getConfForOtherNode(conf);
-    activeNNAddress = getHttpAddress(confForActive);
-    
+    List<Configuration> confForActive = HAUtil.getConfForOtherNodes(conf);
+    activeNNAddresses = new ArrayList<URL>(confForActive.size());
+    for (Configuration activeConf : confForActive) {
+      URL activeNNAddress = getHttpAddress(activeConf);
+
+      // sanity check each possible active NN
+      Preconditions.checkArgument(checkAddress(activeNNAddress),
+          "Bad address for active NN: %s", activeNNAddress);
+
+      activeNNAddresses.add(activeNNAddress);
+    }
+
     // Sanity-check.
-    Preconditions.checkArgument(checkAddress(activeNNAddress),
-        "Bad address for active NN: %s", activeNNAddress);
-    Preconditions.checkArgument(checkAddress(myNNAddress),
-        "Bad address for standby NN: %s", myNNAddress);
+    Preconditions.checkArgument(checkAddress(myNNAddress), "Bad address for standby NN: %s",
+        myNNAddress);
   }
   
   private URL getHttpAddress(Configuration conf) throws IOException {
@@ -127,7 +134,7 @@ public class StandbyCheckpointer {
 
   public void start() {
     LOG.info("Starting standby checkpoint thread...\n" +
-        "Checkpointing active NN at " + activeNNAddress + "\n" +
+        "Checkpointing active NN to possible NNs: " + activeNNAddresses + "\n" +
         "Serving checkpoints at " + myNNAddress);
     thread.start();
   }
@@ -148,11 +155,10 @@ public class StandbyCheckpointer {
     thread.interrupt();
   }
 
-  private void doCheckpoint() throws InterruptedException, IOException {
+  private void doCheckpoint(boolean sendCheckpoint) throws InterruptedException, IOException {
     assert canceler != null;
     final long txid;
     final NameNodeFile imageType;
-    
     // Acquire cpLock to make sure no one is modifying the name system.
     // It does not need the full namesystem write lock, since the only thing
     // that modifies namesystem on standby node is edit log replaying.
@@ -161,9 +167,9 @@ public class StandbyCheckpointer {
       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;
@@ -185,7 +191,7 @@ public class StandbyCheckpointer {
       img.saveNamespace(namesystem, imageType, canceler);
       txid = img.getStorage().getMostRecentCheckpointTxId();
       assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +
-        thisCheckpointTxId + " but instead saved at txid=" + txid;
+          thisCheckpointTxId + " but instead saved at txid=" + txid;
 
       // Save the legacy OIV image, if the output dir is defined.
       String outputDir = checkpointConf.getLegacyOivImageDir();
@@ -195,31 +201,85 @@ public class StandbyCheckpointer {
     } finally {
       namesystem.cpUnlock();
     }
-    
+
+    //early exit if we shouldn't actually send the checkpoint to the ANN
+    if(!sendCheckpoint){
+      return;
+    }
+
     // Upload the saved checkpoint back to the active
-    // Do this in a separate thread to avoid blocking transition to active
+    // Do this in a separate thread to avoid blocking transition to active, but don't allow more
+    // than the expected number of tasks to run or queue up
     // See HDFS-4816
-    ExecutorService executor =
-        Executors.newSingleThreadExecutor(uploadThreadFactory);
-    Future<Void> upload = executor.submit(new Callable<Void>() {
-      @Override
-      public Void call() throws IOException {
-        TransferFsImage.uploadImageFromStorage(activeNNAddress, conf,
-            namesystem.getFSImage().getStorage(), imageType, txid, canceler);
-        return null;
+    ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100,
+        TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()),
+        uploadThreadFactory);
+    // for right now, just match the upload to the nn address by convention. There is no need to
+    // directly tie them together by adding a pair class.
+    List<Future<TransferFsImage.TransferResult>> uploads =
+        new ArrayList<Future<TransferFsImage.TransferResult>>();
+    for (final URL activeNNAddress : activeNNAddresses) {
+      Future<TransferFsImage.TransferResult> upload =
+          executor.submit(new Callable<TransferFsImage.TransferResult>() {
+            @Override
+            public TransferFsImage.TransferResult call() throws IOException {
+              return TransferFsImage.uploadImageFromStorage(activeNNAddress, conf, namesystem
+                  .getFSImage().getStorage(), imageType, txid, canceler);
+            }
+          });
+      uploads.add(upload);
+    }
+    InterruptedException ie = null;
+    IOException ioe= null;
+    int i = 0;
+    boolean success = false;
+    for (; i < uploads.size(); i++) {
+      Future<TransferFsImage.TransferResult> upload = uploads.get(i);
+      try {
+        // TODO should there be some smarts here about retries nodes that are not the active NN?
+        if (upload.get() == TransferFsImage.TransferResult.SUCCESS) {
+          success = true;
+          //avoid getting the rest of the results - we don't care since we had a successful upload
+          break;
+        }
+
+      } catch (ExecutionException e) {
+        ioe = new IOException("Exception during image upload: " + e.getMessage(),
+            e.getCause());
+        break;
+      } catch (InterruptedException e) {
+        ie = e;
+        break;
+      }
+    }
+
+    // we are primary if we successfully updated the ANN
+    this.isPrimaryCheckPointer = success;
+
+    // cleaner than copying code for multiple catch statements and better than catching all
+    // exceptions, so we just handle the ones we expect.
+    if (ie != null || ioe != null) {
+
+      // cancel the rest of the tasks, and close the pool
+      for (; i < uploads.size(); i++) {
+        Future<TransferFsImage.TransferResult> upload = uploads.get(i);
+        // The background thread may be blocked waiting in the throttler, so
+        // interrupt it.
+        upload.cancel(true);
+      }
+
+      // shutdown so we interrupt anything running and don't start anything new
+      executor.shutdownNow();
+      // this is a good bit longer than the thread timeout, just to make sure all the threads
+      // that are not doing any work also stop
+      executor.awaitTermination(500, TimeUnit.MILLISECONDS);
+
+      // re-throw the exception we got, since one of these two must be non-null
+      if (ie != null) {
+        throw ie;
+      } else if (ioe != null) {
+        throw ioe;
       }
-    });
-    executor.shutdown();
-    try {
-      upload.get();
-    } catch (InterruptedException e) {
-      // The background thread may be blocked waiting in the throttler, so
-      // interrupt it.
-      upload.cancel(true);
-      throw e;
-    } catch (ExecutionException e) {
-      throw new IOException("Exception during image upload: " + e.getMessage(),
-          e.getCause());
     }
   }
   
@@ -322,8 +382,10 @@ public class StandbyCheckpointer {
           final long now = monotonicNow();
           final long uncheckpointed = countUncheckpointedTxns();
           final long secsSinceLast = (now - lastCheckpointTime) / 1000;
-          
+
+          // if we need a rollback checkpoint, always attempt to checkpoint
           boolean needCheckpoint = needRollbackCheckpoint;
+
           if (needCheckpoint) {
             LOG.info("Triggering a rollback fsimage for rolling upgrade.");
           } else if (uncheckpointed >= checkpointConf.getTxnCount()) {
@@ -338,19 +400,23 @@ public class StandbyCheckpointer {
                 "exceeds the configured interval " + checkpointConf.getPeriod());
             needCheckpoint = true;
           }
-          
-          synchronized (cancelLock) {
-            if (now < preventCheckpointsUntil) {
-              LOG.info("But skipping this checkpoint since we are about to failover!");
-              canceledCount++;
-              continue;
-            }
-            assert canceler == null;
-            canceler = new Canceler();
-          }
-          
+
           if (needCheckpoint) {
-            doCheckpoint();
+            synchronized (cancelLock) {
+              if (now < preventCheckpointsUntil) {
+                LOG.info("But skipping this checkpoint since we are about to failover!");
+                canceledCount++;
+                continue;
+              }
+              assert canceler == null;
+              canceler = new Canceler();
+            }
+
+            // on all nodes, we build the checkpoint. However, we only ship the checkpoint if have a
+            // rollback request, are the checkpointer, are outside the quiet period.
+            boolean sendRequest = isPrimaryCheckPointer || secsSinceLast >= checkpointConf.getQuietPeriod();
+            doCheckpoint(sendRequest);
+
             // reset needRollbackCheckpoint to false only when we finish a ckpt
             // for rollback image
             if (needRollbackCheckpoint
@@ -379,7 +445,7 @@ public class StandbyCheckpointer {
   }
 
   @VisibleForTesting
-  URL getActiveNNAddress() {
-    return activeNNAddress;
+  List<URL> getActiveNNAddresses() {
+    return activeNNAddresses;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
index f125a27..24e5bef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
@@ -25,6 +25,8 @@ import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
@@ -261,4 +263,15 @@ public class DFSZKFailoverController extends ZKFailoverController {
     return isThreadDumpCaptured;
   }
 
+  @Override
+  public List<HAServiceTarget> getAllOtherNodes() {
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    List<String> otherNn = HAUtil.getNameNodeIdOfOtherNodes(conf, nsId);
+
+    List<HAServiceTarget> targets = new ArrayList<HAServiceTarget>(otherNn.size());
+    for (String nnId : otherNn) {
+      targets.add(new NNHAServiceTarget(conf, nsId, nnId));
+    }
+    return targets;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
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 aaa1c2f..76161a5 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
@@ -914,6 +914,18 @@
 </property>
 
 <property>
+  <name>dfs.namenode.checkpoint.check.quiet-multiplier</name>
+  <value>1.5</value>
+  <description>
+    Used to calculate the amount of time between retries when in the 'quiet' period
+    for creating checkpoints (active namenode already has an up-to-date image from another
+    checkpointer), so we wait a multiplier of the dfs.namenode.checkpoint.check.period before
+    retrying the checkpoint because another node likely is already managing the checkpoints,
+    allowing us to save bandwidth to transfer checkpoints that don't need to be used.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.num.checkpoints.retained</name>
   <value>2</value>
   <description>The number of image checkpoint files (fsimage_*) that will be retained by
@@ -1288,6 +1300,14 @@
 </property>
 
 <property>
+  <name>dfs.ha.tail-edits.namenode-retries</name>
+  <value>3</value>
+  <description>
+    Number of retries to use when contacting the namenode when tailing the log.
+  </description>
+</property>
+
+<property>
   <name>dfs.ha.automatic-failover.enabled</name>
   <value>false</value>
   <description>


[07/21] hadoop git commit: HDFS-8542. WebHDFS getHomeDirectory behavior does not match specification. Contributed by Kanaka Kumar Avvaru.

Posted by ar...@apache.org.
HDFS-8542. WebHDFS getHomeDirectory behavior does not match specification. Contributed by  Kanaka Kumar Avvaru.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fac4e04d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fac4e04d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fac4e04d

Branch: refs/heads/HDFS-7240
Commit: fac4e04dd359a7ff31f286d664fb06f019ec0b58
Parents: 077250d
Author: Jakob Homan <jg...@gmail.com>
Authored: Mon Jun 22 16:30:45 2015 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Mon Jun 22 16:30:45 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  | 10 ++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 33 +++++++++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../web/resources/NamenodeWebHdfsMethods.java   | 12 ++--
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 59 ++++++++++++++++++++
 .../hdfs/web/TestWebHdfsFileSystemContract.java |  2 +-
 6 files changed, 108 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac4e04d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index ca94840..e025e31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -389,6 +389,16 @@ class JsonUtilClient {
     return aclStatusBuilder.build();
   }
 
+  static String getPath(final Map<?, ?> json)
+      throws IOException {
+    if (json == null) {
+      return null;
+    }
+
+    String path = (String) json.get("Path");
+    return path;
+  }
+
   static byte[] getXAttr(final Map<?, ?> json, final String name)
       throws IOException {
     if (json == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac4e04d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index aebd25a..d902738 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -110,6 +110,7 @@ public class WebHdfsFileSystem extends FileSystem
   protected Text tokenServiceName;
   private RetryPolicy retryPolicy = null;
   private Path workingDir;
+  private Path cachedHomeDirectory;
   private InetSocketAddress nnAddrs[];
   private int currentNNAddrIndex;
   private boolean disallowFallbackToInsecureCluster;
@@ -193,7 +194,7 @@ public class WebHdfsFileSystem extends FileSystem
               failoverSleepMaxMillis);
     }
 
-    this.workingDir = getHomeDirectory();
+    this.workingDir = makeQualified(new Path(getHomeDirectoryString(ugi)));
     this.canRefreshDelegationToken = UserGroupInformation.isSecurityEnabled();
     this.disallowFallbackToInsecureCluster = !conf.getBoolean(
         CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
@@ -267,14 +268,35 @@ public class WebHdfsFileSystem extends FileSystem
     return NetUtils.getCanonicalUri(uri, getDefaultPort());
   }
 
-  /** @return the home directory. */
+  /** @return the home directory */
+  @Deprecated
   public static String getHomeDirectoryString(final UserGroupInformation ugi) {
     return "/user/" + ugi.getShortUserName();
   }
 
   @Override
   public Path getHomeDirectory() {
-    return makeQualified(new Path(getHomeDirectoryString(ugi)));
+    if (cachedHomeDirectory == null) {
+      final HttpOpParam.Op op = GetOpParam.Op.GETHOMEDIRECTORY;
+      try {
+        String pathFromDelegatedFS = new FsPathResponseRunner<String>(op, null,
+            new UserParam(ugi)) {
+          @Override
+          String decodeResponse(Map<?, ?> json) throws IOException {
+            return JsonUtilClient.getPath(json);
+          }
+        }   .run();
+
+        cachedHomeDirectory = new Path(pathFromDelegatedFS).makeQualified(
+            this.getUri(), null);
+
+      } catch (IOException e) {
+        LOG.error("Unable to get HomeDirectory from original File System", e);
+        cachedHomeDirectory = new Path("/user/" + ugi.getShortUserName())
+            .makeQualified(this.getUri(), null);
+      }
+    }
+    return cachedHomeDirectory;
   }
 
   @Override
@@ -284,12 +306,13 @@ public class WebHdfsFileSystem extends FileSystem
 
   @Override
   public synchronized void setWorkingDirectory(final Path dir) {
-    String result = makeAbsolute(dir).toUri().getPath();
+    Path absolutePath = makeAbsolute(dir);
+    String result = absolutePath.toUri().getPath();
     if (!DFSUtilClient.isValidName(result)) {
       throw new IllegalArgumentException("Invalid DFS directory name " +
                                          result);
     }
-    workingDir = makeAbsolute(dir);
+    workingDir = absolutePath;
   }
 
   private Path makeAbsolute(Path f) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac4e04d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b9d9943..4bd0e8b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -938,6 +938,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
     Blocks for Lower-Priority Blocks (Derek Dagit via kihwal)
 
+    HDFS-8542. WebHDFS getHomeDirectory behavior does not match specification.
+    (Kanaka Kumar Avvaru via jghoman)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac4e04d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index d33721c..6e880f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -53,6 +53,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -826,6 +827,8 @@ public class NamenodeWebHdfsMethods {
       final TokenServiceParam tokenService
       ) throws IOException, URISyntaxException {
     final NameNode namenode = (NameNode)context.getAttribute("name.node");
+    final Configuration conf = (Configuration) context
+        .getAttribute(JspHelper.CURRENT_CONF);
     final NamenodeProtocols np = getRPCServer(namenode);
 
     switch(op.getValue()) {
@@ -892,11 +895,10 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(token);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
-    case GETHOMEDIRECTORY:
-    {
-      final String js = JsonUtil.toJsonString(
-          org.apache.hadoop.fs.Path.class.getSimpleName(),
-          WebHdfsFileSystem.getHomeDirectoryString(ugi));
+    case GETHOMEDIRECTORY: {
+      final String js = JsonUtil.toJsonString("Path",
+          FileSystem.get(conf != null ? conf : new Configuration())
+              .getHomeDirectory().toUri().getPath());
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
     case GETACLSTATUS: {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac4e04d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 0c963f1..0563f12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.hdfs.web;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
@@ -557,4 +560,60 @@ public class TestWebHDFS {
       }
     }
   }
+
+  @Test(timeout = 30000)
+  public void testGetHomeDirectory() throws Exception {
+
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      DistributedFileSystem hdfs = cluster.getFileSystem();
+
+      final URI uri = new URI(WebHdfsConstants.WEBHDFS_SCHEME + "://"
+          + cluster.getHttpUri(0).replace("http://", ""));
+      final Configuration confTemp = new Configuration();
+
+      {
+        WebHdfsFileSystem webhdfs = (WebHdfsFileSystem) FileSystem.get(uri,
+            confTemp);
+
+        assertEquals(hdfs.getHomeDirectory().toUri().getPath(), webhdfs
+            .getHomeDirectory().toUri().getPath());
+
+        webhdfs.close();
+      }
+
+      {
+        WebHdfsFileSystem webhdfs = createWebHDFSAsTestUser(confTemp, uri,
+            "XXX");
+
+        assertNotEquals(hdfs.getHomeDirectory().toUri().getPath(), webhdfs
+            .getHomeDirectory().toUri().getPath());
+
+        webhdfs.close();
+      }
+
+    } finally {
+      if (cluster != null)
+        cluster.shutdown();
+    }
+  }
+
+  private WebHdfsFileSystem createWebHDFSAsTestUser(final Configuration conf,
+      final URI uri, final String userName) throws Exception {
+
+    final UserGroupInformation ugi = UserGroupInformation.createUserForTesting(
+        userName, new String[] { "supergroup" });
+
+    return ugi.doAs(new PrivilegedExceptionAction<WebHdfsFileSystem>() {
+      @Override
+      public WebHdfsFileSystem run() throws IOException {
+        WebHdfsFileSystem webhdfs = (WebHdfsFileSystem) FileSystem.get(uri,
+            conf);
+        return webhdfs;
+      }
+    });
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac4e04d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
index bc10bca..5d20014 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
@@ -399,7 +399,7 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
       final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
       final Map<?, ?> m = WebHdfsTestUtil.connectAndGetJson(
           conn, HttpServletResponse.SC_OK);
-      assertEquals(WebHdfsFileSystem.getHomeDirectoryString(ugi),
+      assertEquals(webhdfs.getHomeDirectory().toUri().getPath(),
           m.get(Path.class.getSimpleName()));
       conn.disconnect();
     }


[05/21] hadoop git commit: MAPREDUCE-6408. Queue name and user name should be printed on the job page. (Siqi Li via gera)

Posted by ar...@apache.org.
MAPREDUCE-6408. Queue name and user name should be printed on the job page. (Siqi Li via gera)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/11ac8482
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/11ac8482
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/11ac8482

Branch: refs/heads/HDFS-7240
Commit: 11ac84820746e43f1e7072e08f62ddf81713aeb2
Parents: 7b424f9
Author: Gera Shegalov <ge...@apache.org>
Authored: Mon Jun 22 14:23:22 2015 -0700
Committer: Gera Shegalov <ge...@apache.org>
Committed: Mon Jun 22 14:40:32 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                           | 3 +++
 .../org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java    | 2 ++
 .../org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java | 6 ++++++
 .../hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java  | 2 +-
 4 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11ac8482/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index c42e410..bae2674 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -358,6 +358,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6305. AM/Task log page should be able to link back to the job.
     (Siqi Li via gera)
 
+    MAPREDUCE-6408. Queue name and user name should be printed on the job page.
+    (Siqi Li via gera)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11ac8482/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
index 0c6d773..a599870 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
@@ -78,6 +78,8 @@ public class JobBlock extends HtmlBlock {
     JobInfo jinfo = new JobInfo(job, true);
     info("Job Overview").
         _("Job Name:", jinfo.getName()).
+        _("User Name:", jinfo.getUserName()).
+        _("Queue Name:", jinfo.getQueueName()).
         _("State:", jinfo.getState()).
         _("Uberized:", jinfo.isUberized()).
         _("Started:", new Date(jinfo.getStartTime())).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11ac8482/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
index d823cad..ffb243b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
@@ -51,6 +51,7 @@ public class JobInfo {
   protected String id;
   protected String name;
   protected String user;
+  protected String queue;
   protected JobState state;
   protected int mapsTotal;
   protected int mapsCompleted;
@@ -97,6 +98,7 @@ public class JobInfo {
     }
     this.name = job.getName().toString();
     this.user = job.getUserName();
+    this.queue = job.getQueueName();
     this.state = job.getState();
     this.mapsTotal = job.getTotalMaps();
     this.mapsCompleted = job.getCompletedMaps();
@@ -223,6 +225,10 @@ public class JobInfo {
     return this.name;
   }
 
+  public String getQueueName() {
+    return this.queue;
+  }
+
   public String getId() {
     return this.id;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11ac8482/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
index 2783379..a61e407 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
@@ -413,7 +413,7 @@ public class TestAMWebServicesJobs extends JerseyTest {
 
   public void verifyAMJob(JSONObject info, Job job) throws JSONException {
 
-    assertEquals("incorrect number of elements", 30, info.length());
+    assertEquals("incorrect number of elements", 31, info.length());
 
     // everyone access fields
     verifyAMJobGeneric(job, info.getString("id"), info.getString("user"),


[19/21] hadoop git commit: YARN-3832. Resource Localization fails on a cluster due to existing cache directories. Contributed by Brahma Reddy Battula

Posted by ar...@apache.org.
YARN-3832. Resource Localization fails on a cluster due to existing cache directories. Contributed by Brahma Reddy Battula


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8d58512d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d58512d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d58512d

Branch: refs/heads/HDFS-7240
Commit: 8d58512d6e6d9fe93784a9de2af0056bcc316d96
Parents: 2a20dd9
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 24 16:37:39 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 24 16:37:39 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../containermanager/localizer/ResourceLocalizationService.java   | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d58512d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7ecdee3..a5fc86b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -702,6 +702,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3809. Failed to launch new attempts because
     ApplicationMasterLauncher's threads all hang (Jun Gong via jlowe)
 
+    YARN-3832. Resource Localization fails on a cluster due to existing cache
+    directories (Brahma Reddy Battula via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d58512d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index d6e0903..b417c5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -1340,7 +1340,7 @@ public class ResourceLocalizationService extends CompositeService
   }
 
   private void cleanUpLocalDirs(FileContext lfs, DeletionService del) {
-    for (String localDir : dirsHandler.getLocalDirs()) {
+    for (String localDir : dirsHandler.getLocalDirsForCleanup()) {
       cleanUpLocalDir(lfs, del, localDir);
     }
   }


[16/21] hadoop git commit: HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager. Contributed by Kai Sasaki.

Posted by ar...@apache.org.
HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager. Contributed by Kai Sasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2ba64657
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2ba64657
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2ba64657

Branch: refs/heads/HDFS-7240
Commit: 2ba646572185b91d6db1b09837abdcbadbfbeb49
Parents: 49dfad9
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Jun 23 22:16:16 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Tue Jun 23 22:16:16 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../test/java/org/apache/hadoop/test/MiniDFSClusterManager.java  | 4 ++++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ba64657/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 09a6891..d94a213 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -663,6 +663,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6564. Use slf4j instead of common-logging in hdfs-client.
     (Rakesh R via wheat9)
 
+    HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager.
+    (Kai Sasaki via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ba64657/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
index 7029f42..f8e931a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
@@ -65,6 +65,7 @@ public class MiniDFSClusterManager {
   private String writeDetails;
   private int numDataNodes;
   private int nameNodePort;
+  private int nameNodeHttpPort;
   private StartupOption dfsOpts;
   private String writeConfig;
   private Configuration conf;
@@ -84,6 +85,7 @@ public class MiniDFSClusterManager {
         .addOption("cmdport", true,
             "Which port to listen on for commands (default 0--we choose)")
         .addOption("nnport", true, "NameNode port (default 0--we choose)")
+        .addOption("httpport", true, "NameNode http port (default 0--we choose)")
         .addOption("namenode", true, "URL of the namenode (default "
             + "is either the DFS cluster or a temporary dir)")     
         .addOption(OptionBuilder
@@ -137,6 +139,7 @@ public class MiniDFSClusterManager {
    */
   public void start() throws IOException, FileNotFoundException {
     dfs = new MiniDFSCluster.Builder(conf).nameNodePort(nameNodePort)
+                                          .nameNodeHttpPort(nameNodeHttpPort)
                                           .numDataNodes(numDataNodes)
                                           .startupOption(dfsOpts)
                                           .format(format)
@@ -198,6 +201,7 @@ public class MiniDFSClusterManager {
     // HDFS
     numDataNodes = intArgument(cli, "datanodes", 1);
     nameNodePort = intArgument(cli, "nnport", 0);
+    nameNodeHttpPort = intArgument(cli, "httpport", 0);
     if (cli.hasOption("format")) {
       dfsOpts = StartupOption.FORMAT;
       format = true;