[ 
https://issues.apache.org/jira/browse/HDFS-16155?focusedWorklogId=642327&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-642327
 ]

ASF GitHub Bot logged work on HDFS-16155:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 26/Aug/21 13:17
            Start Date: 26/Aug/21 13:17
    Worklog Time Spent: 10m 
      Work Description: 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 @@
   <value>3000</value>
   <description>
     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.
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.retry.window.multiplier</name>
+  <value>1</value>
+  <description>
+    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.).
+  </description>
+</property>
+
+<property>
+  <name>dfs.client.retry.window.max</name>
+  <value>2147483647</value>

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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 642327)
    Time Spent: 2h 10m  (was: 2h)

> 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
>            Assignee: Bryan Beaudreault
>            Priority: Minor
>              Labels: pull-request-available
>          Time Spent: 2h 10m
>  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]

Reply via email to