Browse Source

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

(cherry picked from commit 1d9d166c0beb56aa45e65f779044905acff25d88)
(cherry picked from commit ca8e1b0739b6653833f9bc8990ab126420703f66)
Arpit Agarwal 10 years ago
parent
commit
f81f97eb7c
15 changed files with 277 additions and 21 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 25 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  3. 18 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
  4. 13 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
  6. 139 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeStartupFixesLegacyStorageIDs.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
  10. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz
  11. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.txt
  12. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz
  13. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.txt
  14. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz
  15. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.txt

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

@@ -64,6 +64,9 @@ Release 2.6.1 - UNRELEASED
     HDFS-7533. Datanode sometimes does not shutdown on receiving upgrade
     shutdown command (Eric Payne via kihwal)
 
+    HDFS-7575. Upgrade should generate a unique storage ID for each
+    volume. (Arpit Agarwal)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 25 - 10
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;
     }
     

+ 18 - 1
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

+ 13 - 6
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);
   }
 }

+ 1 - 1
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);
   }
 }

+ 139 - 0
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");
+  }
+}

+ 1 - 1
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]);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -489,7 +489,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 {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -87,7 +87,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;
   }
 

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz


+ 25 - 0
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

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz


+ 25 - 0
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

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz


+ 25 - 0
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