You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/08/06 11:44:20 UTC

[GitHub] [hadoop] bbeaudreault commented on a change in pull request #3271: HDFS-16155: Allow configurable exponential backoff in DFSInputStream refetchLocations

bbeaudreault commented on a change in pull request #3271:
URL: https://github.com/apache/hadoop/pull/3271#discussion_r684170071



##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
##########
@@ -994,4 +983,93 @@ public String confAsString() {
           + domainSocketDisableIntervalSeconds;
     }
   }
+
+  /**
+   * Handles calculating the wait time when BlockMissingException is caught.
+   */
+  public static class FetchBlockLocationsRetryer {
+    private final int maxBlockAcquireFailures;
+    private final int timeWindowBase;
+    private final int timeWindowMultiplier;
+    private final int timeWindowMax;
+    private final boolean enableRandom;
+
+    public FetchBlockLocationsRetryer(Configuration conf) {
+      this(conf, true);
+    }
+
+    /**
+     * It helps for testing to be able to disable the random factor. It should remain
+     * enabled for non-test use
+     */
+    @VisibleForTesting
+    FetchBlockLocationsRetryer(Configuration conf, boolean enableRandom) {
+      maxBlockAcquireFailures = conf.getInt(
+          DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+          DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
+      timeWindowBase = conf.getInt(
+          Retry.WINDOW_BASE_KEY,
+          Retry.WINDOW_BASE_DEFAULT);
+      timeWindowMultiplier = conf.getInt(
+          Retry.WINDOW_MULTIPLIER_KEY,
+          Retry.WINDOW_MULTIPLIER_DEFAULT);
+      timeWindowMax = conf.getInt(
+          Retry.WINDOW_MAXIMUM_KEY,
+          Retry.WINDOW_MAXIMUM_DEFAULT
+      );
+      this.enableRandom = enableRandom;
+    }
+
+    /**
+     * For tests, exposes the maximum allowed failures
+     */
+    @VisibleForTesting
+    public int getMaxBlockAcquireFailures() {
+      return maxBlockAcquireFailures;
+    }
+
+    /**
+     * Returns whether the passed number of failures is greater or equal to the maximum
+     * allowed failures.
+     */
+    public boolean isMaxFailuresExceeded(int numFailures) {
+      return numFailures >= maxBlockAcquireFailures;
+    }
+
+    /**
+     * The wait time is calculated using a grace period, a time window, and a random factor
+     * applied to that time window. With each subsequent failure, the grace period expands
+     * to the maximum value of the previous time window, and the time window upper limit expands
+     * by a constant exponential multiplier. The first retry has a grace period of 0ms.
+     *
+     * With default settings, the first failure will result in a wait time of a random number
+     * between 0 and 3000ms. The second failure will have a grace period of 3000ms, and an
+     * additional wait time of a random number between 0 and 6000ms. Subsequent failures will
+     * expand to 6000ms grace period and 0 - 9000ms, then 9000ms grace and 0 - 12000ms, etc.
+     *
+     * This behavior can be made more and less aggressive by configuring the base value (default 3000ms)
+     * and constant exponential multiplier (default 1). For example, a base of 10 and multiplier 5 could
+     * result in one very fast retry that quickly backs off in case of multiple failures. This may be useful
+     * for low latency applications. One downside with high multipliers is how quickly the backoff can get
+     * to very high numbers. One can further customize this by setting a maximum window size to cap
+     */
+    public double getWaitTime(int numFailures) {
+      double gracePeriod = backoff(numFailures);
+      double waitTimeWithRandomFactor = backoff(numFailures + 1) * getRandomFactor();
+
+      return gracePeriod + waitTimeWithRandomFactor;
+    }
+
+    private double backoff(int failures) {
+      return Math.min(timeWindowBase * Math.pow(timeWindowMultiplier, failures) * failures, timeWindowMax);

Review comment:
       If i were adding this feature as brand new, I probably wouldn't include the `* failures` here -- the base and exponential are good enough IMO. But I needed this to maintain 100% parity with the existing backoff strategy

##########
File path: hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
##########
@@ -994,4 +983,93 @@ public String confAsString() {
           + domainSocketDisableIntervalSeconds;
     }
   }
+
+  /**
+   * Handles calculating the wait time when BlockMissingException is caught.
+   */
+  public static class FetchBlockLocationsRetryer {
+    private final int maxBlockAcquireFailures;
+    private final int timeWindowBase;
+    private final int timeWindowMultiplier;
+    private final int timeWindowMax;
+    private final boolean enableRandom;
+
+    public FetchBlockLocationsRetryer(Configuration conf) {
+      this(conf, true);
+    }
+
+    /**
+     * It helps for testing to be able to disable the random factor. It should remain
+     * enabled for non-test use
+     */
+    @VisibleForTesting
+    FetchBlockLocationsRetryer(Configuration conf, boolean enableRandom) {
+      maxBlockAcquireFailures = conf.getInt(
+          DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+          DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
+      timeWindowBase = conf.getInt(
+          Retry.WINDOW_BASE_KEY,
+          Retry.WINDOW_BASE_DEFAULT);
+      timeWindowMultiplier = conf.getInt(
+          Retry.WINDOW_MULTIPLIER_KEY,
+          Retry.WINDOW_MULTIPLIER_DEFAULT);
+      timeWindowMax = conf.getInt(
+          Retry.WINDOW_MAXIMUM_KEY,
+          Retry.WINDOW_MAXIMUM_DEFAULT
+      );
+      this.enableRandom = enableRandom;
+    }
+
+    /**
+     * For tests, exposes the maximum allowed failures
+     */
+    @VisibleForTesting
+    public int getMaxBlockAcquireFailures() {
+      return maxBlockAcquireFailures;
+    }
+
+    /**
+     * Returns whether the passed number of failures is greater or equal to the maximum
+     * allowed failures.
+     */
+    public boolean isMaxFailuresExceeded(int numFailures) {
+      return numFailures >= maxBlockAcquireFailures;
+    }
+
+    /**
+     * The wait time is calculated using a grace period, a time window, and a random factor
+     * applied to that time window. With each subsequent failure, the grace period expands
+     * to the maximum value of the previous time window, and the time window upper limit expands
+     * by a constant exponential multiplier. The first retry has a grace period of 0ms.
+     *
+     * With default settings, the first failure will result in a wait time of a random number
+     * between 0 and 3000ms. The second failure will have a grace period of 3000ms, and an
+     * additional wait time of a random number between 0 and 6000ms. Subsequent failures will
+     * expand to 6000ms grace period and 0 - 9000ms, then 9000ms grace and 0 - 12000ms, etc.
+     *
+     * This behavior can be made more and less aggressive by configuring the base value (default 3000ms)
+     * and constant exponential multiplier (default 1). For example, a base of 10 and multiplier 5 could
+     * result in one very fast retry that quickly backs off in case of multiple failures. This may be useful
+     * for low latency applications. One downside with high multipliers is how quickly the backoff can get
+     * to very high numbers. One can further customize this by setting a maximum window size to cap
+     */
+    public double getWaitTime(int numFailures) {
+      double gracePeriod = backoff(numFailures);
+      double waitTimeWithRandomFactor = backoff(numFailures + 1) * getRandomFactor();
+
+      return gracePeriod + waitTimeWithRandomFactor;
+    }
+
+    private double backoff(int failures) {
+      return Math.min(timeWindowBase * Math.pow(timeWindowMultiplier, failures) * failures, timeWindowMax);

Review comment:
       Note: If i were adding this feature as brand new, I probably wouldn't include the `* failures` here -- the base and exponential are good enough IMO. But I needed this to maintain 100% parity with the existing backoff strategy




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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