From 314aa21a89134fac68ac3cb95efdeb56bd3d7b05 Mon Sep 17 00:00:00 2001 From: Andrew Wang Date: Wed, 13 Apr 2016 16:39:50 -0700 Subject: [PATCH] HDFS-10279. Improve validation of the configured number of tolerated failed volumes. Contributed by Lin Yiqun. --- .../hadoop/hdfs/server/datanode/DNConf.java | 18 ++++++++++++++++++ .../hadoop/hdfs/server/datanode/DataNode.java | 9 +++++++++ .../fsdataset/impl/FsDatasetImpl.java | 13 ++----------- .../TestDataNodeVolumeFailureToleration.java | 19 +++++++++++++++++-- 4 files changed, 46 insertions(+), 13 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java index 5cff2d3447e..b6164140b58 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java @@ -114,6 +114,9 @@ public class DNConf { // Allow LAZY_PERSIST writes from non-local clients? private final boolean allowNonLocalLazyPersist; + private final int volFailuresTolerated; + private final int volsConfigured; + public DNConf(Configuration conf) { this.conf = conf; socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, @@ -238,6 +241,13 @@ public class DNConf { this.bpReadyTimeout = conf.getLong( DFS_DATANODE_BP_READY_TIMEOUT_KEY, DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT); + + this.volFailuresTolerated = + conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, + DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT); + String[] dataDirs = + conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY); + this.volsConfigured = (dataDirs == null) ? 0 : dataDirs.length; } // We get minimumNameNodeVersion via a method so it can be mocked out in tests. @@ -363,4 +373,12 @@ public class DNConf { public long getLifelineIntervalMs() { return lifelineIntervalMs; } + + public int getVolFailuresTolerated() { + return volFailuresTolerated; + } + + public int getVolsConfigured() { + return volsConfigured; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 625eb3f3b6f..288fc3eb9f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -1268,6 +1268,15 @@ public class DataNode extends ReconfigurableBase LOG.info("Starting DataNode with maxLockedMemory = " + dnConf.maxLockedMemory); + int volFailuresTolerated = dnConf.getVolFailuresTolerated(); + int volsConfigured = dnConf.getVolsConfigured(); + if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) { + throw new DiskErrorException("Invalid value configured for " + + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated + + ". Value configured is either less than 0 or >= " + + "to the number of configured volumes (" + volsConfigured + ")."); + } + storage = new DataStorage(); // global DN settings diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index 381c79913ef..f7e0aaed565 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -268,24 +268,15 @@ class FsDatasetImpl implements FsDatasetSpi { this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf); // The number of volumes required for operation is the total number // of volumes minus the number of failed volumes we can tolerate. - volFailuresTolerated = - conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, - DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT); + volFailuresTolerated = datanode.getDnConf().getVolFailuresTolerated(); - String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY); Collection dataLocations = DataNode.getStorageLocations(conf); List volumeFailureInfos = getInitialVolumeFailureInfos( dataLocations, storage); - int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length; + int volsConfigured = datanode.getDnConf().getVolsConfigured(); int volsFailed = volumeFailureInfos.size(); - if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) { - throw new DiskErrorException("Invalid value configured for " - + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated - + ". Value configured is either less than 0 or >= " - + "to the number of configured volumes (" + volsConfigured + ")."); - } if (volsFailed > volFailuresTolerated) { throw new DiskErrorException("Too many failed volumes - " + "current valid volumes: " + storage.getNumStorageDirs() diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java index 1eb8bcaf77d..2f8239e3275 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java @@ -34,6 +34,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -229,9 +231,22 @@ public class TestDataNodeVolumeFailureToleration { prepareDirToFail(dirs[i]); } restartDatanodes(volumesTolerated, manageDfsDirs); - assertEquals(expectedBPServiceState, cluster.getDataNodes().get(0) - .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId())); + } catch (DiskErrorException e) { + GenericTestUtils.assertExceptionContains("Invalid value configured for " + + "dfs.datanode.failed.volumes.tolerated", e); } finally { + boolean bpServiceState; + // If the datanode not registered successfully, + // because the invalid value configured for tolerated volumes + if (cluster.getDataNodes().size() == 0) { + bpServiceState = false; + } else { + bpServiceState = + cluster.getDataNodes().get(0) + .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()); + } + assertEquals(expectedBPServiceState, bpServiceState); + for (File dir : dirs) { FileUtil.chmod(dir.toString(), "755"); }