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 zj...@apache.org on 2015/01/27 19:40:39 UTC

[21/50] [abbrv] hadoop git commit: HDFS-7575. Upgrade should generate a unique storage ID for each volume. (Contributed by Arpit Agarwal)

HDFS-7575. Upgrade should generate a unique storage ID for each volume. (Contributed by Arpit Agarwal)


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

Branch: refs/heads/YARN-2928
Commit: d34074e237ee10b83aeb02294f595714d43e39e4
Parents: 825923f
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jan 22 14:08:20 2015 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jan 22 14:08:20 2015 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/DataStorage.java       |  35 +++--
 .../hdfs/server/protocol/DatanodeStorage.java   |  19 ++-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |  19 ++-
 .../hadoop/hdfs/TestDatanodeLayoutUpgrade.java  |   2 +-
 ...estDatanodeStartupFixesLegacyStorageIDs.java | 139 +++++++++++++++++++
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |   2 +-
 .../server/datanode/SimulatedFSDataset.java     |   2 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |   2 +-
 .../testUpgradeFrom22FixesStorageIDs.tgz        | Bin 0 -> 3260 bytes
 .../testUpgradeFrom22FixesStorageIDs.txt        |  25 ++++
 .../testUpgradeFrom22via26FixesStorageIDs.tgz   | Bin 0 -> 3635 bytes
 .../testUpgradeFrom22via26FixesStorageIDs.txt   |  25 ++++
 .../testUpgradeFrom26PreservesStorageIDs.tgz    | Bin 0 -> 3852 bytes
 .../testUpgradeFrom26PreservesStorageIDs.txt    |  25 ++++
 14 files changed, 274 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 3ea8ce3..0602dfb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -142,11 +143,20 @@ public class DataStorage extends Storage {
     this.datanodeUuid = newDatanodeUuid;
   }
 
-  /** Create an ID for this storage. */
-  public synchronized void createStorageID(StorageDirectory sd) {
-    if (sd.getStorageUuid() == null) {
+  /** Create an ID for this storage.
+   * @return true if a new storage ID was generated.
+   * */
+  public synchronized boolean createStorageID(
+      StorageDirectory sd, boolean regenerateStorageIds) {
+    final String oldStorageID = sd.getStorageUuid();
+    if (oldStorageID == null || regenerateStorageIds) {
       sd.setStorageUuid(DatanodeStorage.generateUuid());
+      LOG.info("Generated new storageID " + sd.getStorageUuid() +
+          " for directory " + sd.getRoot() +
+          (oldStorageID == null ? "" : (" to replace " + oldStorageID)));
+      return true;
     }
+    return false;
   }
 
   /**
@@ -677,20 +687,25 @@ public class DataStorage extends Storage {
           + sd.getRoot().getCanonicalPath() + ": namenode clusterID = "
           + nsInfo.getClusterID() + "; datanode clusterID = " + getClusterID());
     }
-    
-    // After addition of the federation feature, ctime check is only 
-    // meaningful at BlockPoolSliceStorage level. 
 
-    // regular start up. 
+    // Clusters previously upgraded from layout versions earlier than
+    // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
+    // new storage ID. We check for that and fix it now.
+    boolean haveValidStorageId =
+        DataNodeLayoutVersion.supports(
+            LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, layoutVersion) &&
+            DatanodeStorage.isValidStorageId(sd.getStorageUuid());
+
+    // regular start up.
     if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION) {
-      createStorageID(sd);
+      createStorageID(sd, !haveValidStorageId);
       return; // regular startup
     }
-    
+
     // do upgrade
     if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
       doUpgrade(datanode, sd, nsInfo);  // upgrade
-      createStorageID(sd);
+      createStorageID(sd, !haveValidStorageId);
       return;
     }
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
index 4fe07b9..4d224d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
@@ -47,6 +47,7 @@ public class DatanodeStorage {
   private final String storageID;
   private final State state;
   private final StorageType storageType;
+  private static final String STORAGE_ID_PREFIX = "DS-";
 
   /**
    * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
@@ -80,7 +81,23 @@ public class DatanodeStorage {
    * @return unique storage ID
    */
   public static String generateUuid() {
-    return "DS-" + UUID.randomUUID();
+    return STORAGE_ID_PREFIX + UUID.randomUUID();
+  }
+
+  /**
+   * Verify that a given string is a storage ID in the "DS-..uuid.." format.
+   */
+  public static boolean isValidStorageId(final String storageID) {
+    try {
+      // Attempt to parse the UUID.
+      if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
+        UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
+        return true;
+      }
+    } catch (IllegalArgumentException iae) {
+    }
+
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/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 88ad0cc..ad907f6 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
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
@@ -91,6 +90,10 @@ public class TestDFSUpgradeFromImage {
     }
   }
   
+  public interface ClusterVerifier {
+    public void verifyClusterPostUpgrade(final MiniDFSCluster cluster) throws IOException;
+  }
+
   final LinkedList<ReferenceFileInfo> refList = new LinkedList<ReferenceFileInfo>();
   Iterator<ReferenceFileInfo> refIter;
   
@@ -119,7 +122,7 @@ public class TestDFSUpgradeFromImage {
       if (line.length() <= 0 || line.startsWith("#")) {
         continue;
       }
-      String[] arr = line.split("\\s+\t\\s+");
+      String[] arr = line.split("\\s+");
       if (arr.length < 1) {
         continue;
       }
@@ -288,7 +291,7 @@ public class TestDFSUpgradeFromImage {
   public void testUpgradeFromRel22Image() throws IOException {
     unpackStorage(HADOOP22_IMAGE, HADOOP_DFS_DIR_TXT);
     upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
-        numDataNodes(4));
+        numDataNodes(4), null);
   }
   
   /**
@@ -316,7 +319,7 @@ public class TestDFSUpgradeFromImage {
     // Upgrade should now fail
     try {
       upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
-          numDataNodes(4));
+          numDataNodes(4), null);
       fail("Upgrade did not fail with bad MD5");
     } catch (IOException ioe) {
       String msg = StringUtils.stringifyException(ioe);
@@ -573,7 +576,7 @@ public class TestDFSUpgradeFromImage {
     } while (dirList.hasMore());
   }
   
-  void upgradeAndVerify(MiniDFSCluster.Builder bld)
+  void upgradeAndVerify(MiniDFSCluster.Builder bld, ClusterVerifier verifier)
       throws IOException {
     MiniDFSCluster cluster = null;
     try {
@@ -592,6 +595,10 @@ public class TestDFSUpgradeFromImage {
       }
       recoverAllLeases(dfsClient, new Path("/"));
       verifyFileSystem(dfs);
+
+      if (verifier != null) {
+        verifier.verifyClusterPostUpgrade(cluster);
+      }
     } finally {
       if (cluster != null) { cluster.shutdown(); }
     } 
@@ -611,6 +618,6 @@ public class TestDFSUpgradeFromImage {
         "data1");
     upgradeAndVerify(new MiniDFSCluster.Builder(conf).
           numDataNodes(1).enableManagedDfsDirsRedundancy(false).
-          manageDataDfsDirs(false));
+          manageDataDfsDirs(false), null);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
index 0966301..343320c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
@@ -43,6 +43,6 @@ public class TestDatanodeLayoutUpgrade {
         System.getProperty("test.build.data") + File.separator +
             "dfs" + File.separator + "name");
     upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
-    .manageDataDfsDirs(false).manageNameDfsDirs(false));
+    .manageDataDfsDirs(false).manageNameDfsDirs(false), null);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
new file mode 100644
index 0000000..e262abc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.TestDFSUpgradeFromImage.ClusterVerifier;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+
+/**
+ * The test verifies that legacy storage IDs in older DataNode
+ * images are replaced with UUID-based storage IDs. The startup may
+ * or may not involve a Datanode Layout upgrade. Each test case uses
+ * the following resource files.
+ *
+ *    1. testCaseName.tgz - NN and DN directories corresponding
+ *                          to a specific layout version.
+ *    2. testCaseName.txt - Text file listing the checksum of each file
+ *                          in the cluster and overall checksum. See
+ *                          TestUpgradeFromImage for the file format.
+ *
+ * If any test case is renamed then the corresponding resource files must
+ * also be renamed.
+ */
+public class TestDatanodeStartupFixesLegacyStorageIDs {
+
+  /**
+   * Perform a upgrade using the test image corresponding to
+   * testCaseName.
+   *
+   * @param testCaseName
+   * @param expectedStorageId if null, then the upgrade generates a new
+   *                          unique storage ID.
+   * @throws IOException
+   */
+  private static void runLayoutUpgradeTest(final String testCaseName,
+                                           final String expectedStorageId)
+      throws IOException {
+    TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
+    upgrade.unpackStorage(testCaseName + ".tgz", testCaseName + ".txt");
+    Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
+    initStorageDirs(conf, testCaseName);
+    upgradeAndVerify(upgrade, conf, new ClusterVerifier() {
+      @Override
+      public void verifyClusterPostUpgrade(MiniDFSCluster cluster) throws IOException {
+        // Verify that a GUID-based storage ID was generated.
+        final String bpid = cluster.getNamesystem().getBlockPoolId();
+        StorageReport[] reports =
+            cluster.getDataNodes().get(0).getFSDataset().getStorageReports(bpid);
+        assertThat(reports.length, is(1));
+        final String storageID = reports[0].getStorage().getStorageID();
+        assertTrue(DatanodeStorage.isValidStorageId(storageID));
+
+        if (expectedStorageId != null) {
+          assertThat(storageID, is(expectedStorageId));
+        }
+      }
+    });
+  }
+
+  private static void initStorageDirs(final Configuration conf,
+                                      final String testName) {
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+             System.getProperty("test.build.data") + File.separator +
+                 testName + File.separator + "dfs" + File.separator + "data");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+             System.getProperty("test.build.data") + File.separator +
+                 testName + File.separator + "dfs" + File.separator + "name");
+
+  }
+
+  private static void upgradeAndVerify(final TestDFSUpgradeFromImage upgrade,
+                                       final Configuration conf,
+                                       final ClusterVerifier verifier)
+      throws IOException{
+    upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf)
+                                 .numDataNodes(1)
+                                 .manageDataDfsDirs(false)
+                                 .manageNameDfsDirs(false), verifier);
+  }
+
+  /**
+   * Upgrade from 2.2 (no storage IDs per volume) correctly generates
+   * GUID-based storage IDs. Test case for HDFS-7575.
+   */
+  @Test (timeout=300000)
+  public void testUpgradeFrom22FixesStorageIDs() throws IOException {
+    runLayoutUpgradeTest(GenericTestUtils.getMethodName(), null);
+  }
+
+  /**
+   * Startup from a 2.6-layout that has legacy storage IDs correctly
+   * generates new storage IDs.
+   * Test case for HDFS-7575.
+   */
+  @Test (timeout=300000)
+  public void testUpgradeFrom22via26FixesStorageIDs() throws IOException {
+    runLayoutUpgradeTest(GenericTestUtils.getMethodName(), null);
+  }
+
+  /**
+   * Startup from a 2.6-layout that already has unique storage IDs does
+   * not regenerate the storage IDs.
+   * Test case for HDFS-7575.
+   */
+  @Test (timeout=300000)
+  public void testUpgradeFrom26PreservesStorageIDs() throws IOException {
+    // StorageId present in the image testUpgradeFrom26PreservesStorageId.tgz
+    runLayoutUpgradeTest(GenericTestUtils.getMethodName(),
+                         "DS-a0e39cfa-930f-4abd-813c-e22b59223774");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
index bbaf3ed..dac26a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
@@ -476,7 +476,7 @@ public class UpgradeUtilities {
     for (int i = 0; i < parent.length; i++) {
       File versionFile = new File(parent[i], "VERSION");
       StorageDirectory sd = new StorageDirectory(parent[i].getParentFile());
-      storage.createStorageID(sd);
+      storage.createStorageID(sd, false);
       storage.writeProperties(versionFile, sd);
       versionFiles[i] = versionFile;
       File bpDir = BlockPoolSliceStorage.getBpRoot(bpid, parent[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 0610b94..6ff4603 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -510,7 +510,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public SimulatedFSDataset(DataStorage storage, Configuration conf) {
     if (storage != null) {
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
-        storage.createStorageID(storage.getStorageDir(i));
+        storage.createStorageID(storage.getStorageDir(i), false);
       }
       this.datanodeUuid = storage.getDatanodeUuid();
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index f3d15de..8f87f57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -83,7 +83,7 @@ public class TestFsDatasetImpl {
 
   private static Storage.StorageDirectory createStorageDirectory(File root) {
     Storage.StorageDirectory sd = new Storage.StorageDirectory(root);
-    dsForStorageUuid.createStorageID(sd);
+    dsForStorageUuid.createStorageID(sd, false);
     return sd;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz
new file mode 100644
index 0000000..30b0324
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
new file mode 100644
index 0000000..6b0e1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
@@ -0,0 +1,25 @@
+# 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.
+
+# Similar to hadoop-dfs-dir.txt, except this is used for a datanode layout
+# upgrade test.
+# Uncomment the following line to produce checksum info for a new DFS image.
+#printChecksums
+
+/f01    4021661486
+/f02    4021661486
+/f03    4021661486
+/f04    4021661486
+overallCRC	3193029345

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz
new file mode 100644
index 0000000..74c1649
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
new file mode 100644
index 0000000..6b0e1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
@@ -0,0 +1,25 @@
+# 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.
+
+# Similar to hadoop-dfs-dir.txt, except this is used for a datanode layout
+# upgrade test.
+# Uncomment the following line to produce checksum info for a new DFS image.
+#printChecksums
+
+/f01    4021661486
+/f02    4021661486
+/f03    4021661486
+/f04    4021661486
+overallCRC	3193029345

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz
new file mode 100644
index 0000000..69fbaf6
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d34074e2/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt
new file mode 100644
index 0000000..6b0e1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt
@@ -0,0 +1,25 @@
+# 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.
+
+# Similar to hadoop-dfs-dir.txt, except this is used for a datanode layout
+# upgrade test.
+# Uncomment the following line to produce checksum info for a new DFS image.
+#printChecksums
+
+/f01    4021661486
+/f02    4021661486
+/f03    4021661486
+/f04    4021661486
+overallCRC	3193029345