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 3ea8ce36dfd01..0602dfb2c99be 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.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 synchronized void setDatanodeUuid(String newDatanodeUuid) { 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 @@ private void doTransition( DataNode datanode, + 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; } 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 4fe07b9b4b37d..4d224d5fd19ee 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 enum State { 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 StorageType getStorageType() { * @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 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 88ad0cc2dac7c..ad907f6374870 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 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 @@ private static class ReferenceFileInfo { } } + public interface ClusterVerifier { + public void verifyClusterPostUpgrade(final MiniDFSCluster cluster) throws IOException; + } + final LinkedList refList = new LinkedList(); Iterator refIter; @@ -119,7 +122,7 @@ void unpackStorage(String tarFileName, String referenceName) 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 void testFailOnPreUpgradeImage() throws IOException { 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 void testUpgradeFromCorruptRel22Image() throws IOException { // 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 @@ static void recoverAllLeases(DFSClient dfs, } 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 @@ void upgradeAndVerify(MiniDFSCluster.Builder bld) } recoverAllLeases(dfsClient, new Path("/")); verifyFileSystem(dfs); + + if (verifier != null) { + verifier.verifyClusterPostUpgrade(cluster); + } } finally { if (cluster != null) { cluster.shutdown(); } } @@ -611,6 +618,6 @@ public void testUpgradeFromRel1BBWImage() throws IOException { "data1"); upgradeAndVerify(new MiniDFSCluster.Builder(conf). numDataNodes(1).enableManagedDfsDirsRedundancy(false). - manageDataDfsDirs(false)); + manageDataDfsDirs(false), null); } } 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 0966301cb4e76..343320c0326fa 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 void testUpgradeToIdBasedLayout() throws IOException { 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); } } 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 0000000000000..e262abcb6a8a9 --- /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"); + } +} 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 bbaf3ed0e3e4f..dac26a08b9890 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 static void createDataNodeVersionFile(File[] parent, 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]); 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 0610b94008287..6ff4603a04d50 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 FsDatasetSpi getDataset() { 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 { 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 f3d15de694334..8f87f57306c6c 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; } 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 0000000000000..30b03246f955a Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22FixesStorageIDs.tgz differ 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 0000000000000..6b0e1eacdf323 --- /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 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 0000000000000..74c1649abcaf0 Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom22via26FixesStorageIDs.tgz differ 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 0000000000000..6b0e1eacdf323 --- /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 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 0000000000000..69fbaf61ac814 Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testUpgradeFrom26PreservesStorageIDs.tgz differ 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 0000000000000..6b0e1eacdf323 --- /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