[
https://issues.apache.org/jira/browse/HBASE-11544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14308054#comment-14308054
]
Jonathan Lawlor commented on HBASE-11544:
-----------------------------------------
Thanks for the feedback [~lhofhansl].
Setting 2mb as the default maxResultSize definitely makes sense in the current
workflow where caching is set very high. In cases where the size limit is hit
before the caching limit this will almost be equivalent to how I was thinking
of implementing solution #1.
Certainly the move to a full streaming protocol (solution #2) would be the best
solution. However, the implementation of solution #1 seems like it would fix
the major pain point of running out of memory on the server and may even help
us improve the current paradigm of how the network is used.
In the move to solution #1 we could improve network usage by reworking the
semantics behind Scan#setCaching() (as [~lhofhansl] has called out in
HBASE-12976). Rather than looking to fetch a certain number of rows from an RPC
request we woud instead try to ensure that we always fill the maxResultSize in
the RPC response sent back (if the chunk cannot be filled, just send back what
we have). The rework of the caching could then instead change caching into more
of a row limit concept where by default the limit is Long.MAX. Then, in the
default case we could service RPC requests on the basis of Batch and
MaxResultSize alone. We would no longer need to see if we had accumulated
enough rows to meet the caching limit. Instead we would do the following:
- While the maxResultSize has not been reached, accumulate results. If
Scan#setBatch() is not set, each Result will represent a row. If
Scan#setBatch() is set, each Result will represent a group of cells from a row.
- The maxResultSize would be checked at the cell level rather than the row
level like it currently is. What this means is that rather than fetching a row
and seeing if we have met our maxResultSize cap, we would be fetching a cell
and seeing if we have hit our maxResultSize cap. This finer grain size check
means it will be possible for us to retrieve rows that would otherwise cause
OOME exceptions. Thus, the RPC response would be returning a list of Results
where the last Result will likely be a partial result (i.e. will not contain
all of the cells for its rows) since the maxResultSize limit would be triggered
after fetching a cell.
Then on the client side we can determine, maybe based on a new Scan setting,
whether or not the caller will be okay with receiving the results for a row in
parts (similar to batching). If the caller expects each Result to contain all
of the cells for a single row, then we can accumulate the partial results for a
row on the client side and then combine them into a single result before adding
them to our client side cache (note that as [~lhofhansl] has mentioned, this
presents the possibility that the client may OOME when reading large rows
because the entire row would need to be constructed client side).
In summary, what the implementation of solution #1 does for us is:
- Reduce the likelihood of OOME on the server. OOME may still occur if a single
cell is too large for the server to handle. This issue could be fixed with the
move to a full streaming protocol (solution #2)
- Allows the client to retrieve rows that they may otherwise be unable to
retrieve currently due to RowTooBigExceptions
- Provides a finer grained restriction on the ResultSize on the server -- Since
the limit on the result size is checked on a cell by cell basis, we will no
longer overshoot the result size limit by large amounts when the rows are large
- Addresses points #1 and #2 of [~lhofhansl]'s list above. #3 would be
addressed by an implementation of streaming
The implementation details would be as in my previous comment, but with the
addition of changing the semantics behind caching to act more as a limit on the
number of rows. Once again, any feedback is greatly appreciated.
Thanks
> [Ergonomics] hbase.client.scanner.caching is dogged and will try to return
> batch even if it means OOME
> ------------------------------------------------------------------------------------------------------
>
> Key: HBASE-11544
> URL: https://issues.apache.org/jira/browse/HBASE-11544
> Project: HBase
> Issue Type: Bug
> Reporter: stack
> Priority: Critical
> Labels: beginner
>
> Running some tests, I set hbase.client.scanner.caching=1000. Dataset has
> large cells. I kept OOME'ing.
> Serverside, we should measure how much we've accumulated and return to the
> client whatever we've gathered once we pass out a certain size threshold
> rather than keep accumulating till we OOME.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)