Repository: hbase Updated Branches: refs/heads/master 05ab41d1b -> 63bd8be57
HBASE-17403 ClientAsyncPrefetchScanner doesnât load any data if the MaxResultSize is too small (ChiaPing Tsai) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/63bd8be5 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/63bd8be5 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/63bd8be5 Branch: refs/heads/master Commit: 63bd8be576b02981afda4cc491f6de89253e9ca9 Parents: 05ab41d Author: tedyu <[email protected]> Authored: Wed Jan 4 08:21:25 2017 -0800 Committer: tedyu <[email protected]> Committed: Wed Jan 4 08:21:25 2017 -0800 ---------------------------------------------------------------------- .../hadoop/hbase/client/ClientAsyncPrefetchScanner.java | 6 +++--- .../hbase/TestServerSideScanMetricsFromClientSide.java | 12 +++++++++++- 2 files changed, 14 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/63bd8be5/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 6b70a88..ee323a9 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 @@ -186,7 +186,7 @@ public class ClientAsyncPrefetchScanner extends ClientScanner { capacity = DEFAULT_QUEUE_CAPACITY; } } - return capacity; + return Math.max(capacity, 1); } private boolean prefetchCondition() { @@ -197,11 +197,11 @@ public class ClientAsyncPrefetchScanner extends ClientScanner { } private int getCountThreshold() { - return cacheCapacity / 2 ; + return Math.max(cacheCapacity / 2, 1); } private long getSizeThreshold() { - return maxScannerResultSize / 2 ; + return Math.max(maxScannerResultSize / 2, 1); } private long getCacheSizeInBytes() { http://git-wip-us.apache.org/repos/asf/hbase/blob/63bd8be5/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java index 6f91515..b516cbb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java @@ -152,11 +152,21 @@ public class TestServerSideScanMetricsFromClientSide { } @Test - public void testRowsSeenMetric() throws Exception { + public void testRowsSeenMetricWithSync() throws Exception { + testRowsSeenMetric(false); + } + + @Test + public void testRowsSeenMetricWithAsync() throws Exception { + testRowsSeenMetric(true); + } + + private void testRowsSeenMetric(boolean async) throws Exception { // Base scan configuration Scan baseScan; baseScan = new Scan(); baseScan.setScanMetricsEnabled(true); + baseScan.setAsyncPrefetch(async); testRowsSeenMetric(baseScan); // Test case that only a single result will be returned per RPC to the serer
