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

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

                Author: ASF GitHub Bot
            Created on: 05/Aug/21 18:03
            Start Date: 05/Aug/21 18:03
    Worklog Time Spent: 10m 
      Work Description: bbeaudreault opened a new pull request #3271:
URL: https://github.com/apache/hadoop/pull/3271


   Per https://issues.apache.org/jira/browse/HDFS-16155, we would like the 
ability to customize the backoff strategy when BlockMissingException occurs. 
This can happen when the balancer moves blocks, and in low latency clusters the 
existing backoff is too conservative. Drastically reducing the existing window 
base config would help but expose the namenode to a potential DDOS if many 
blocks became missing, because the current backoff would grow slowly.
   
   Adding a configurable exponential component allows for aggressive early 
retries that back off quickly enough to mitigate stampeding herds. We make the 
backoff configurable by adding two new configs:
   
   - `dfs.client.retry.window.multiplier`: defaults to 1 to preserve existing 
behavior. Increasing this can result in a steeper backoff curve when desired
   - `dfs.client.retry.window.max`: defaults to Int.MAX to preserve existing 
behavior. Decreasing this can help put a ceiling on exponential backoffs that 
could quickly grow to effectively unlimited levels.
   
   As described, the default behavior is maintained and I've added a test case 
to verify that. Someone looking for a more aggressive initial retry that backs 
off quickly in case of continuous failure could try setting `window.base` to 
10, `window.multiplier` to 5, and `window.max` to 10000. This would result in a 
quick initial retry of max 50ms, but quickly backoff to a few seconds within 3 
retries.
   
   In order to improve the testability of this feature, I pulled out the 
existing refetchLocations retry configs into a FetchBlockLocationsRetryer 
class. I also improved the readability of the comment describing the backoff 
strategy, and fully tested the new retryer in TestFetchBlockLocationsRetryer.


-- 
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: 634701)
    Remaining Estimate: 0h
            Time Spent: 10m

> 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
>          Time Spent: 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