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 bo...@apache.org on 2018/08/02 17:17:35 UTC

[12/50] [abbrv] hadoop git commit: HDFS-12716. 'dfs.datanode.failed.volumes.tolerated' to support minimum number of volumes to be available. Contributed by Ranith Sardar and usharani

HDFS-12716. 'dfs.datanode.failed.volumes.tolerated' to support minimum number of volumes to be available. Contributed by Ranith Sardar and usharani


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

Branch: refs/heads/YARN-7402
Commit: 3108d27edde941d153a58f71fb1096cce2995531
Parents: 63e08ec
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Mon Jul 30 15:50:04 2018 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Mon Jul 30 15:50:04 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/DataNode.java   |  7 +++-
 .../datanode/checker/DatasetVolumeChecker.java  |  6 ++-
 .../checker/StorageLocationChecker.java         | 28 ++++++++++----
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 40 ++++++++++++++++----
 .../src/main/resources/hdfs-default.xml         |  2 +
 .../TestDataNodeVolumeFailureToleration.java    |  6 ++-
 6 files changed, 68 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
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 7df92f6..1e9c57a 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
@@ -293,6 +293,8 @@ public class DataNode extends ReconfigurableBase
       "  and rolling upgrades.";
 
   static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
+  public static final int MAX_VOLUME_FAILURE_TOLERATED_LIMIT = -1;
+  public static final String MAX_VOLUME_FAILURES_TOLERATED_MSG = "should be greater than -1";
 
   /** A list of property that are reconfigurable at runtime. */
   private static final List<String> RECONFIGURABLE_PROPERTIES =
@@ -1389,10 +1391,11 @@ public class DataNode extends ReconfigurableBase
 
     int volFailuresTolerated = dnConf.getVolFailuresTolerated();
     int volsConfigured = dnConf.getVolsConfigured();
-    if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
+    if (volFailuresTolerated < MAX_VOLUME_FAILURE_TOLERATED_LIMIT
+        || volFailuresTolerated >= volsConfigured) {
       throw new DiskErrorException("Invalid value configured for "
           + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
-          + ". Value configured is either less than 0 or >= "
+          + ". Value configured is either greater than -1 or >= "
           + "to the number of configured volumes (" + volsConfigured + ").");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
index 3889e23..30602c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/DatasetVolumeChecker.java
@@ -28,6 +28,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -153,10 +154,11 @@ public class DatasetVolumeChecker {
 
     lastAllVolumesCheck = timer.monotonicNow() - minDiskCheckGapMs;
 
-    if (maxVolumeFailuresTolerated < 0) {
+    if (maxVolumeFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
       throw new DiskErrorException("Invalid value configured for "
           + DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
-          + maxVolumeFailuresTolerated + " (should be non-negative)");
+          + maxVolumeFailuresTolerated + " "
+          + DataNode.MAX_VOLUME_FAILURES_TOLERATED_MSG);
     }
 
     delegateChecker = new ThrottledAsyncChecker<>(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
index 81575e2..dabaa83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/StorageLocationChecker.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation.CheckContext;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -105,10 +106,11 @@ public class StorageLocationChecker {
         DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
         DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
 
-    if (maxVolumeFailuresTolerated < 0) {
+    if (maxVolumeFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
       throw new DiskErrorException("Invalid value configured for "
           + DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
-          + maxVolumeFailuresTolerated + " (should be non-negative)");
+          + maxVolumeFailuresTolerated + " "
+          + DataNode.MAX_VOLUME_FAILURES_TOLERATED_MSG);
     }
 
     this.timer = timer;
@@ -213,12 +215,22 @@ public class StorageLocationChecker {
       }
     }
 
-    if (failedLocations.size() > maxVolumeFailuresTolerated) {
-      throw new DiskErrorException("Too many failed volumes - "
-          + "current valid volumes: " + goodLocations.size()
-          + ", volumes configured: " + dataDirs.size()
-          + ", volumes failed: " + failedLocations.size()
-          + ", volume failures tolerated: " + maxVolumeFailuresTolerated);
+    if (maxVolumeFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
+      if (dataDirs.size() == failedLocations.size()) {
+        throw new DiskErrorException("Too many failed volumes - "
+            + "current valid volumes: " + goodLocations.size()
+            + ", volumes configured: " + dataDirs.size()
+            + ", volumes failed: " + failedLocations.size()
+            + ", volume failures tolerated: " + maxVolumeFailuresTolerated);
+      }
+    } else {
+      if (failedLocations.size() > maxVolumeFailuresTolerated) {
+        throw new DiskErrorException("Too many failed volumes - "
+            + "current valid volumes: " + goodLocations.size()
+            + ", volumes configured: " + dataDirs.size()
+            + ", volumes failed: " + failedLocations.size()
+            + ", volume failures tolerated: " + maxVolumeFailuresTolerated);
+      }
     }
 
     if (goodLocations.size() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
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 89c278a..d7f133e 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
@@ -237,6 +237,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   final FsDatasetCache cacheManager;
   private final Configuration conf;
   private final int volFailuresTolerated;
+  private final int volsConfigured;
   private volatile boolean fsRunning;
 
   final ReplicaMap volumeMap;
@@ -285,15 +286,32 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     List<VolumeFailureInfo> volumeFailureInfos = getInitialVolumeFailureInfos(
         dataLocations, storage);
 
-    int volsConfigured = datanode.getDnConf().getVolsConfigured();
+    volsConfigured = datanode.getDnConf().getVolsConfigured();
     int volsFailed = volumeFailureInfos.size();
 
-    if (volsFailed > volFailuresTolerated) {
-      throw new DiskErrorException("Too many failed volumes - "
-          + "current valid volumes: " + storage.getNumStorageDirs() 
-          + ", volumes configured: " + volsConfigured 
-          + ", volumes failed: " + volsFailed
-          + ", volume failures tolerated: " + volFailuresTolerated);
+    if (volFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT
+        || volFailuresTolerated >= volsConfigured) {
+      throw new DiskErrorException("Invalid value configured for "
+          + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
+          + ". Value configured is either less than maxVolumeFailureLimit or greater than "
+          + "to the number of configured volumes (" + volsConfigured + ").");
+    }
+    if (volFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
+      if (volsConfigured == volsFailed) {
+        throw new DiskErrorException(
+            "Too many failed volumes - " + "current valid volumes: "
+                + storage.getNumStorageDirs() + ", volumes configured: "
+                + volsConfigured + ", volumes failed: " + volsFailed
+                + ", volume failures tolerated: " + volFailuresTolerated);
+      }
+    } else {
+      if (volsFailed > volFailuresTolerated) {
+        throw new DiskErrorException(
+            "Too many failed volumes - " + "current valid volumes: "
+                + storage.getNumStorageDirs() + ", volumes configured: "
+                + volsConfigured + ", volumes failed: " + volsFailed
+                + ", volume failures tolerated: " + volFailuresTolerated);
+      }
     }
 
     storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
@@ -597,7 +615,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    */
   @Override // FsDatasetSpi
   public boolean hasEnoughResource() {
-    return getNumFailedVolumes() <= volFailuresTolerated;
+    if (volFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
+      // If volFailuresTolerated configured maxVolumeFailureLimit then minimum
+      // one volume is required.
+      return volumes.getVolumes().size() >= 1;
+    } else {
+      return getNumFailedVolumes() <= volFailuresTolerated;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index a10be27..9e73197 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -1276,6 +1276,8 @@
   <description>The number of volumes that are allowed to
   fail before a datanode stops offering service. By default
   any volume failure will cause a datanode to shutdown.
+  The range of the value is -1 now, -1 represents the minimum
+  of volume valids is 1.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3108d27e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
----------------------------------------------------------------------
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 f83609a..825887c 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
@@ -201,7 +201,11 @@ public class TestDataNodeVolumeFailureToleration {
   @Test
   public void testVolumeAndTolerableConfiguration() throws Exception {
     // Check if Block Pool Service exit for an invalid conf value.
-    testVolumeConfig(-1, 0, false, true);
+    testVolumeConfig(-2, 0, false, true);
+    // Test for one good volume at least
+    testVolumeConfig(-1, 0, true, true);
+    testVolumeConfig(-1, 1, true, true);
+    testVolumeConfig(-1, 2, false, true);
 
     // Ditto if the value is too big.
     testVolumeConfig(100, 0, false, true);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org