Browse Source

HDFS-17430. RecoveringBlock will skip no live replicas when get block recovery command. (#6635)

huhaiyang 1 year ago
parent
commit
8cd4704e0a

+ 21 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -1729,12 +1729,13 @@ public class DatanodeManager {
             " where it is not under construction.");
       }
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-      // Skip stale nodes during recovery
-      final List<DatanodeStorageInfo> recoveryLocations =
+      // Skip stale and dead nodes during recovery.
+      List<DatanodeStorageInfo> recoveryLocations =
           new ArrayList<>(storages.length);
-      final List<Integer> storageIdx = new ArrayList<>(storages.length);
+      List<Integer> storageIdx = new ArrayList<>(storages.length);
       for (int i = 0; i < storages.length; ++i) {
-        if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
+        if (!storages[i].getDatanodeDescriptor().isStale(staleInterval) &&
+            storages[i].getDatanodeDescriptor().isAlive()) {
           recoveryLocations.add(storages[i]);
           storageIdx.add(i);
         }
@@ -1747,20 +1748,31 @@ public class DatanodeManager {
       ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
           new ExtendedBlock(blockPoolId, uc.getTruncateBlock()) :
           new ExtendedBlock(blockPoolId, b);
-      // If we only get 1 replica after eliminating stale nodes, choose all
+      // If we only get 1 replica after eliminating stale and dead nodes, choose all live
       // replicas for recovery and let the primary data node handle failures.
       DatanodeInfo[] recoveryInfos;
       if (recoveryLocations.size() > 1) {
         if (recoveryLocations.size() != storages.length) {
-          LOG.info("Skipped stale nodes for recovery : "
+          LOG.info("Skipped stale and dead nodes for recovery : "
               + (storages.length - recoveryLocations.size()));
         }
-        recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(recoveryLocations);
       } else {
-        // If too many replicas are stale, then choose all replicas to
+        // If too many replicas are stale, then choose all live replicas to
         // participate in block recovery.
-        recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
+        recoveryLocations.clear();
+        storageIdx.clear();
+        for (int i = 0; i < storages.length; ++i) {
+          if (storages[i].getDatanodeDescriptor().isAlive()) {
+            recoveryLocations.add(storages[i]);
+            storageIdx.add(i);
+          }
+        }
+        if (recoveryLocations.size() != storages.length) {
+          LOG.info("Skipped dead nodes for recovery : {}",
+              storages.length - recoveryLocations.size());
+        }
       }
+      recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(recoveryLocations);
       RecoveringBlock rBlock;
       if (truncateRecovery) {
         Block recoveryBlock = (copyOnTruncateRecovery) ? b : uc.getTruncateBlock();