[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_r696619390 ## File path: hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml ## @@ -4440,8 +4440,29 @@ 3000 Base time window in ms for DFSClient retries. For each retry attempt, -this value is extended linearly (e.g. 3000 ms for first attempt and -first retry, 6000 ms for second retry, 9000 ms for third retry, etc.). +this value is extended exponentially based on dfs.client.retry.window.multiplier. + + + + + dfs.client.retry.window.multiplier + 1 + +Multiplier for extending the retry time window. For each retry attempt, +the retry time window is extended by multiplying dfs.client.retry.window.base +by this multiplier raised to the power of the current failure count. The default +value of 1 means the window will expand linearly (e.g. 3000 ms for first attempt +and first retry, 6000 ms for second retry, 9000 ms for third retry, etc.). + + + + + dfs.client.retry.window.max + 2147483647 Review comment: @Hexiaoqiao I've pushed a commit which lowers the window max to 30s. As mentioned above, this may cap some custom backoffs people have configured. But that may be beneficial. It should not affect the default case, given the default of 3 retries does not reach 30s. Let me know if you'd prefer a different default. -- 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-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[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_r695678157 ## File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestFetchBlockLocationsRetryer.java ## @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.client.impl; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry; +import org.apache.hadoop.hdfs.client.impl.DfsClientConf.FetchBlockLocationsRetryer; +import org.junit.Test; + +public class TestFetchBlockLocationsRetryer { + + private static final double EPSILON = 0.001; + + @Test + public void testIsMaxFailuresExceeded() { +Configuration conf = new Configuration(); + + conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 3); +FetchBlockLocationsRetryer retryer = new FetchBlockLocationsRetryer(conf); + +assertFalse(retryer.isMaxFailuresExceeded(1)); +assertTrue(retryer.isMaxFailuresExceeded(3)); +assertTrue(retryer.isMaxFailuresExceeded(5)); + } + + @Test + public void testDefaultRetryPolicy() { Review comment: Per the comment in the original backoff policy: > // Introducing a random factor to the wait time before another retry. > // The wait time is dependent on # of failures and a random factor. > // At the first time of getting a BlockMissingException, the wait time > // is a random number between 0..3000 ms. If the first retry > // still fails, we will wait 3000 ms grace period before the 2nd retry. > // Also at the second retry, the waiting window is expanded to 6000 ms > // alleviating the request rate from the server. Similarly the 3rd retry > // will wait 6000ms grace period before retry and the waiting window is > // expanded to 9000ms. - The first backoff should be between 0-3000ms. - The second should be 3000 plus a random number between 0-6000ms. So the full range is 3000-9000. - The third retry should be 6000 plus a random number between 0-9000ms. So the full range is 6000-15000ms. This test proves that this original retry strategy continues to work with the new code. It's hard to test with randomness, so the random factor is disabled. We're left with only the worst case scenario (if `rand()` returned 1). See the assertions below to see that the results adhere to the original description above. -- 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-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[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_r695678157 ## File path: hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestFetchBlockLocationsRetryer.java ## @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.client.impl; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry; +import org.apache.hadoop.hdfs.client.impl.DfsClientConf.FetchBlockLocationsRetryer; +import org.junit.Test; + +public class TestFetchBlockLocationsRetryer { + + private static final double EPSILON = 0.001; + + @Test + public void testIsMaxFailuresExceeded() { +Configuration conf = new Configuration(); + + conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY, 3); +FetchBlockLocationsRetryer retryer = new FetchBlockLocationsRetryer(conf); + +assertFalse(retryer.isMaxFailuresExceeded(1)); +assertTrue(retryer.isMaxFailuresExceeded(3)); +assertTrue(retryer.isMaxFailuresExceeded(5)); + } + + @Test + public void testDefaultRetryPolicy() { Review comment: Per the comment in the original backoff policy: > // Introducing a random factor to the wait time before another retry. > // The wait time is dependent on # of failures and a random factor. > // At the first time of getting a BlockMissingException, the wait time > // is a random number between 0..3000 ms. If the first retry > // still fails, we will wait 3000 ms grace period before the 2nd retry. > // Also at the second retry, the waiting window is expanded to 6000 ms > // alleviating the request rate from the server. Similarly the 3rd retry > // will wait 6000ms grace period before retry and the waiting window is > // expanded to 9000ms. The first backoff should be between 0-3000ms. The second should be between 3000ms-9000ms (3000 + 6000). The third retry should be between 6000-15000ms (6000 + 9000). The random factor is applied to the 2nd part of that. So the second retry is `3000 + rand() * 6000`, etc. This test proves that this original retry strategy continues to work with the new code. It's hard to test with randomness, so the random factor is disabled. We're left with only the worst case scenario (if `rand()` returned 1). See the assertions below to see that the results adhere to the original description above. -- 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-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[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_r695662268 ## File path: hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml ## @@ -4440,8 +4440,29 @@ 3000 Base time window in ms for DFSClient retries. For each retry attempt, -this value is extended linearly (e.g. 3000 ms for first attempt and -first retry, 6000 ms for second retry, 9000 ms for third retry, etc.). +this value is extended exponentially based on dfs.client.retry.window.multiplier. + + + + + dfs.client.retry.window.multiplier + 1 + +Multiplier for extending the retry time window. For each retry attempt, +the retry time window is extended by multiplying dfs.client.retry.window.base +by this multiplier raised to the power of the current failure count. The default +value of 1 means the window will expand linearly (e.g. 3000 ms for first attempt +and first retry, 6000 ms for second retry, 9000 ms for third retry, etc.). + + + + + dfs.client.retry.window.max + 2147483647 Review comment: Thanks so much for the review @Hexiaoqiao. The reason I chose this value was that I wanted the changes in this PR to be totally transparent to existing users -- so the backoff should work exactly as it does today for anyone who upgrades. I don't know how people have tuned their backoffs today, so adding a lower max might affect their configured backoffs. The default case will be well-bounded by the default retries of 3. That said, I agree that there's very little utility in waiting many minutes on a backoff. What if I put this to 30s? Was this the only concern in terms of the default action? My test case [testDefaultRetryPolicy](https://github.com/apache/hadoop/pull/3271/files#diff-ce7234e2a352097945c5348774888466b48f9574d4764d9415559d975d380884R47) proves that the default case remains unchanged from trunk. The default case was determined based on the [comment in DFSInputStream](https://github.com/apache/hadoop/pull/3271/files#diff-f6cd7fbb9a909e1679c2d77df54b0b2fe661a66c8c89fb455fde313966d51a61L1018-L1026), the old implementation details, and my own testing of the backoff policy prior to this change. I also created this spreadsheet that helped me to determine how different multiplier values might affect the backoff: https://docs.google.com/spreadsheets/d/1I9ejqDtJ6-krSh-YBt0qHTf3JwZu5zRlrOhbzY0kJAg/edit?usp=sharing -- 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-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[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