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

Yu Li commented on HBASE-14521:
-------------------------------

Thanks for the comments Nicolas.

{quote}
We should continue to count the calls, no?
{quote}
>From codes in AsyncProcess, after getting AsyncRequestFutureImpl instance from 
>createAsyncRequestFuture call, the code flow is like:
{noformat}
AsyncRequestFutureImpl#sendMultiAction->SingleServerRequestRunnable#run
{noformat}
where it will createCaller first and invoke callWithoutRetries to issue the 
real call:
{code}
        try {
          callable = createCallable(server, tableName, multiAction);
          try {
            RpcRetryingCaller<MultiResponse> caller = createCaller(callable);
            if (callsInProgress != null) callsInProgress.add(callable);
            res = caller.callWithoutRetries(callable, timeout);
{code}
Actually the ideal way is to increase the callsCt inside callWithoutRetries, 
but it's also ok to do the increment in createCaller, just as we do in 
TestAsyncProcess.
If the above analysis is correct (plz correct me if I stated anything wrong), 
then the count of calls in createAsyncRequestFuture is redundant. I'd also 
regard it as a trick to work around the inconsistency of 
hbase.client.retries.number before.

{quote}
Note that setting retries to zero is most of the time an error as we can have a 
retry in many cases, for example iif the client cache is not up to date 
(contains the wrong region server for a region).
{quote}
Agree that retry is necessary for client cache update, and we shouldn't set 
retries to zero under most common cases, but in some corner cases we may still 
don't want any retry, just like we did in some of the UT cases. Let me add a 
line in the release note.

> Unify the semantic of hbase.client.retries.number
> -------------------------------------------------
>
>                 Key: HBASE-14521
>                 URL: https://issues.apache.org/jira/browse/HBASE-14521
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.98.14, 1.1.2
>            Reporter: Yu Li
>            Assignee: Yu Li
>             Fix For: 2.0.0, 1.3.0
>
>         Attachments: HBASE-14521.patch, HBASE-14521_v2.patch, 
> HBASE-14521_v3.patch
>
>
> From name of the _hbase.client.retries.number_ property, it should be the 
> number of maximum *retries*, or say if we set the property to 1, there should 
> be 2 attempts in total. However, there're two different semantics when using 
> it in current code base.
> For example, in ConnectionImplementation#locateRegionInMeta:
> {code}
>     int localNumRetries = (retry ? numTries : 1);
>     for (int tries = 0; true; tries++) {
>       if (tries >= localNumRetries) {
>         throw new NoServerForRegionException("Unable to find region for "
>             + Bytes.toStringBinary(row) + " in " + tableName +
>             " after " + numTries + " tries.");
>       }
> {code}
> the retries number is regarded as max times for *tries*
> While in RpcRetryingCallerImpl#callWithRetries:
> {code}
>     for (int tries = 0;; tries++) {
>       long expectedSleep;
>       try {
>         callable.prepare(tries != 0); // if called with false, check table 
> status on ZK
>         interceptor.intercept(context.prepare(callable, tries));
>         return callable.call(getRemainingTime(callTimeout));
>       } catch (PreemptiveFastFailException e) {
>         throw e;
>       } catch (Throwable t) {
>         ...
>         if (tries >= retries - 1) {
>           throw new RetriesExhaustedException(tries, exceptions);
>         }
> {code}
> it's regarded as exactly for *REtry* (try a call first with no condition and 
> then check whether to retry or exceeds maximum retry number)
> This inconsistency will cause misunderstanding in usage, such as one of our 
> customer set the property to zero expecting one single call but finally 
> received NoServerForRegionException.
> We should unify the semantic of the property, and I suggest to keep the 
> original one for retry rather than total tries.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to