瀏覽代碼

HDFS-16888. BlockManager#maxReplicationStreams, replicationStreamsHardLimit, blocksReplWorkMultiplier and PendingReconstructionBlocks#timeout should be volatile (#5296)

Reviewed-by: Tao Li <tomscut@apache.org>
Signed-off-by: Takanobu Asanuma <tasanuma@apache.org>
(cherry picked from commit 88c8ac750dc630040a37014dd13391922e625292)
huhaiyang 2 年之前
父節點
當前提交
80b42625cf

+ 13 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -386,12 +386,12 @@ public class BlockManager implements BlockStatsMXBean {
    * The maximum number of outgoing replication streams a given node should have
    * at one time considering all but the highest priority replications needed.
     */
-  int maxReplicationStreams;
+  private volatile int maxReplicationStreams;
   /**
    * The maximum number of outgoing replication streams a given node should have
    * at one time.
    */
-  int replicationStreamsHardLimit;
+  private volatile int replicationStreamsHardLimit;
   /** Minimum copies needed or else write is disallowed */
   public final short minReplication;
   /** Default number of replicas */
@@ -400,7 +400,7 @@ public class BlockManager implements BlockStatsMXBean {
   final int maxCorruptFilesReturned;
 
   final float blocksInvalidateWorkPct;
-  private int blocksReplWorkMultiplier;
+  private volatile int blocksReplWorkMultiplier;
 
   // whether or not to issue block encryption keys.
   final boolean encryptDataTransfer;
@@ -974,12 +974,19 @@ public class BlockManager implements BlockStatsMXBean {
    *
    * @param newVal - Must be a positive non-zero integer.
    */
-  public void setMaxReplicationStreams(int newVal) {
-    ensurePositiveInt(newVal,
-        DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY);
+  @VisibleForTesting
+  public void setMaxReplicationStreams(int newVal, boolean ensurePositiveInt) {
+    if (ensurePositiveInt) {
+      ensurePositiveInt(newVal,
+          DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY);
+    }
     maxReplicationStreams = newVal;
   }
 
+  public void setMaxReplicationStreams(int newVal) {
+    setMaxReplicationStreams(newVal, true);
+  }
+
   /** Returns the current setting for maxReplicationStreamsHardLimit, set by
    * {@code DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY}.
    *

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

@@ -59,7 +59,7 @@ class PendingReconstructionBlocks {
   // It might take anywhere between 5 to 10 minutes before
   // a request is timed out.
   //
-  private long timeout =
+  private volatile long timeout =
       DFS_NAMENODE_RECONSTRUCTION_PENDING_TIMEOUT_SEC_DEFAULT * 1000;
   private final static long DEFAULT_RECHECK_INTERVAL = 5 * 60 * 1000;
 

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

@@ -44,7 +44,7 @@ public class BlockManagerTestUtil {
 
   public static void setNodeReplicationLimit(final BlockManager blockManager,
       final int limit) {
-    blockManager.maxReplicationStreams = limit;
+    blockManager.setMaxReplicationStreams(limit, false);
   }
 
   /** @return the datanode descriptor for the given the given storageID. */

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

@@ -677,8 +677,8 @@ public class TestBlockManager {
    */
   @Test
   public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception {
-    bm.maxReplicationStreams = 0;
-    bm.replicationStreamsHardLimit = 1;
+    bm.setMaxReplicationStreams(0, false);
+    bm.setReplicationStreamsHardLimit(1);
 
     long blockId = 42;         // arbitrary
     Block aBlock = new Block(blockId, 0, 0);
@@ -735,7 +735,7 @@ public class TestBlockManager {
 
   @Test
   public void testChooseSrcDatanodesWithDupEC() throws Exception {
-    bm.maxReplicationStreams = 4;
+    bm.setMaxReplicationStreams(4, false);
 
     long blockId = -9223372036854775776L; // real ec block id
     Block aBlock = new Block(blockId, 0, 0);
@@ -895,7 +895,7 @@ public class TestBlockManager {
     assertNotNull(work);
 
     // simulate the 2 nodes reach maxReplicationStreams
-    for(int i = 0; i < bm.maxReplicationStreams; i++){
+    for(int i = 0; i < bm.getMaxReplicationStreams(); i++){
       ds3.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets();
       ds4.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets();
     }
@@ -939,7 +939,7 @@ public class TestBlockManager {
     assertNotNull(work);
 
     // simulate the 1 node reaches maxReplicationStreams
-    for(int i = 0; i < bm.maxReplicationStreams; i++){
+    for(int i = 0; i < bm.getMaxReplicationStreams(); i++){
       ds2.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets();
     }
 
@@ -948,7 +948,7 @@ public class TestBlockManager {
     assertNotNull(work);
 
     // simulate the 1 more node reaches maxReplicationStreams
-    for(int i = 0; i < bm.maxReplicationStreams; i++){
+    for(int i = 0; i < bm.getMaxReplicationStreams(); i++){
       ds3.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets();
     }
 
@@ -997,7 +997,7 @@ public class TestBlockManager {
     DatanodeDescriptor[] dummyDDArray = new DatanodeDescriptor[]{dummyDD};
     DatanodeStorageInfo[] dummyDSArray = new DatanodeStorageInfo[]{ds1};
     // Simulate the 2 nodes reach maxReplicationStreams.
-    for(int i = 0; i < bm.maxReplicationStreams; i++){ //Add some dummy EC reconstruction task.
+    for(int i = 0; i < bm.getMaxReplicationStreams(); i++){ //Add some dummy EC reconstruction task.
       ds3.getDatanodeDescriptor().addBlockToBeErasureCoded(dummyBlock, dummyDDArray,
               dummyDSArray, new byte[0], new byte[0], ecPolicy);
       ds4.getDatanodeDescriptor().addBlockToBeErasureCoded(dummyBlock, dummyDDArray,
@@ -1011,8 +1011,8 @@ public class TestBlockManager {
 
   @Test
   public void testFavorDecomUntilHardLimit() throws Exception {
-    bm.maxReplicationStreams = 0;
-    bm.replicationStreamsHardLimit = 1;
+    bm.setMaxReplicationStreams(0, false);
+    bm.setReplicationStreamsHardLimit(1);
 
     long blockId = 42;         // arbitrary
     Block aBlock = new Block(blockId, 0, 0);

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

@@ -159,9 +159,9 @@ public class TestUnderReplicatedBlocks {
 
       BlockManagerTestUtil.updateState(bm);
       assertTrue("The number of blocks to be replicated should be less than "
-          + "or equal to " + bm.replicationStreamsHardLimit,
+          + "or equal to " + bm.getReplicationStreamsHardLimit(),
           secondDn.getNumberOfBlocksToBeReplicated()
-          <= bm.replicationStreamsHardLimit);
+          <= bm.getReplicationStreamsHardLimit());
       DFSTestUtil.verifyClientStats(conf, cluster);
     } finally {
       cluster.shutdown();