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

Todd Lipcon commented on HBASE-5974:
------------------------------------

bq. In the scenario Todd described, can the server keep the most recent batch 
of results so that when server sees cookie from client carrying 1 less than 
expected value, the server can resend the stored batch ?

I think it's too expensive in terms of memory consumption on the server. My 
assertion is that this issue is very rare (or else people probably would have 
noticed the missing rows a lot more often). So, keeping the last response batch 
cached server side seems a poor use of RAM.
                
> Scanner retry behavior with RPC timeout on next() seems incorrect
> -----------------------------------------------------------------
>
>                 Key: HBASE-5974
>                 URL: https://issues.apache.org/jira/browse/HBASE-5974
>             Project: HBase
>          Issue Type: Bug
>          Components: client, regionserver
>    Affects Versions: 0.90.7, 0.92.1, 0.94.0, 0.96.0
>            Reporter: Todd Lipcon
>            Priority: Critical
>
> I'm seeing the following behavior:
> - set RPC timeout to a short value
> - call next() for some batch of rows, big enough so the client times out 
> before the result is returned
> - the HConnectionManager stuff will retry the next() call to the same server. 
> At this point, one of two things can happen: 1) the previous next() call will 
> still be processing, in which case you get a LeaseException, because it was 
> removed from the map during the processing, or 2) the next() call will 
> succeed but skip the prior batch of rows.

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

        

Reply via email to