HBASE-3550 FilterList reports false positives

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1072481 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Kyle Purtell 2011-02-20 02:50:48 +00:00
parent e3fa6497fe
commit 3564a25e85
3 changed files with 67 additions and 2 deletions

View File

@ -49,7 +49,8 @@ Release 0.91.0 - Unreleased
HBASE-3525 mvn assembly is over-filling the hbase lib dir HBASE-3525 mvn assembly is over-filling the hbase lib dir
HBASE-3531 When under global memstore pressure, dont try to flush HBASE-3531 When under global memstore pressure, dont try to flush
unflushable regions. unflushable regions.
HBASE-3550 FilterList reports false positives (Bill Graham via Andrew
Purtell)
IMPROVEMENTS IMPROVEMENTS
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack) HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)

View File

@ -168,12 +168,15 @@ public class FilterList implements Filter {
if (filter.filterAllRemaining()) { if (filter.filterAllRemaining()) {
return ReturnCode.NEXT_ROW; return ReturnCode.NEXT_ROW;
} }
switch (filter.filterKeyValue(v)) { ReturnCode code = filter.filterKeyValue(v);
switch (code) {
case INCLUDE: case INCLUDE:
continue; continue;
case NEXT_ROW: case NEXT_ROW:
case SKIP: case SKIP:
return ReturnCode.SKIP; return ReturnCode.SKIP;
default:
return code;
} }
} else if (operator == Operator.MUST_PASS_ONE) { } else if (operator == Operator.MUST_PASS_ONE) {
if (filter.filterAllRemaining()) { if (filter.filterAllRemaining()) {

View File

@ -74,7 +74,68 @@ public class TestColumnPrefixFilter {
scan.setMaxVersions(); scan.setMaxVersions();
for (String s: prefixMap.keySet()) { for (String s: prefixMap.keySet()) {
filter = new ColumnPrefixFilter(Bytes.toBytes(s)); filter = new ColumnPrefixFilter(Bytes.toBytes(s));
scan.setFilter(filter); scan.setFilter(filter);
InternalScanner scanner = region.getScanner(scan);
List<KeyValue> results = new ArrayList<KeyValue>();
while(scanner.next(results));
assertEquals(prefixMap.get(s).size(), results.size());
}
}
@Test
public void testColumnPrefixFilterWithFilterList() throws IOException {
String family = "Family";
HTableDescriptor htd = new HTableDescriptor("TestColumnPrefixFilter");
htd.addFamily(new HColumnDescriptor(family));
HRegionInfo info = new HRegionInfo(htd, null, null, false);
HRegion region = HRegion.createHRegion(info, HBaseTestingUtility.
getTestDir(), TEST_UTIL.getConfiguration());
List<String> rows = generateRandomWords(100, "row");
List<String> columns = generateRandomWords(10000, "column");
long maxTimestamp = 2;
List<KeyValue> kvList = new ArrayList<KeyValue>();
Map<String, List<KeyValue>> prefixMap = new HashMap<String,
List<KeyValue>>();
prefixMap.put("p", new ArrayList<KeyValue>());
prefixMap.put("s", new ArrayList<KeyValue>());
String valueString = "ValueString";
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
valueString);
p.add(kv);
kvList.add(kv);
for (String s: prefixMap.keySet()) {
if (column.startsWith(s)) {
prefixMap.get(s).add(kv);
}
}
}
}
region.put(p);
}
ColumnPrefixFilter filter;
Scan scan = new Scan();
scan.setMaxVersions();
for (String s: prefixMap.keySet()) {
filter = new ColumnPrefixFilter(Bytes.toBytes(s));
//this is how this test differs from the one above
FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
filterList.addFilter(filter);
scan.setFilter(filterList);
InternalScanner scanner = region.getScanner(scan); InternalScanner scanner = region.getScanner(scan);
List<KeyValue> results = new ArrayList<KeyValue>(); List<KeyValue> results = new ArrayList<KeyValue>();
while(scanner.next(results)); while(scanner.next(results));