HBASE-23238 Additional test and checks for null references on ScannerCallableWithReplicas (#780)
Signed-off-by: Sean Busbey <busbey@apache.org>
(cherry picked from commit 577db5d7e5
- Test only, removed changes from no more exisiting ScannerCallableWithReplicas class)
This commit is contained in:
parent
ea5c572963
commit
b8a4504a26
|
@ -44,6 +44,9 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
|||
import org.apache.hadoop.hbase.HTestConst;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
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.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
|
@ -52,7 +55,6 @@ import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
|
|||
import org.apache.hadoop.hbase.filter.QualifierFilter;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.junit.After;
|
||||
|
@ -798,7 +800,7 @@ public class TestScannersFromClientSide {
|
|||
final byte[] LARGE_VALUE = generateHugeValue(128 * 1024);
|
||||
|
||||
try (Table table = TEST_UTIL.createTable(tableName, FAMILY);
|
||||
Admin admin = TEST_UTIL.getAdmin()) {
|
||||
Admin admin = TEST_UTIL.getAdmin()) {
|
||||
List<Put> putList = new ArrayList<>();
|
||||
for (long i = 0; i < ROWS_TO_INSERT; i++) {
|
||||
Put put = new Put(Bytes.toBytes(i));
|
||||
|
@ -828,7 +830,79 @@ public class TestScannersFromClientSide {
|
|||
}
|
||||
}
|
||||
assertEquals("Expected " + ROWS_TO_INSERT + " rows in the table but it is " + count,
|
||||
ROWS_TO_INSERT, count);
|
||||
ROWS_TO_INSERT, count);
|
||||
}
|
||||
}
|
||||
|
||||
@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 filterCell(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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue