Browse Source

HDFS-9775. Erasure Coding : Rename BlockRecoveryWork to BlockReconstructionWork. Contributed by Rakesh R.

Change-Id: I6dfc8efd94fa2bbb4eec0e4730a5a4f92c8a5519
Zhe Zhang 9 years ago
parent
commit
a0fb2eff9b

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

@@ -921,6 +921,9 @@ Trunk (Unreleased)
       HDFS-9658. Erasure Coding: allow to use multiple EC policies in striping
       related tests. (Rui Li via zhz)
 
+      HDFS-9775. Erasure Coding : Rename BlockRecoveryWork to
+      BlockReconstructionWork. (Rakesh R via zhz)
+
 Release 2.9.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 30 - 25
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1450,7 +1450,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Scan blocks in {@link #neededReplications} and assign recovery
+   * Scan blocks in {@link #neededReplications} and assign reconstruction
    * (replication or erasure coding) work to data-nodes they belong to.
    *
    * The number of process blocks equals either twice the number of live
@@ -1458,7 +1458,7 @@ public class BlockManager implements BlockStatsMXBean {
    *
    * @return number of blocks scheduled for replication during this iteration.
    */
-  int computeBlockRecoveryWork(int blocksToProcess) {
+  int computeBlockReconstructionWork(int blocksToProcess) {
     List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
@@ -1468,30 +1468,33 @@ public class BlockManager implements BlockStatsMXBean {
     } finally {
       namesystem.writeUnlock();
     }
-    return computeRecoveryWorkForBlocks(blocksToReplicate);
+    return computeReconstructionWorkForBlocks(blocksToReplicate);
   }
 
   /**
-   * Recover a set of blocks to full strength through replication or
+   * Reconstruct a set of blocks to full strength through replication or
    * erasure coding
    *
-   * @param blocksToRecover blocks to be recovered, for each priority
+   * @param blocksToReconstruct blocks to be reconstructed, for each priority
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
+  int computeReconstructionWorkForBlocks(
+      List<List<BlockInfo>> blocksToReconstruct) {
     int scheduledWork = 0;
-    List<BlockRecoveryWork> recovWork = new LinkedList<>();
+    List<BlockReconstructionWork> reconWork = new LinkedList<>();
 
     // Step 1: categorize at-risk blocks into replication and EC tasks
     namesystem.writeLock();
     try {
       synchronized (neededReplications) {
-        for (int priority = 0; priority < blocksToRecover.size(); priority++) {
-          for (BlockInfo block : blocksToRecover.get(priority)) {
-            BlockRecoveryWork rw = scheduleRecovery(block, priority);
+        for (int priority = 0; priority < blocksToReconstruct
+            .size(); priority++) {
+          for (BlockInfo block : blocksToReconstruct.get(priority)) {
+            BlockReconstructionWork rw = scheduleReconstruction(block,
+                priority);
             if (rw != null) {
-              recovWork.add(rw);
+              reconWork.add(rw);
             }
           }
         }
@@ -1500,9 +1503,9 @@ public class BlockManager implements BlockStatsMXBean {
       namesystem.writeUnlock();
     }
 
-    // Step 2: choose target nodes for each recovery task
+    // Step 2: choose target nodes for each reconstruction task
     final Set<Node> excludedNodes = new HashSet<>();
-    for(BlockRecoveryWork rw : recovWork){
+    for(BlockReconstructionWork rw : reconWork){
       // Exclude all of the containing nodes from being targets.
       // This list includes decommissioning or corrupt nodes.
       excludedNodes.clear();
@@ -1521,7 +1524,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Step 3: add tasks to the DN
     namesystem.writeLock();
     try {
-      for(BlockRecoveryWork rw : recovWork){
+      for(BlockReconstructionWork rw : reconWork){
         final DatanodeStorageInfo[] targets = rw.getTargets();
         if(targets == null || targets.length == 0){
           rw.resetTargets();
@@ -1529,7 +1532,7 @@ public class BlockManager implements BlockStatsMXBean {
         }
 
         synchronized (neededReplications) {
-          if (validateRecoveryWork(rw)) {
+          if (validateReconstructionWork(rw)) {
             scheduledWork++;
           }
         }
@@ -1540,7 +1543,7 @@ public class BlockManager implements BlockStatsMXBean {
 
     if (blockLog.isDebugEnabled()) {
       // log which blocks have been scheduled for replication
-      for(BlockRecoveryWork rw : recovWork){
+      for(BlockReconstructionWork rw : reconWork){
         DatanodeStorageInfo[] targets = rw.getTargets();
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
@@ -1567,7 +1570,8 @@ public class BlockManager implements BlockStatsMXBean {
         (pendingReplicaNum > 0 || isPlacementPolicySatisfied(block));
   }
 
-  private BlockRecoveryWork scheduleRecovery(BlockInfo block, int priority) {
+  private BlockReconstructionWork scheduleReconstruction(BlockInfo block,
+      int priority) {
     // block should belong to a file
     BlockCollection bc = getBlockCollection(block);
     // abandoned block or block reopened for append
@@ -1589,8 +1593,8 @@ public class BlockManager implements BlockStatsMXBean {
         containingNodes, liveReplicaNodes, numReplicas,
         liveBlockIndices, priority);
     if(srcNodes == null || srcNodes.length == 0) {
-      // block can not be recovered from any node
-      LOG.debug("Block " + block + " cannot be recovered " +
+      // block can not be reconstructed from any node
+      LOG.debug("Block " + block + " cannot be reconstructed " +
           "from any node");
       return null;
     }
@@ -1618,7 +1622,7 @@ public class BlockManager implements BlockStatsMXBean {
 
     if (block.isStriped()) {
       if (pendingNum > 0) {
-        // Wait the previous recovery to finish.
+        // Wait the previous reconstruction to finish.
         return null;
       }
       byte[] indices = new byte[liveBlockIndices.size()];
@@ -1635,7 +1639,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
-  private boolean validateRecoveryWork(BlockRecoveryWork rw) {
+  private boolean validateReconstructionWork(BlockReconstructionWork rw) {
     BlockInfo block = rw.getBlock();
     int priority = rw.getPriority();
     // Recheck since global lock was released
@@ -1672,11 +1676,12 @@ public class BlockManager implements BlockStatsMXBean {
       }
     }
 
-    // Add block to the to be recovered list
+    // Add block to the to be reconstructed list
     if (block.isStriped()) {
       assert rw instanceof ErasureCodingWork;
       assert rw.getTargets().length > 0;
-      assert pendingNum == 0: "Should wait the previous recovery to finish";
+      assert pendingNum == 0 : "Should wait the previous reconstruction"
+          + " to finish";
       String src = getBlockCollection(block).getName();
       ErasureCodingPolicy ecPolicy = null;
       try {
@@ -1687,7 +1692,7 @@ public class BlockManager implements BlockStatsMXBean {
       }
       if (ecPolicy == null) {
         blockLog.warn("No erasure coding policy found for the file {}. "
-            + "So cannot proceed for recovery", src);
+            + "So cannot proceed for reconstruction", src);
         // TODO: we may have to revisit later for what we can do better to
         // handle this case.
         return false;
@@ -4239,7 +4244,7 @@ public class BlockManager implements BlockStatsMXBean {
     final int nodesToProcess = (int) Math.ceil(numlive
         * this.blocksInvalidateWorkPct);
 
-    int workFound = this.computeBlockRecoveryWork(blocksToProcess);
+    int workFound = this.computeBlockReconstructionWork(blocksToProcess);
 
     // Update counters
     namesystem.writeLock();

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReconstructionWork.java

@@ -25,17 +25,17 @@ import java.util.Set;
 
 /**
  * This class is used internally by
- * {@link BlockManager#computeRecoveryWorkForBlocks} to represent a task to
- * recover a block through replication or erasure coding. Recovery is done by
- * transferring data from srcNodes to targets
+ * {@link BlockManager#computeReconstructionWorkForBlocks} to represent a
+ * task to reconstruct a block through replication or erasure coding.
+ * Reconstruction is done by transferring data from srcNodes to targets
  */
-abstract class BlockRecoveryWork {
+abstract class BlockReconstructionWork {
   private final BlockInfo block;
 
   private final BlockCollection bc;
 
   /**
-   * An erasure coding recovery task has multiple source nodes.
+   * An erasure coding reconstruction task has multiple source nodes.
    * A replication task only has 1 source node, stored on top of the array
    */
   private final DatanodeDescriptor[] srcNodes;
@@ -48,7 +48,7 @@ abstract class BlockRecoveryWork {
   private DatanodeStorageInfo[] targets;
   private final int priority;
 
-  public BlockRecoveryWork(BlockInfo block,
+  public BlockReconstructionWork(BlockInfo block,
       BlockCollection bc,
       DatanodeDescriptor[] srcNodes,
       List<DatanodeDescriptor> containingNodes,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -608,7 +608,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     BlockECReconstructionInfo task = new BlockECReconstructionInfo(block,
         sources, targets, liveBlockIndices, ecPolicy);
     erasurecodeBlocks.offer(task);
-    BlockManager.LOG.debug("Adding block recovery task " + task + "to "
+    BlockManager.LOG.debug("Adding block reconstruction task " + task + "to "
         + getName() + ", current queue size is " + erasurecodeBlocks.size());
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java

@@ -22,7 +22,7 @@ import org.apache.hadoop.net.Node;
 import java.util.List;
 import java.util.Set;
 
-class ErasureCodingWork extends BlockRecoveryWork {
+class ErasureCodingWork extends BlockReconstructionWork {
   private final byte[] liveBlockIndicies;
 
   public ErasureCodingWork(BlockInfo block,

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java

@@ -19,11 +19,10 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.net.Node;
 
-import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
-class ReplicationWork extends BlockRecoveryWork {
+class ReplicationWork extends BlockReconstructionWork {
   public ReplicationWork(BlockInfo block, BlockCollection bc,
       DatanodeDescriptor[] srcNodes, List<DatanodeDescriptor> containingNodes,
       List<DatanodeStorageInfo> liveReplicaStorages, int additionalReplRequired,
@@ -33,7 +32,8 @@ class ReplicationWork extends BlockRecoveryWork {
     assert getSrcNodes().length == 1 :
         "There should be exactly 1 source node that have been selected";
     getSrcNodes()[0].incrementPendingReplicationWithoutTargets();
-    BlockManager.LOG.debug("Creating a ReplicationWork to recover " + block);
+    BlockManager.LOG
+        .debug("Creating a ReplicationWork to reconstruct " + block);
   }
 
   @Override

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

@@ -163,7 +163,7 @@ public class BlockManagerTestUtil {
    */
   public static int computeAllPendingWork(BlockManager bm) {
     int work = computeInvalidationWork(bm);
-    work += bm.computeBlockRecoveryWork(Integer.MAX_VALUE);
+    work += bm.computeBlockReconstructionWork(Integer.MAX_VALUE);
     return work;
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -540,8 +540,8 @@ public class TestBlockManager {
     assertEquals("Block not initially pending replication", 0,
         bm.pendingReplications.getNumReplicas(block));
     assertEquals(
-        "computeBlockRecoveryWork should indicate replication is needed", 1,
-        bm.computeRecoveryWorkForBlocks(list_all));
+        "computeBlockReconstructionWork should indicate replication is needed",
+        1, bm.computeReconstructionWorkForBlocks(list_all));
     assertTrue("replication is pending after work is computed",
         bm.pendingReplications.getNumReplicas(block) > 0);