[
https://issues.apache.org/jira/browse/HDFS-16155?focusedWorklogId=635084&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-635084
]
ASF GitHub Bot logged work on HDFS-16155:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 06/Aug/21 11:44
Start Date: 06/Aug/21 11:44
Worklog Time Spent: 10m
Work Description: 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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 635084)
Time Spent: 40m (was: 0.5h)
> Allow configurable exponential backoff in DFSInputStream refetchLocations
> -------------------------------------------------------------------------
>
> Key: HDFS-16155
> URL: https://issues.apache.org/jira/browse/HDFS-16155
> Project: Hadoop HDFS
> Issue Type: Improvement
> Components: dfsclient
> Reporter: Bryan Beaudreault
> Priority: Minor
> Labels: pull-request-available
> Time Spent: 40m
> Remaining Estimate: 0h
>
> The retry policy in
> [DFSInputStream#refetchLocations|https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L1018-L1040]
> was first written many years ago. It allows configuration of the base time
> window, but subsequent retries double in an un-configurable way. This retry
> strategy makes sense in some clusters as it's very conservative and will
> avoid DDOSing the namenode in certain systemic failure modes – for example,
> if a file is being read by a large hadoop job and the underlying blocks are
> moved by the balancer. In this case, enough datanodes would be added to the
> deadNodes list and all hadoop tasks would simultaneously try to refetch the
> blocks. The 3s doubling with random factor helps break up that stampeding
> herd.
> However, not all cluster use-cases are created equal, so there are other
> cases where a more aggressive initial backoff is preferred. For example in a
> low-latency single reader scenario. In this case, if the balancer moves
> enough blocks, the reader hits this 3s backoff which is way too long for a
> low latency use-case.
> One could configure the the window very low (10ms), but then you can hit
> other systemic failure modes which would result in readers DDOSing the
> namenode again. For example, if blocks went missing due to truly dead
> datanodes. In this case, many readers might be refetching locations for
> different files with retry backoffs like 10ms, 20ms, 40ms, etc. It takes a
> while to backoff enough to avoid impacting the namenode with that strategy.
> I suggest adding a configurable multiplier to the backoff strategy so that
> operators can tune this as they see fit for their use-case. In the above low
> latency case, one could set the base very low (say 2ms) and the multiplier
> very high (say 50). This gives an aggressive first retry that very quickly
> backs off.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]