浏览代码

Merge r1609845 through r1611734 from tunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-6584@1611736 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 年之前
父节点
当前提交
2b07af0c59
共有 21 个文件被更改,包括 438 次插入156 次删除
  1. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 23 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
  3. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
  4. 12 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  5. 169 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java
  6. 21 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  7. 34 31
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  8. 15 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  9. 16 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java
  10. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  11. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  12. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  13. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  15. 16 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  16. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  17. 29 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  18. 19 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  19. 18 19
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
  20. 18 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java
  21. 6 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java

+ 6 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -177,6 +177,12 @@ Trunk (Unreleased)
 
     HADOOP-10824. Refactor KMSACLs to avoid locking. (Benoy Antony via umamahesh)
 
+    HADOOP-10841. EncryptedKeyVersion should have a key name property. 
+    (asuresh via tucu)
+
+    HADOOP-10842. CryptoExtension generateEncryptedKey method should 
+    receive the key name. (asuresh via tucu)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.

+ 23 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java

@@ -44,17 +44,23 @@ public class KeyProviderCryptoExtension extends
    * used to generate the encrypted Key and the encrypted KeyVersion
    */
   public static class EncryptedKeyVersion {
+    private String keyName;
     private String keyVersionName;
     private byte[] iv;
     private KeyVersion encryptedKey;
 
-    protected EncryptedKeyVersion(String keyVersionName, byte[] iv,
-        KeyVersion encryptedKey) {
+    protected EncryptedKeyVersion(String keyName, String keyVersionName,
+        byte[] iv, KeyVersion encryptedKey) {
+      this.keyName = keyName;
       this.keyVersionName = keyVersionName;
       this.iv = iv;
       this.encryptedKey = encryptedKey;
     }
 
+    public String getKeyName() {
+      return keyName;
+    }
+
     public String getKeyVersionName() {
       return keyVersionName;
     }
@@ -78,14 +84,13 @@ public class KeyProviderCryptoExtension extends
     /**
      * Generates a key material and encrypts it using the given key version name
      * and initialization vector. The generated key material is of the same
-     * length as the <code>KeyVersion</code> material and is encrypted using the
-     * same cipher.
+     * length as the <code>KeyVersion</code> material of the latest key version
+     * of the key and is encrypted using the same cipher.
      * <p/>
      * NOTE: The generated key is not stored by the <code>KeyProvider</code>
      * 
-     * @param encryptionKeyVersion
-     *          a KeyVersion object containing the keyVersion name and material
-     *          to encrypt.
+     * @param encryptionKeyName
+     *          The latest KeyVersion of this key's material will be encrypted.
      * @return EncryptedKeyVersion with the generated key material, the version
      *         name is 'EEK' (for Encrypted Encryption Key)
      * @throws IOException
@@ -95,7 +100,7 @@ public class KeyProviderCryptoExtension extends
      *           cryptographic issue.
      */
     public EncryptedKeyVersion generateEncryptedKey(
-        KeyVersion encryptionKeyVersion) throws IOException,
+        String encryptionKeyName) throws IOException,
         GeneralSecurityException;
 
     /**
@@ -140,12 +145,11 @@ public class KeyProviderCryptoExtension extends
     }
 
     @Override
-    public EncryptedKeyVersion generateEncryptedKey(KeyVersion keyVersion)
+    public EncryptedKeyVersion generateEncryptedKey(String encryptionKeyName)
         throws IOException, GeneralSecurityException {
-      KeyVersion keyVer =
-          keyProvider.getKeyVersion(keyVersion.getVersionName());
-      Preconditions.checkNotNull(keyVer, "KeyVersion name '%s' does not exist",
-          keyVersion.getVersionName());
+      KeyVersion keyVer = keyProvider.getCurrentKey(encryptionKeyName);
+      Preconditions.checkNotNull(keyVer, "No KeyVersion exists for key '%s' ",
+          encryptionKeyName);
       byte[] newKey = new byte[keyVer.getMaterial().length];
       SecureRandom.getInstance("SHA1PRNG").nextBytes(newKey);
       Cipher cipher = Cipher.getInstance("AES/CTR/NoPadding");
@@ -153,7 +157,8 @@ public class KeyProviderCryptoExtension extends
       cipher.init(Cipher.ENCRYPT_MODE, new SecretKeySpec(keyVer.getMaterial(),
           "AES"), new IvParameterSpec(flipIV(iv)));
       byte[] ek = cipher.doFinal(newKey);
-      return new EncryptedKeyVersion(keyVersion.getVersionName(), iv,
+      return new EncryptedKeyVersion(encryptionKeyName,
+          keyVer.getVersionName(), iv,
           new KeyVersion(keyVer.getName(), EEK, ek));
     }
 
@@ -190,18 +195,18 @@ public class KeyProviderCryptoExtension extends
    * <p/>
    * NOTE: The generated key is not stored by the <code>KeyProvider</code>
    *
-   * @param encryptionKey a KeyVersion object containing the keyVersion name and 
-   * material to encrypt.
+   * @param encryptionKeyName The latest KeyVersion of this key's material will
+   * be encrypted.
    * @return EncryptedKeyVersion with the generated key material, the version
    * name is 'EEK' (for Encrypted Encryption Key)
    * @throws IOException thrown if the key material could not be generated
    * @throws GeneralSecurityException thrown if the key material could not be 
    * encrypted because of a cryptographic issue.
    */
-  public EncryptedKeyVersion generateEncryptedKey(KeyVersion encryptionKey) 
+  public EncryptedKeyVersion generateEncryptedKey(String encryptionKeyName)
       throws IOException,
                                            GeneralSecurityException {
-    return getExtension().generateEncryptedKey(encryptionKey);
+    return getExtension().generateEncryptedKey(encryptionKeyName);
   }
 
   /**

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java

@@ -42,9 +42,10 @@ public class TestKeyProviderCryptoExtension {
         KeyProviderCryptoExtension.createKeyProviderCryptoExtension(kp);
     
     KeyProviderCryptoExtension.EncryptedKeyVersion ek1 = 
-        kpExt.generateEncryptedKey(kv);
+        kpExt.generateEncryptedKey(kv.getName());
     Assert.assertEquals(KeyProviderCryptoExtension.EEK, 
         ek1.getEncryptedKey().getVersionName());
+    Assert.assertEquals("foo", ek1.getKeyName());
     Assert.assertNotNull(ek1.getEncryptedKey().getMaterial());
     Assert.assertEquals(kv.getMaterial().length, 
         ek1.getEncryptedKey().getMaterial().length);
@@ -55,7 +56,7 @@ public class TestKeyProviderCryptoExtension {
     Assert.assertEquals(kv.getMaterial().length, k1.getMaterial().length);
 
     KeyProviderCryptoExtension.EncryptedKeyVersion ek2 = 
-        kpExt.generateEncryptedKey(kv);
+        kpExt.generateEncryptedKey(kv.getName());
     KeyProvider.KeyVersion k2 = kpExt.decryptEncryptedKey(ek2);
     boolean eq = true;
     for (int i = 0; eq && i < ek2.getEncryptedKey().getMaterial().length; i++) {

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -303,6 +303,18 @@ Release 2.6.0 - UNRELEASED
     HDFS-5624. Add HDFS tests for ACLs in combination with viewfs.
     (Stephen Chu via cnauroth)
 
+    HDFS-6655. Add 'header banner' to 'explorer.html' also in Namenode UI
+    (vinayakumarb)
+
+    HDFS-4120. Add a new "-skipSharedEditsCheck" option for BootstrapStandby
+    (Liang Xie and Rakesh R via vinayakumarb)
+
+    HDFS-6597. Add a new option to NN upgrade to terminate the process after
+    upgrade on NN is completed. (Danilo Vunjak via cnauroth)
+
+    HDFS-6700. BlockPlacementPolicy shoud choose storage but not datanode for
+    deletion. (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBootstrapStandbyWithBKJM.java

@@ -0,0 +1,169 @@
+/**
+ * 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.contrib.bkjournal;
+
+import java.io.File;
+import java.io.FileFilter;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.namenode.ha.BootstrapStandby;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints.SlowCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+
+public class TestBootstrapStandbyWithBKJM {
+  private static BKJMUtil bkutil;
+  protected MiniDFSCluster cluster;
+
+  @BeforeClass
+  public static void setupBookkeeper() throws Exception {
+    bkutil = new BKJMUtil(3);
+    bkutil.start();
+  }
+
+  @AfterClass
+  public static void teardownBookkeeper() throws Exception {
+    bkutil.teardown();
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, BKJMUtil
+        .createJournalURI("/bootstrapStandby").toString());
+    BKJMUtil.addJournalManagerDefinition(conf);
+    conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
+    conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY,
+        SlowCodec.class.getCanonicalName());
+    CompressionCodecFactory.setCodecClasses(conf,
+        ImmutableList.<Class> of(SlowCodec.class));
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+        .addNameservice(new MiniDFSNNTopology.NSConf("ns1").addNN(
+            new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001)).addNN(
+            new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
+    cluster = new MiniDFSCluster.Builder(conf).nnTopology(topology)
+        .numDataNodes(1).manageNameDfsSharedDirs(false).build();
+    cluster.waitActive();
+  }
+
+  /**
+   * While boostrapping, in_progress transaction entries should be skipped.
+   * Bootstrap usage for BKJM : "-force", "-nonInteractive", "-skipSharedEditsCheck"
+   */
+  @Test
+  public void testBootstrapStandbyWithActiveNN() throws Exception {
+    // make nn0 active
+    cluster.transitionToActive(0);
+   
+    // do ops and generate in-progress edit log data
+    Configuration confNN1 = cluster.getConfiguration(1);
+    DistributedFileSystem dfs = (DistributedFileSystem) HATestUtil
+        .configureFailoverFs(cluster, confNN1);
+    for (int i = 1; i <= 10; i++) {
+      dfs.mkdirs(new Path("/test" + i));
+    }
+    dfs.close();
+
+    // shutdown nn1 and delete its edit log files
+    cluster.shutdownNameNode(1);
+    deleteEditLogIfExists(confNN1);
+    cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_ENTER, true);
+    cluster.getNameNodeRpc(0).saveNamespace();
+    cluster.getNameNodeRpc(0).setSafeMode(SafeModeAction.SAFEMODE_LEAVE, true);
+
+    // check without -skipSharedEditsCheck, Bootstrap should fail for BKJM
+    // immediately after saveNamespace
+    int rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive" },
+      confNN1);
+    Assert.assertEquals("Mismatches return code", 6, rc);
+
+    // check with -skipSharedEditsCheck
+    rc = BootstrapStandby.run(new String[] { "-force", "-nonInteractive",
+        "-skipSharedEditsCheck" }, confNN1);
+    Assert.assertEquals("Mismatches return code", 0, rc);
+
+    // Checkpoint as fast as we can, in a tight loop.
+    confNN1.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 1);
+    cluster.restartNameNode(1);
+    cluster.transitionToStandby(1);
+   
+    NameNode nn0 = cluster.getNameNode(0);
+    HATestUtil.waitForStandbyToCatchUp(nn0, cluster.getNameNode(1));
+    long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
+        .getFSImage().getMostRecentCheckpointTxId();
+    HATestUtil.waitForCheckpoint(cluster, 1,
+        ImmutableList.of((int) expectedCheckpointTxId));
+
+    // Should have copied over the namespace
+    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
+        ImmutableList.of((int) expectedCheckpointTxId));
+    FSImageTestUtil.assertNNFilesMatch(cluster);
+  }
+
+  private void deleteEditLogIfExists(Configuration confNN1) {
+    String editDirs = confNN1.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
+    String[] listEditDirs = StringUtils.split(editDirs, ',');
+    Assert.assertTrue("Wrong edit directory path!", listEditDirs.length > 0);
+
+    for (String dir : listEditDirs) {
+      File curDir = new File(dir, "current");
+      File[] listFiles = curDir.listFiles(new FileFilter() {
+        @Override
+        public boolean accept(File f) {
+          if (!f.getName().startsWith("edits")) {
+            return true;
+          }
+          return false;
+        }
+      });
+      if (listFiles != null && listFiles.length > 0) {
+        for (File file : listFiles) {
+          Assert.assertTrue("Failed to delete edit files!", file.delete());
+        }
+      }
+    }
+  }
+}

+ 21 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -2660,7 +2660,7 @@ public class BlockManager {
     if (addedNode == delNodeHint) {
       delNodeHint = null;
     }
-    Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
+    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<DatanodeStorageInfo>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) {
@@ -2680,7 +2680,7 @@ public class BlockManager {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
           // exclude corrupt replicas
           if (corruptNodes == null || !corruptNodes.contains(cur)) {
-            nonExcess.add(cur);
+            nonExcess.add(storage);
           }
         }
       }
@@ -2704,7 +2704,7 @@ public class BlockManager {
    * If no such a node is available,
    * then pick a node with least free space
    */
-  private void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
+  private void chooseExcessReplicates(final Collection<DatanodeStorageInfo> nonExcess, 
                               Block b, short replication,
                               DatanodeDescriptor addedNode,
                               DatanodeDescriptor delNodeHint,
@@ -2712,28 +2712,33 @@ public class BlockManager {
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
     BlockCollection bc = getBlockCollection(b);
-    final Map<String, List<DatanodeDescriptor>> rackMap
-        = new HashMap<String, List<DatanodeDescriptor>>();
-    final List<DatanodeDescriptor> moreThanOne = new ArrayList<DatanodeDescriptor>();
-    final List<DatanodeDescriptor> exactlyOne = new ArrayList<DatanodeDescriptor>();
+
+    final Map<String, List<DatanodeStorageInfo>> rackMap
+        = new HashMap<String, List<DatanodeStorageInfo>>();
+    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<DatanodeStorageInfo>();
+    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<DatanodeStorageInfo>();
     
     // split nodes into two sets
     // moreThanOne contains nodes on rack with more than one replica
     // exactlyOne contains the remaining nodes
-    replicator.splitNodesWithRack(nonExcess, rackMap, moreThanOne,
-        exactlyOne);
+    replicator.splitNodesWithRack(nonExcess, rackMap, moreThanOne, exactlyOne);
     
     // pick one node to delete that favors the delete hint
     // otherwise pick one with least space from priSet if it is not empty
     // otherwise one node with least space from remains
     boolean firstOne = true;
+    final DatanodeStorageInfo delNodeHintStorage
+        = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, delNodeHint);
+    final DatanodeStorageInfo addedNodeStorage
+        = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, addedNode);
     while (nonExcess.size() - replication > 0) {
       // check if we can delete delNodeHint
-      final DatanodeInfo cur;
-      if (firstOne && delNodeHint !=null && nonExcess.contains(delNodeHint)
-          && (moreThanOne.contains(delNodeHint)
-              || (addedNode != null && !moreThanOne.contains(addedNode))) ) {
-        cur = delNodeHint;
+      final DatanodeStorageInfo cur;
+      if (firstOne && delNodeHintStorage != null
+          && (moreThanOne.contains(delNodeHintStorage)
+              || (addedNodeStorage != null
+                  && !moreThanOne.contains(addedNodeStorage)))) {
+        cur = delNodeHintStorage;
       } else { // regular excessive replica removal
         cur = replicator.chooseReplicaToDelete(bc, b, replication,
         		moreThanOne, exactlyOne);
@@ -2745,7 +2750,7 @@ public class BlockManager {
           exactlyOne, cur);
 
       nonExcess.remove(cur);
-      addToExcessReplicate(cur, b);
+      addToExcessReplicate(cur.getDatanodeDescriptor(), b);
 
       //
       // The 'excessblocks' tracks blocks until we get confirmation
@@ -2756,7 +2761,7 @@ public class BlockManager {
       // should be deleted.  Items are removed from the invalidate list
       // upon giving instructions to the namenode.
       //
-      addToInvalidates(b, cur);
+      addToInvalidates(b, cur.getDatanodeDescriptor());
       blockLog.info("BLOCK* chooseExcessReplicates: "
                 +"("+cur+", "+b+") is added to invalidated blocks set");
     }

+ 34 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -125,11 +125,12 @@ public abstract class BlockPlacementPolicy {
                    listed in the previous parameter.
    * @return the replica that is the best candidate for deletion
    */
-  abstract public DatanodeDescriptor chooseReplicaToDelete(BlockCollection srcBC,
-                                      Block block, 
-                                      short replicationFactor,
-                                      Collection<DatanodeDescriptor> existingReplicas,
-                                      Collection<DatanodeDescriptor> moreExistingReplicas);
+  abstract public DatanodeStorageInfo chooseReplicaToDelete(
+      BlockCollection srcBC,
+      Block block, 
+      short replicationFactor,
+      Collection<DatanodeStorageInfo> existingReplicas,
+      Collection<DatanodeStorageInfo> moreExistingReplicas);
 
   /**
    * Used to setup a BlockPlacementPolicy object. This should be defined by 
@@ -176,21 +177,23 @@ public abstract class BlockPlacementPolicy {
    * @param exactlyOne The List of replica nodes on rack with only one replica
    * @param cur current replica to remove
    */
-  public void adjustSetsWithChosenReplica(final Map<String, 
-      List<DatanodeDescriptor>> rackMap,
-      final List<DatanodeDescriptor> moreThanOne,
-      final List<DatanodeDescriptor> exactlyOne, final DatanodeInfo cur) {
+  public void adjustSetsWithChosenReplica(
+      final Map<String, List<DatanodeStorageInfo>> rackMap,
+      final List<DatanodeStorageInfo> moreThanOne,
+      final List<DatanodeStorageInfo> exactlyOne,
+      final DatanodeStorageInfo cur) {
     
-    String rack = getRack(cur);
-    final List<DatanodeDescriptor> datanodes = rackMap.get(rack);
-    datanodes.remove(cur);
-    if (datanodes.isEmpty()) {
+    final String rack = getRack(cur.getDatanodeDescriptor());
+    final List<DatanodeStorageInfo> storages = rackMap.get(rack);
+    storages.remove(cur);
+    if (storages.isEmpty()) {
       rackMap.remove(rack);
     }
     if (moreThanOne.remove(cur)) {
-      if (datanodes.size() == 1) {
-        moreThanOne.remove(datanodes.get(0));
-        exactlyOne.add(datanodes.get(0));
+      if (storages.size() == 1) {
+        final DatanodeStorageInfo remaining = storages.get(0);
+        moreThanOne.remove(remaining);
+        exactlyOne.add(remaining);
       }
     } else {
       exactlyOne.remove(cur);
@@ -215,28 +218,28 @@ public abstract class BlockPlacementPolicy {
    * @param exactlyOne remains contains the remaining nodes
    */
   public void splitNodesWithRack(
-      Collection<DatanodeDescriptor> dataNodes,
-      final Map<String, List<DatanodeDescriptor>> rackMap,
-      final List<DatanodeDescriptor> moreThanOne,
-      final List<DatanodeDescriptor> exactlyOne) {
-    for(DatanodeDescriptor node : dataNodes) {
-      final String rackName = getRack(node);
-      List<DatanodeDescriptor> datanodeList = rackMap.get(rackName);
-      if (datanodeList == null) {
-        datanodeList = new ArrayList<DatanodeDescriptor>();
-        rackMap.put(rackName, datanodeList);
+      final Iterable<DatanodeStorageInfo> storages,
+      final Map<String, List<DatanodeStorageInfo>> rackMap,
+      final List<DatanodeStorageInfo> moreThanOne,
+      final List<DatanodeStorageInfo> exactlyOne) {
+    for(DatanodeStorageInfo s: storages) {
+      final String rackName = getRack(s.getDatanodeDescriptor());
+      List<DatanodeStorageInfo> storageList = rackMap.get(rackName);
+      if (storageList == null) {
+        storageList = new ArrayList<DatanodeStorageInfo>();
+        rackMap.put(rackName, storageList);
       }
-      datanodeList.add(node);
+      storageList.add(s);
     }
     
     // split nodes into two sets
-    for(List<DatanodeDescriptor> datanodeList : rackMap.values()) {
-      if (datanodeList.size() == 1) {
+    for(List<DatanodeStorageInfo> storageList : rackMap.values()) {
+      if (storageList.size() == 1) {
         // exactlyOne contains nodes on rack with only one replica
-        exactlyOne.add(datanodeList.get(0));
+        exactlyOne.add(storageList.get(0));
       } else {
         // moreThanOne contains nodes on rack with more than one replica
-        moreThanOne.addAll(datanodeList);
+        moreThanOne.addAll(storageList);
       }
     }
   }

+ 15 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -756,31 +756,34 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   @Override
-  public DatanodeDescriptor chooseReplicaToDelete(BlockCollection bc,
+  public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection bc,
       Block block, short replicationFactor,
-      Collection<DatanodeDescriptor> first,
-      Collection<DatanodeDescriptor> second) {
+      Collection<DatanodeStorageInfo> first,
+      Collection<DatanodeStorageInfo> second) {
     long oldestHeartbeat =
       now() - heartbeatInterval * tolerateHeartbeatMultiplier;
-    DatanodeDescriptor oldestHeartbeatNode = null;
+    DatanodeStorageInfo oldestHeartbeatStorage = null;
     long minSpace = Long.MAX_VALUE;
-    DatanodeDescriptor minSpaceNode = null;
+    DatanodeStorageInfo minSpaceStorage = null;
 
     // Pick the node with the oldest heartbeat or with the least free space,
     // if all hearbeats are within the tolerable heartbeat interval
-    for(DatanodeDescriptor node : pickupReplicaSet(first, second)) {
+    for(DatanodeStorageInfo storage : pickupReplicaSet(first, second)) {
+      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       long free = node.getRemaining();
       long lastHeartbeat = node.getLastUpdate();
       if(lastHeartbeat < oldestHeartbeat) {
         oldestHeartbeat = lastHeartbeat;
-        oldestHeartbeatNode = node;
+        oldestHeartbeatStorage = storage;
       }
       if (minSpace > free) {
         minSpace = free;
-        minSpaceNode = node;
+        minSpaceStorage = storage;
       }
     }
-    return oldestHeartbeatNode != null ? oldestHeartbeatNode : minSpaceNode;
+
+    return oldestHeartbeatStorage != null? oldestHeartbeatStorage
+        : minSpaceStorage;
   }
 
   /**
@@ -789,9 +792,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * replica while second set contains remaining replica nodes.
    * So pick up first set if not empty. If first is empty, then pick second.
    */
-  protected Collection<DatanodeDescriptor> pickupReplicaSet(
-      Collection<DatanodeDescriptor> first,
-      Collection<DatanodeDescriptor> second) {
+  protected Collection<DatanodeStorageInfo> pickupReplicaSet(
+      Collection<DatanodeStorageInfo> first,
+      Collection<DatanodeStorageInfo> second) {
     return first.isEmpty() ? second : first;
   }
   

+ 16 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyWithNodeGroup.java

@@ -286,9 +286,9 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
    * If first is empty, then pick second.
    */
   @Override
-  public Collection<DatanodeDescriptor> pickupReplicaSet(
-      Collection<DatanodeDescriptor> first,
-      Collection<DatanodeDescriptor> second) {
+  public Collection<DatanodeStorageInfo> pickupReplicaSet(
+      Collection<DatanodeStorageInfo> first,
+      Collection<DatanodeStorageInfo> second) {
     // If no replica within same rack, return directly.
     if (first.isEmpty()) {
       return second;
@@ -296,25 +296,24 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
     // Split data nodes in the first set into two sets, 
     // moreThanOne contains nodes on nodegroup with more than one replica
     // exactlyOne contains the remaining nodes
-    Map<String, List<DatanodeDescriptor>> nodeGroupMap = 
-        new HashMap<String, List<DatanodeDescriptor>>();
+    Map<String, List<DatanodeStorageInfo>> nodeGroupMap = 
+        new HashMap<String, List<DatanodeStorageInfo>>();
     
-    for(DatanodeDescriptor node : first) {
-      final String nodeGroupName = 
-          NetworkTopology.getLastHalf(node.getNetworkLocation());
-      List<DatanodeDescriptor> datanodeList = 
-          nodeGroupMap.get(nodeGroupName);
-      if (datanodeList == null) {
-        datanodeList = new ArrayList<DatanodeDescriptor>();
-        nodeGroupMap.put(nodeGroupName, datanodeList);
+    for(DatanodeStorageInfo storage : first) {
+      final String nodeGroupName = NetworkTopology.getLastHalf(
+          storage.getDatanodeDescriptor().getNetworkLocation());
+      List<DatanodeStorageInfo> storageList = nodeGroupMap.get(nodeGroupName);
+      if (storageList == null) {
+        storageList = new ArrayList<DatanodeStorageInfo>();
+        nodeGroupMap.put(nodeGroupName, storageList);
       }
-      datanodeList.add(node);
+      storageList.add(storage);
     }
     
-    final List<DatanodeDescriptor> moreThanOne = new ArrayList<DatanodeDescriptor>();
-    final List<DatanodeDescriptor> exactlyOne = new ArrayList<DatanodeDescriptor>();
+    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<DatanodeStorageInfo>();
+    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<DatanodeStorageInfo>();
     // split nodes into two sets
-    for(List<DatanodeDescriptor> datanodeList : nodeGroupMap.values()) {
+    for(List<DatanodeStorageInfo> datanodeList : nodeGroupMap.values()) {
       if (datanodeList.size() == 1 ) {
         // exactlyOne contains nodes on nodegroup with exactly one replica
         exactlyOne.add(datanodeList.get(0));

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java

@@ -22,6 +22,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -290,4 +291,21 @@ public class DatanodeStorageInfo {
   public String toString() {
     return "[" + storageType + "]" + storageID + ":" + state;
   }
+
+  /** @return the first {@link DatanodeStorageInfo} corresponding to
+   *          the given datanode
+   */
+  static DatanodeStorageInfo getDatanodeStorageInfo(
+      final Iterable<DatanodeStorageInfo> infos,
+      final DatanodeDescriptor datanode) {
+    if (datanode == null) {
+      return null;
+    }
+    for(DatanodeStorageInfo storage : infos) {
+      if (storage.getDatanodeDescriptor() == datanode) {
+        return storage;
+      }
+    }
+    return null;
+  }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -93,7 +93,8 @@ public final class HdfsServerConstants {
     FORCE("-force"),
     NONINTERACTIVE("-nonInteractive"),
     RENAMERESERVED("-renameReserved"),
-    METADATAVERSION("-metadataVersion");
+    METADATAVERSION("-metadataVersion"),
+    UPGRADEONLY("-upgradeOnly");
 
     private static final Pattern ENUM_WITH_ROLLING_UPGRADE_OPTION = Pattern.compile(
         "(\\w+)\\((\\w+)\\)");

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -225,6 +225,7 @@ public class FSImage implements Closeable {
       NNStorage.checkVersionUpgradable(storage.getLayoutVersion());
     }
     if (startOpt != StartupOption.UPGRADE
+        && startOpt != StartupOption.UPGRADEONLY
         && !RollingUpgradeStartupOption.STARTED.matches(startOpt)
         && layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
         && layoutVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
@@ -263,6 +264,7 @@ public class FSImage implements Closeable {
     // 3. Do transitions
     switch(startOpt) {
     case UPGRADE:
+    case UPGRADEONLY:
       doUpgrade(target);
       return false; // upgrade saved image already
     case IMPORT:
@@ -748,11 +750,13 @@ public class FSImage implements Closeable {
       editLog.recoverUnclosedStreams();
     } else if (HAUtil.isHAEnabled(conf, nameserviceId)
         && (startOpt == StartupOption.UPGRADE
+            || startOpt == StartupOption.UPGRADEONLY
             || RollingUpgradeStartupOption.ROLLBACK.matches(startOpt))) {
       // This NN is HA, but we're doing an upgrade or a rollback of rolling
       // upgrade so init the edit log for write.
       editLog.initJournalsForWrite();
-      if (startOpt == StartupOption.UPGRADE) {
+      if (startOpt == StartupOption.UPGRADE
+          || startOpt == StartupOption.UPGRADEONLY) {
         long sharedLogCTime = editLog.getSharedLogCTime();
         if (this.storage.getCTime() < sharedLogCTime) {
           throw new IOException("It looks like the shared log is already " +

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -978,7 +978,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       // This will start a new log segment and write to the seen_txid file, so
       // we shouldn't do it when coming up in standby state
-      if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)) {
+      if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)
+          || (haEnabled && startOpt == StartupOption.UPGRADEONLY)) {
         fsImage.openEditLogForWrite();
       }
       success = true;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -836,7 +836,7 @@ public class NNStorage extends Storage implements Closeable,
    */
   void processStartupOptionsForUpgrade(StartupOption startOpt, int layoutVersion)
       throws IOException {
-    if (startOpt == StartupOption.UPGRADE) {
+    if (startOpt == StartupOption.UPGRADE || startOpt == StartupOption.UPGRADEONLY) {
       // If upgrade from a release that does not support federation,
       // if clusterId is provided in the startupOptions use it.
       // Else generate a new cluster ID      

+ 16 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -210,6 +210,9 @@ public class NameNode implements NameNodeStatusMXBean {
       + StartupOption.UPGRADE.getName() + 
         " [" + StartupOption.CLUSTERID.getName() + " cid]" +
         " [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | \n\t["
+      + StartupOption.UPGRADEONLY.getName() + 
+        " [" + StartupOption.CLUSTERID.getName() + " cid]" +
+        " [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | \n\t["
       + StartupOption.ROLLBACK.getName() + "] | \n\t["
       + StartupOption.ROLLINGUPGRADE.getName() + " <"
       + RollingUpgradeStartupOption.DOWNGRADE.name().toLowerCase() + "|"
@@ -713,6 +716,7 @@ public class NameNode implements NameNodeStatusMXBean {
    * <li>{@link StartupOption#BACKUP BACKUP} - start backup node</li>
    * <li>{@link StartupOption#CHECKPOINT CHECKPOINT} - start checkpoint node</li>
    * <li>{@link StartupOption#UPGRADE UPGRADE} - start the cluster  
+   * <li>{@link StartupOption#UPGRADEONLY UPGRADEONLY} - upgrade the cluster  
    * upgrade and create a snapshot of the current file system state</li> 
    * <li>{@link StartupOption#RECOVER RECOVERY} - recover name node
    * metadata</li>
@@ -767,7 +771,8 @@ public class NameNode implements NameNodeStatusMXBean {
   }
 
   protected HAState createHAState(StartupOption startOpt) {
-    if (!haEnabled || startOpt == StartupOption.UPGRADE) {
+    if (!haEnabled || startOpt == StartupOption.UPGRADE 
+        || startOpt == StartupOption.UPGRADEONLY) {
       return ACTIVE_STATE;
     } else {
       return STANDBY_STATE;
@@ -1198,8 +1203,10 @@ public class NameNode implements NameNodeStatusMXBean {
         startOpt = StartupOption.BACKUP;
       } else if (StartupOption.CHECKPOINT.getName().equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.CHECKPOINT;
-      } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) {
-        startOpt = StartupOption.UPGRADE;
+      } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)
+          || StartupOption.UPGRADEONLY.getName().equalsIgnoreCase(cmd)) {
+        startOpt = StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd) ? 
+            StartupOption.UPGRADE : StartupOption.UPGRADEONLY;
         /* Can be followed by CLUSTERID with a required parameter or
          * RENAMERESERVED with an optional parameter
          */
@@ -1407,6 +1414,12 @@ public class NameNode implements NameNodeStatusMXBean {
         terminate(0);
         return null; // avoid javac warning
       }
+      case UPGRADEONLY: {
+        DefaultMetricsSystem.initialize("NameNode");
+        new NameNode(conf);
+        terminate(0);
+        return null;
+      }
       default: {
         DefaultMetricsSystem.initialize("NameNode");
         return new NameNode(conf);

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -81,6 +81,7 @@ public class BootstrapStandby implements Tool, Configurable {
   
   private boolean force = false;
   private boolean interactive = true;
+  private boolean skipSharedEditsCheck = false;
 
   // Exit/return codes.
   static final int ERR_CODE_FAILED_CONNECT = 2;
@@ -117,6 +118,8 @@ public class BootstrapStandby implements Tool, Configurable {
         force = true;
       } else if ("-nonInteractive".equals(arg)) {
         interactive = false;
+      } else if ("-skipSharedEditsCheck".equals(arg)) {
+        skipSharedEditsCheck = true;
       } else {
         printUsage();
         throw new HadoopIllegalArgumentException(
@@ -127,7 +130,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
   private void printUsage() {
     System.err.println("Usage: " + this.getClass().getSimpleName() +
-        "[-force] [-nonInteractive]");
+        " [-force] [-nonInteractive] [-skipSharedEditsCheck]");
   }
   
   private NamenodeProtocol createNNProtocolProxy()
@@ -200,7 +203,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
       // Ensure that we have enough edits already in the shared directory to
       // start up from the last checkpoint on the active.
-      if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) {
+      if (!skipSharedEditsCheck && !checkLogsAvailableForRead(image, imageTxId, curTxId)) {
         return ERR_CODE_LOGS_UNAVAILABLE;
       }
 

+ 29 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -24,6 +24,29 @@
     <title>Browsing HDFS</title>
   </head>
   <body>
+
+    <header class="navbar navbar-inverse bs-docs-nav" role="banner">
+    <div class="container">
+      <div class="navbar-header">
+        <div class="navbar-brand">Hadoop</div>
+      </div>
+
+      <ul class="nav navbar-nav" id="ui-tabs">
+        <li><a href="dfshealth.html#tab-overview">Overview</a></li>
+        <li><a href="dfshealth.html#tab-datanode">Datanodes</a></li>
+        <li><a href="dfshealth.html#tab-snapshot">Snapshot</a></li>
+        <li><a href="dfshealth.html#tab-startup-progress">Startup Progress</a></li>
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" data-toggle="dropdown">Utilities <b class="caret"></b></a>
+          <ul class="dropdown-menu">
+            <li><a href="#">Browse the file system</a></li>
+            <li><a href="logs">Logs</a></li>
+          </ul>
+        </li>
+      </ul>
+    </div>
+    </header>
+
     <div class="modal" id="file-info" tabindex="-1" role="dialog" aria-hidden="true">
       <div class="modal-dialog">
 	<div class="modal-content">
@@ -73,6 +96,12 @@
       </div>
       <br />
       <div id="panel"></div>
+
+      <div class="row">
+        <hr />
+        <div class="col-xs-2"><p>Hadoop, 2014.</p></div>
+      </div>
+
     </div>
 
     <script type="text/x-dust-template" id="tmpl-explorer">
@@ -126,7 +155,5 @@
     </script><script type="text/javascript" src="/static/dfs-dust.js">
     </script><script type="text/javascript" src="explorer.js">
     </script>
-    <hr />
-    <p>Hadoop, 2014.</p>
   </body>
 </html>

+ 19 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -905,49 +905,46 @@ public class TestReplicationPolicy {
    */
   @Test
   public void testChooseReplicaToDelete() throws Exception {
-    List<DatanodeDescriptor> replicaNodeList = new 
-        ArrayList<DatanodeDescriptor>();
-    final Map<String, List<DatanodeDescriptor>> rackMap
-        = new HashMap<String, List<DatanodeDescriptor>>();
+    List<DatanodeStorageInfo> replicaList = new ArrayList<DatanodeStorageInfo>();
+    final Map<String, List<DatanodeStorageInfo>> rackMap
+        = new HashMap<String, List<DatanodeStorageInfo>>();
     
     dataNodes[0].setRemaining(4*1024*1024);
-    replicaNodeList.add(dataNodes[0]);
+    replicaList.add(storages[0]);
     
     dataNodes[1].setRemaining(3*1024*1024);
-    replicaNodeList.add(dataNodes[1]);
+    replicaList.add(storages[1]);
     
     dataNodes[2].setRemaining(2*1024*1024);
-    replicaNodeList.add(dataNodes[2]);
+    replicaList.add(storages[2]);
     
     dataNodes[5].setRemaining(1*1024*1024);
-    replicaNodeList.add(dataNodes[5]);
+    replicaList.add(storages[5]);
     
     // Refresh the last update time for all the datanodes
     for (int i = 0; i < dataNodes.length; i++) {
       dataNodes[i].setLastUpdate(Time.now());
     }
     
-    List<DatanodeDescriptor> first = new ArrayList<DatanodeDescriptor>();
-    List<DatanodeDescriptor> second = new ArrayList<DatanodeDescriptor>();
-    replicator.splitNodesWithRack(
-        replicaNodeList, rackMap, first, second);
-    // dataNodes[0] and dataNodes[1] are in first set as their rack has two 
-    // replica nodes, while datanodes[2] and dataNodes[5] are in second set.
+    List<DatanodeStorageInfo> first = new ArrayList<DatanodeStorageInfo>();
+    List<DatanodeStorageInfo> second = new ArrayList<DatanodeStorageInfo>();
+    replicator.splitNodesWithRack(replicaList, rackMap, first, second);
+    // storages[0] and storages[1] are in first set as their rack has two 
+    // replica nodes, while storages[2] and dataNodes[5] are in second set.
     assertEquals(2, first.size());
     assertEquals(2, second.size());
-    DatanodeDescriptor chosenNode = replicator.chooseReplicaToDelete(
+    DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete(
         null, null, (short)3, first, second);
-    // Within first set, dataNodes[1] with less free space
-    assertEquals(chosenNode, dataNodes[1]);
+    // Within first set, storages[1] with less free space
+    assertEquals(chosen, storages[1]);
 
-    replicator.adjustSetsWithChosenReplica(
-        rackMap, first, second, chosenNode);
+    replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosen);
     assertEquals(0, first.size());
     assertEquals(3, second.size());
-    // Within second set, dataNodes[5] with less free space
-    chosenNode = replicator.chooseReplicaToDelete(
+    // Within second set, storages[5] with less free space
+    chosen = replicator.chooseReplicaToDelete(
         null, null, (short)2, first, second);
-    assertEquals(chosenNode, dataNodes[5]);
+    assertEquals(chosen, storages[5]);
   }
   
   /**

+ 18 - 19
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java

@@ -591,51 +591,50 @@ public class TestReplicationPolicyWithNodeGroup {
    */
   @Test
   public void testChooseReplicaToDelete() throws Exception {
-    List<DatanodeDescriptor> replicaNodeList = 
-        new ArrayList<DatanodeDescriptor>();
-    final Map<String, List<DatanodeDescriptor>> rackMap = 
-        new HashMap<String, List<DatanodeDescriptor>>();
+    List<DatanodeStorageInfo> replicaList = new ArrayList<DatanodeStorageInfo>();
+    final Map<String, List<DatanodeStorageInfo>> rackMap
+        = new HashMap<String, List<DatanodeStorageInfo>>();
     dataNodes[0].setRemaining(4*1024*1024);
-    replicaNodeList.add(dataNodes[0]);
+    replicaList.add(storages[0]);
 
     dataNodes[1].setRemaining(3*1024*1024);
-    replicaNodeList.add(dataNodes[1]);
+    replicaList.add(storages[1]);
 
     dataNodes[2].setRemaining(2*1024*1024);
-    replicaNodeList.add(dataNodes[2]);
+    replicaList.add(storages[2]);
 
     dataNodes[5].setRemaining(1*1024*1024);
-    replicaNodeList.add(dataNodes[5]);
+    replicaList.add(storages[5]);
 
-    List<DatanodeDescriptor> first = new ArrayList<DatanodeDescriptor>();
-    List<DatanodeDescriptor> second = new ArrayList<DatanodeDescriptor>();
+    List<DatanodeStorageInfo> first = new ArrayList<DatanodeStorageInfo>();
+    List<DatanodeStorageInfo> second = new ArrayList<DatanodeStorageInfo>();
     replicator.splitNodesWithRack(
-        replicaNodeList, rackMap, first, second);
+        replicaList, rackMap, first, second);
     assertEquals(3, first.size());
     assertEquals(1, second.size());
-    DatanodeDescriptor chosenNode = replicator.chooseReplicaToDelete(
+    DatanodeStorageInfo chosen = replicator.chooseReplicaToDelete(
         null, null, (short)3, first, second);
     // Within first set {dataNodes[0], dataNodes[1], dataNodes[2]}, 
     // dataNodes[0] and dataNodes[1] are in the same nodegroup, 
     // but dataNodes[1] is chosen as less free space
-    assertEquals(chosenNode, dataNodes[1]);
+    assertEquals(chosen, storages[1]);
 
-    replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosenNode);
+    replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosen);
     assertEquals(2, first.size());
     assertEquals(1, second.size());
     // Within first set {dataNodes[0], dataNodes[2]}, dataNodes[2] is chosen
     // as less free space
-    chosenNode = replicator.chooseReplicaToDelete(
+    chosen = replicator.chooseReplicaToDelete(
         null, null, (short)2, first, second);
-    assertEquals(chosenNode, dataNodes[2]);
+    assertEquals(chosen, storages[2]);
 
-    replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosenNode);
+    replicator.adjustSetsWithChosenReplica(rackMap, first, second, chosen);
     assertEquals(0, first.size());
     assertEquals(2, second.size());
     // Within second set, dataNodes[5] with less free space
-    chosenNode = replicator.chooseReplicaToDelete(
+    chosen = replicator.chooseReplicaToDelete(
         null, null, (short)1, first, second);
-    assertEquals(chosenNode, dataNodes[5]);
+    assertEquals(chosen, storages[5]);
   }
   
   /**

+ 18 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java

@@ -21,6 +21,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.net.URI;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 
 import org.apache.hadoop.conf.Configuration;
@@ -30,11 +32,15 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  * This class tests various upgrade cases from earlier versions to current
  * version with and without clusterid.
  */
+@RunWith(value = Parameterized.class)
 public class TestStartupOptionUpgrade {
 
   private Configuration conf;
@@ -42,10 +48,21 @@ public class TestStartupOptionUpgrade {
   private int layoutVersion;
   NNStorage storage;
 
+  @Parameters
+  public static Collection<Object[]> startOption() {
+    Object[][] params = new Object[][] { { StartupOption.UPGRADE },
+        { StartupOption.UPGRADEONLY } };
+    return Arrays.asList(params);
+  }
+
+  public TestStartupOptionUpgrade(StartupOption startOption) {
+    super();
+    this.startOpt = startOption;
+  }
+  
   @Before
   public void setUp() throws Exception {
     conf = new HdfsConfiguration();
-    startOpt = StartupOption.UPGRADE;
     startOpt.setClusterId(null);
     storage = new NNStorage(conf,
       Collections.<URI>emptyList(),

+ 6 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -585,15 +585,14 @@ public class TestDNFencing {
     }
 
     @Override
-    public DatanodeDescriptor chooseReplicaToDelete(BlockCollection inode,
+    public DatanodeStorageInfo chooseReplicaToDelete(BlockCollection inode,
         Block block, short replicationFactor,
-        Collection<DatanodeDescriptor> first,
-        Collection<DatanodeDescriptor> second) {
+        Collection<DatanodeStorageInfo> first,
+        Collection<DatanodeStorageInfo> second) {
       
-      Collection<DatanodeDescriptor> chooseFrom =
-        !first.isEmpty() ? first : second;
+      Collection<DatanodeStorageInfo> chooseFrom = !first.isEmpty() ? first : second;
 
-      List<DatanodeDescriptor> l = Lists.newArrayList(chooseFrom);
+      List<DatanodeStorageInfo> l = Lists.newArrayList(chooseFrom);
       return l.get(DFSUtil.getRandom().nextInt(l.size()));
     }
   }