HBASE-23238 Additional test and checks for null references on ScannerCallableWithReplicas (#780)

Signed-off-by: Sean Busbey <busbey@apache.org>
This commit is contained in:
Wellington Ramos Chevreuil 2019-10-31 17:11:20 +00:00 committed by GitHub
parent 4bcc397f3e
commit 577db5d7e5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 84 additions and 1 deletions

View File

@ -94,7 +94,12 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
}
public void 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

View File

@ -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();
}
}
}