[ 
https://issues.apache.org/jira/browse/HBASE-2445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12856772#action_12856772
 ] 

Todd Lipcon commented on HBASE-2445:
------------------------------------

One particular example:

when processing a batch put, HConnectionManager uses the same retry count for 
the outer loop (number of batches to attempt) and the inner loop (number of 
times to retry an individual region server). For each region server, it treats 
socket layer exceptions and application layer exceptions the same with regard 
to retries.

This is not ideal: if I kill a region server while running an import, I find 
one of these two things happens:
- if I leave the number of retries configured at the default, the "outer loop" 
runs out of retries before all of the regions have been reassigned, thus the 
multiput fails
- if I configure the number of retries to 80 (meaning 80 seconds at the default 
sleep time of 1sec for most operations) then I actually end up retrying the 
same RS for 80 seconds without even refreshing the locations (the inner loop 
doesn't refresh).

I would like the fine grained configuration to be able to say:
- never retry a "connection refused" or "no route to host" error in the inner 
loop - I'd rather go back to meta to see if it's been reassigned
- in this particular case, same if I get NotServingRegionException - no sense 
retrying!
- for other errors, it may be worth one or two retries [not sure what errors 
those might be!]

For the outer loop I'd really like enough retries to wait around for at least 
90 seconds, to give the master time to notice the dead RS and reassign the 
regions.

This is just one example, but there are other places where being able to 
specify a more complete retry policy would help.

> Clean up client retry policies
> ------------------------------
>
>                 Key: HBASE-2445
>                 URL: https://issues.apache.org/jira/browse/HBASE-2445
>             Project: Hadoop HBase
>          Issue Type: Improvement
>          Components: client
>            Reporter: Todd Lipcon
>            Assignee: Todd Lipcon
>
> Right now almost all retry behavior is governed by a single parameter that 
> determines the number of retries. In a few places, there are also conf for 
> the number of millis to sleep between retries. This isn't quite flexible 
> enough. If we can refactor some of the retry logic into a RetryPolicy class, 
> we could introduce exponential backoff where appropriate, clean up some of 
> the config, etc

-- 
This message is automatically generated by JIRA.
-
If you think it was sent incorrectly contact one of the administrators: 
https://issues.apache.org/jira/secure/Administrators.jspa
-
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to