HBASE-8555 FilterList correctness may be affected by random ordering of sub-filter(list) (Liang Xie)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1485308 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
15d7ae0bc6
commit
8bd59042ba
|
@ -150,20 +150,21 @@ public class FilterList extends Filter {
|
|||
|
||||
@Override
|
||||
public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
|
||||
boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
|
||||
for (Filter filter : filters) {
|
||||
if (this.operator == Operator.MUST_PASS_ALL) {
|
||||
if (filter.filterAllRemaining() ||
|
||||
filter.filterRowKey(rowKey, offset, length)) {
|
||||
return true;
|
||||
flag = true;
|
||||
}
|
||||
} else if (this.operator == Operator.MUST_PASS_ONE) {
|
||||
if (!filter.filterAllRemaining() &&
|
||||
!filter.filterRowKey(rowKey, offset, length)) {
|
||||
return false;
|
||||
flag = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
return this.operator == Operator.MUST_PASS_ONE;
|
||||
return flag;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1768,6 +1768,97 @@ public class TestFilter {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public void testNestedFilterListWithSCVF() throws IOException {
|
||||
byte[] columnStatus = Bytes.toBytes("S");
|
||||
HTableDescriptor htd = new HTableDescriptor("testNestedFilterListWithSCVF");
|
||||
htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
|
||||
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
|
||||
HRegion testRegion = HRegion.createHRegion(info, TEST_UTIL.getDataTestDir(),
|
||||
TEST_UTIL.getConfiguration(), htd);
|
||||
for(int i=0; i<10; i++) {
|
||||
Put p = new Put(Bytes.toBytes("row" + i));
|
||||
p.setDurability(Durability.SKIP_WAL);
|
||||
p.add(FAMILIES[0], columnStatus, Bytes.toBytes(i%2));
|
||||
testRegion.put(p);
|
||||
}
|
||||
testRegion.flushcache();
|
||||
// 1. got rows > "row4"
|
||||
Filter rowFilter = new RowFilter(CompareOp.GREATER,new BinaryComparator(Bytes.toBytes("row4")));
|
||||
Scan s1 = new Scan();
|
||||
s1.setFilter(rowFilter);
|
||||
InternalScanner scanner = testRegion.getScanner(s1);
|
||||
List<KeyValue> results = new ArrayList<KeyValue>();
|
||||
int i = 5;
|
||||
for (boolean done = true; done; i++) {
|
||||
done = scanner.next(results);
|
||||
assertTrue(Bytes.equals(results.get(0).getRow(), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(results.get(0).getValue()), i%2);
|
||||
results.clear();
|
||||
}
|
||||
// 2. got rows <= "row4" and S=
|
||||
FilterList subFilterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
|
||||
Filter subFilter1 = new RowFilter(CompareOp.LESS_OR_EQUAL,
|
||||
new BinaryComparator(Bytes.toBytes("row4")));
|
||||
subFilterList.addFilter(subFilter1);
|
||||
Filter subFilter2 = new SingleColumnValueFilter(FAMILIES[0], columnStatus, CompareOp.EQUAL,
|
||||
Bytes.toBytes(0));
|
||||
subFilterList.addFilter(subFilter2);
|
||||
s1 = new Scan();
|
||||
s1.setFilter(subFilterList);
|
||||
scanner = testRegion.getScanner(s1);
|
||||
results = new ArrayList<KeyValue>();
|
||||
for (i=0; i<=4; i+=2) {
|
||||
scanner.next(results);
|
||||
assertTrue(Bytes.equals(results.get(0).getRow(), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(results.get(0).getValue()), i%2);
|
||||
results.clear();
|
||||
}
|
||||
assertFalse(scanner.next(results));
|
||||
// 3. let's begin to verify nested filter list
|
||||
// 3.1 add rowFilter, then add subFilterList
|
||||
FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
|
||||
filterList.addFilter(rowFilter);
|
||||
filterList.addFilter(subFilterList);
|
||||
s1 = new Scan();
|
||||
s1.setFilter(filterList);
|
||||
scanner = testRegion.getScanner(s1);
|
||||
results = new ArrayList<KeyValue>();
|
||||
for (i=0; i<=4; i+=2) {
|
||||
scanner.next(results);
|
||||
assertTrue(Bytes.equals(results.get(0).getRow(), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(results.get(0).getValue()), i%2);
|
||||
results.clear();
|
||||
}
|
||||
for (i=5; i<=9; i++) {
|
||||
scanner.next(results);
|
||||
assertTrue(Bytes.equals(results.get(0).getRow(), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(results.get(0).getValue()), i%2);
|
||||
results.clear();
|
||||
}
|
||||
assertFalse(scanner.next(results));
|
||||
// 3.2 MAGIC here! add subFilterList first, then add rowFilter
|
||||
filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
|
||||
filterList.addFilter(subFilterList);
|
||||
filterList.addFilter(rowFilter);
|
||||
s1 = new Scan();
|
||||
s1.setFilter(filterList);
|
||||
scanner = testRegion.getScanner(s1);
|
||||
results = new ArrayList<KeyValue>();
|
||||
for (i=0; i<=4; i+=2) {
|
||||
scanner.next(results);
|
||||
assertTrue(Bytes.equals(results.get(0).getRow(), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(results.get(0).getValue()), i%2);
|
||||
results.clear();
|
||||
}
|
||||
for (i=5; i<=9; i++) {
|
||||
scanner.next(results);
|
||||
assertTrue(Bytes.equals(results.get(0).getRow(), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(results.get(0).getValue()), i%2);
|
||||
results.clear();
|
||||
}
|
||||
assertFalse(scanner.next(results));
|
||||
HLog hlog = testRegion.getLog();
|
||||
testRegion.close();
|
||||
hlog.closeAndDelete();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -115,33 +115,32 @@ public class TestFilterList {
|
|||
byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
|
||||
for (int i = 0; i < MAX_PAGES - 1; i++) {
|
||||
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
|
||||
assertFalse(filterMPONE.filterRow());
|
||||
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
|
||||
Bytes.toBytes(i));
|
||||
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
|
||||
assertFalse(filterMPONE.filterRow());
|
||||
}
|
||||
|
||||
/* Only pass PageFilter */
|
||||
rowkey = Bytes.toBytes("z");
|
||||
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
|
||||
assertFalse(filterMPONE.filterRow());
|
||||
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
|
||||
Bytes.toBytes(0));
|
||||
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
|
||||
|
||||
/* PageFilter will fail now, but should pass because we match yyy */
|
||||
rowkey = Bytes.toBytes("yyy");
|
||||
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
|
||||
assertFalse(filterMPONE.filterRow());
|
||||
|
||||
/* reach MAX_PAGES already, should filter any rows */
|
||||
rowkey = Bytes.toBytes("yyy");
|
||||
assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
|
||||
kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
|
||||
Bytes.toBytes(0));
|
||||
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
|
||||
assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
|
||||
assertFalse(filterMPONE.filterRow());
|
||||
|
||||
/* We should filter any row */
|
||||
rowkey = Bytes.toBytes("z");
|
||||
assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
|
||||
assertTrue(filterMPONE.filterAllRemaining());
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue