HBASE-13071 synchronous scanner -- cache size-in-bytes bug fix Signed-off-by: stack <[email protected]>
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/7f2b33db Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7f2b33db Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7f2b33db Branch: refs/heads/hbase-12439 Commit: 7f2b33dbbf90474a8f73e4d38ea8f6817ee3dcdb Parents: dc72dad Author: eshcar <[email protected]> Authored: Sun May 17 23:01:40 2015 +0300 Committer: stack <[email protected]> Committed: Wed May 20 12:35:26 2015 -0700 ---------------------------------------------------------------------- .../hbase/client/ClientAsyncPrefetchScanner.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/7f2b33db/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java index c4eb478..f0903db 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java @@ -104,10 +104,7 @@ public class ClientAsyncPrefetchScanner extends ClientScanner { while (isPrefetchRunning()) { // prefetch running or still pending if (getCacheCount() > 0) { - Result res = cache.poll(); - long estimatedSize = calcEstimatedSize(res); - addEstimatedSize(-estimatedSize); - return res; + return pollCache(); } else { // (busy) wait for a record - sleep Threads.sleep(1); @@ -115,7 +112,7 @@ public class ClientAsyncPrefetchScanner extends ClientScanner { } if (getCacheCount() > 0) { - return cache.poll(); + return pollCache(); } // if we exhausted this scanner before calling close, write out the scan metrics @@ -209,6 +206,12 @@ public class ClientAsyncPrefetchScanner extends ClientScanner { return cacheSizeInBytes.get(); } + private Result pollCache() { + Result res = cache.poll(); + long estimatedSize = calcEstimatedSize(res); + addEstimatedSize(-estimatedSize); + return res; + } private class PrefetchRunnable implements Runnable {
