You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2019/10/31 17:11:30 UTC
[hbase] branch branch-1 updated: HBASE-23238 Additional test and
checks for null references on ScannerCallableWithReplicas (#780)
This is an automated email from the ASF dual-hosted git repository.
wchevreuil pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-1 by this push:
new 577db5d HBASE-23238 Additional test and checks for null references on ScannerCallableWithReplicas (#780)
577db5d is described below
commit 577db5d7e50c56b4773c9ce92b807aae80bf5706
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Thu Oct 31 17:11:20 2019 +0000
HBASE-23238 Additional test and checks for null references on ScannerCallableWithReplicas (#780)
Signed-off-by: Sean Busbey <bu...@apache.org>
---
.../hbase/client/ScannerCallableWithReplicas.java | 11 +++-
.../hbase/client/TestScannersFromClientSide.java | 74 ++++++++++++++++++++++
2 files changed, 84 insertions(+), 1 deletion(-)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index 10c20d7..50cfe1c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -94,7 +94,12 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
}
public void setClose() {
- currentScannerCallable.setClose();
+ if(currentScannerCallable != null) {
+ currentScannerCallable.setClose();
+ } else {
+ LOG.warn("Calling close on ScannerCallable reference that is already null, "
+ + "which shouldn't happen.");
+ }
}
public void setRenew(boolean val) {
@@ -136,6 +141,10 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
Result[] r = currentScannerCallable.call(timeout);
currentScannerCallable = null;
return r;
+ } else if(currentScannerCallable == null) {
+ LOG.warn("Another call received, but our ScannerCallable is already null. "
+ + "This shouldn't happen, but there's not much to do, so logging and returning null.");
+ return null;
}
// We need to do the following:
//1. When a scan goes out to a certain replica (default or not), we need to
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 4bac32d..eb2863f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -37,6 +37,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTestConst;
import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
@@ -737,4 +739,76 @@ public class TestScannersFromClientSide {
}
}
}
+
+ @Test
+ public void testScannerWithPartialResults() throws Exception {
+ TableName tableName = TableName.valueOf("testScannerWithPartialResults");
+ try (Table table = TEST_UTIL.createMultiRegionTable(tableName,
+ Bytes.toBytes("c"), 4)) {
+ List<Put> puts = new ArrayList<>();
+ byte[] largeArray = new byte[10000];
+ Put put = new Put(Bytes.toBytes("aaaa0"));
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), Bytes.toBytes("1"));
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("2"), Bytes.toBytes("2"));
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("3"), Bytes.toBytes("3"));
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("4"), Bytes.toBytes("4"));
+ puts.add(put);
+ put = new Put(Bytes.toBytes("aaaa1"));
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), Bytes.toBytes("1"));
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("2"), largeArray);
+ put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("3"), largeArray);
+ puts.add(put);
+ table.put(puts);
+ Scan scan = new Scan();
+ scan.addFamily(Bytes.toBytes("c"));
+ scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, tableName.getName());
+ scan.setMaxResultSize(10001);
+ scan.setStopRow(Bytes.toBytes("bbbb"));
+ scan.setFilter(new LimitKVsReturnFilter());
+ ResultScanner rs = table.getScanner(scan);
+ Result result;
+ int expectedKvNumber = 6;
+ int returnedKvNumber = 0;
+ while((result = rs.next()) != null){
+ returnedKvNumber += result.listCells().size();
+ }
+ rs.close();
+ assertEquals(expectedKvNumber, returnedKvNumber);
+ }
+ }
+
+ public static class LimitKVsReturnFilter extends FilterBase {
+
+ private static int total = 0;
+
+ @Override
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
+ if(total>=6) {
+ total++;
+ return ReturnCode.SKIP;
+ }
+ total++;
+ return ReturnCode.INCLUDE;
+ }
+
+ @Override
+ public boolean filterAllRemaining() throws IOException {
+ if(total<7) {
+ return false;
+ }
+ total++;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return this.getClass().getSimpleName();
+ }
+
+ public static LimitKVsReturnFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ return new LimitKVsReturnFilter();
+ }
+ }
+
}