瀏覽代碼

HDFS-13882. Set a maximum delay for retrying locateFollowingBlock. Contributed by Kitti Nanasi.

Signed-off-by: Xiao Chen <xiao@apache.org>
Kitti Nanasi 6 年之前
父節點
當前提交
10185d9a77

+ 19 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -942,6 +942,7 @@ public class DFSOutputStream extends FSOutputSummer
     long localstart = Time.monotonicNow();
     final DfsClientConf conf = dfsClient.getConf();
     long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+    long maxSleepTime = conf.getBlockWriteLocateFollowingMaxDelayMs();
     boolean fileComplete = false;
     int retries = conf.getNumBlockWriteLocateFollowingRetry();
     while (!fileComplete) {
@@ -965,7 +966,7 @@ public class DFSOutputStream extends FSOutputSummer
           }
           retries--;
           Thread.sleep(sleeptime);
-          sleeptime *= 2;
+          sleeptime = calculateDelayForNextRetry(sleeptime, maxSleepTime);
           if (Time.monotonicNow() - localstart > 5000) {
             DFSClient.LOG.info("Could not complete " + src + " retrying...");
           }
@@ -1075,6 +1076,7 @@ public class DFSOutputStream extends FSOutputSummer
     final DfsClientConf conf = dfsClient.getConf();
     int retries = conf.getNumBlockWriteLocateFollowingRetry();
     long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+    long maxSleepTime = conf.getBlockWriteLocateFollowingMaxDelayMs();
     long localstart = Time.monotonicNow();
     while (true) {
       try {
@@ -1106,7 +1108,7 @@ public class DFSOutputStream extends FSOutputSummer
               LOG.warn("NotReplicatedYetException sleeping " + src
                   + " retries left " + retries);
               Thread.sleep(sleeptime);
-              sleeptime *= 2;
+              sleeptime = calculateDelayForNextRetry(sleeptime, maxSleepTime);
             } catch (InterruptedException ie) {
               LOG.warn("Caught exception", ie);
             }
@@ -1117,4 +1119,19 @@ public class DFSOutputStream extends FSOutputSummer
       }
     }
   }
+
+  /**
+   * Calculates the delay for the next retry.
+   *
+   * The delay is increased exponentially until the maximum delay is reached.
+   *
+   * @param previousDelay delay for the previous retry
+   * @param maxDelay maximum delay
+   * @return the minimum of the double of the previous delay
+   * and the maximum delay
+   */
+  private static long calculateDelayForNextRetry(long previousDelay,
+                                                 long maxDelay) {
+    return Math.min(previousDelay * 2, maxDelay);
+  }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java

@@ -322,6 +322,9 @@ public interface HdfsClientConfigKeys {
     String  LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY =
         PREFIX + "locateFollowingBlock.initial.delay.ms";
     int     LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
+    String  LOCATEFOLLOWINGBLOCK_MAX_DELAY_MS_KEY =
+        PREFIX + "locateFollowingBlock.max.delay.ms";
+    int     LOCATEFOLLOWINGBLOCK_MAX_DELAY_MS_DEFAULT = 60000;
 
     interface ReplaceDatanodeOnFailure {
       String PREFIX = BlockWrite.PREFIX + "replace-datanode-on-failure.";

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java

@@ -122,6 +122,7 @@ public class DfsClientConf {
   private final int numBlockWriteRetry;
   private final int numBlockWriteLocateFollowingRetry;
   private final int blockWriteLocateFollowingInitialDelayMs;
+  private final int blockWriteLocateFollowingMaxDelayMs;
   private final long defaultBlockSize;
   private final long prefetchSize;
   private final short defaultReplication;
@@ -237,6 +238,9 @@ public class DfsClientConf {
     blockWriteLocateFollowingInitialDelayMs = conf.getInt(
         BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY,
         BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT);
+    blockWriteLocateFollowingMaxDelayMs = conf.getInt(
+        BlockWrite.LOCATEFOLLOWINGBLOCK_MAX_DELAY_MS_KEY,
+        BlockWrite.LOCATEFOLLOWINGBLOCK_MAX_DELAY_MS_DEFAULT);
     uMask = FsPermission.getUMask(conf);
     connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
         DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
@@ -349,6 +353,10 @@ public class DfsClientConf {
     return blockWriteLocateFollowingInitialDelayMs;
   }
 
+  public int getBlockWriteLocateFollowingMaxDelayMs() {
+    return blockWriteLocateFollowingMaxDelayMs;
+  }
+
   /**
    * @return the hdfsTimeout
    */

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -3135,10 +3135,21 @@
   <name>dfs.client.block.write.locateFollowingBlock.initial.delay.ms</name>
   <value>400</value>
   <description>The initial delay (unit is ms) for locateFollowingBlock,
-    the delay time will increase exponentially(double) for each retry.
+    the delay time will increase exponentially(double) for each retry
+    until dfs.client.block.write.locateFollowingBlock.max.delay.ms is reached,
+    after that the delay for each retry will be
+    dfs.client.block.write.locateFollowingBlock.max.delay.ms.
   </description>
 </property>
 
+<property>
+    <name>dfs.client.block.write.locateFollowingBlock.max.delay.ms</name>
+    <value>60000</value>
+    <description>
+      The maximum delay (unit is ms) before retrying locateFollowingBlock.
+    </description>
+</property>
+
 <property>
   <name>dfs.ha.zkfc.nn.http.timeout.ms</name>
   <value>20000</value>

+ 37 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -1229,27 +1229,54 @@ public class TestDFSClientRetries {
     }
   }
 
+  /**
+   * Tests default configuration values and configuration setting
+   * of locate following block delays and number of retries.
+   *
+   * Configuration values tested:
+   * - dfs.client.block.write.locateFollowingBlock.initial.delay.ms
+   * - dfs.client.block.write.locateFollowingBlock.max.delay.ms
+   * - dfs.client.block.write.locateFollowingBlock.retries
+   */
   @Test
-  public void testDFSClientConfigurationLocateFollowingBlockInitialDelay()
+  public void testDFSClientConfigurationLocateFollowingBlock()
       throws Exception {
-    // test if HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY
-    // is not configured, verify DFSClient uses the default value 400.
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    final int initialDelayTestValue = 1000;
+    final int maxDelayTestValue = 35000;
+    final int retryTestValue = 7;
+
+    final int defaultInitialDelay = 400;
+    final int defaultMaxDelay = 60000;
+    final int defultRetry = 5;
+
     try {
       cluster.waitActive();
       NamenodeProtocols nn = cluster.getNameNodeRpc();
       DFSClient client = new DFSClient(null, nn, conf, null);
-      assertEquals(client.getConf().
-          getBlockWriteLocateFollowingInitialDelayMs(), 400);
+      assertEquals(defaultInitialDelay, client.getConf().
+          getBlockWriteLocateFollowingInitialDelayMs());
+      assertEquals(defaultMaxDelay, client.getConf().
+          getBlockWriteLocateFollowingMaxDelayMs());
+      assertEquals(defultRetry, client.getConf().
+          getNumBlockWriteLocateFollowingRetry());
 
-      // change HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_KEY,
-      // verify DFSClient uses the configured value 1000.
       conf.setInt(
           HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY,
-          1000);
+          initialDelayTestValue);
+      conf.setInt(
+          HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_MAX_DELAY_MS_KEY,
+          maxDelayTestValue);
+      conf.setInt(
+          HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
+          retryTestValue);
       client = new DFSClient(null, nn, conf, null);
-      assertEquals(client.getConf().
-          getBlockWriteLocateFollowingInitialDelayMs(), 1000);
+      assertEquals(initialDelayTestValue, client.getConf().
+          getBlockWriteLocateFollowingInitialDelayMs());
+      assertEquals(maxDelayTestValue, client.getConf().
+          getBlockWriteLocateFollowingMaxDelayMs());
+      assertEquals(retryTestValue, client.getConf().
+          getNumBlockWriteLocateFollowingRetry());
     } finally {
       cluster.shutdown();
     }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java

@@ -43,6 +43,7 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
     configurationClasses = new Class[] { HdfsClientConfigKeys.class,
         HdfsClientConfigKeys.Failover.class,
         HdfsClientConfigKeys.StripedRead.class, DFSConfigKeys.class,
+        HdfsClientConfigKeys.BlockWrite.class,
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.class };
 
     // Set error modes