You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2011/02/20 03:50:48 UTC
svn commit: r1072481 - in /hbase/trunk: CHANGES.txt
src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
Author: apurtell
Date: Sun Feb 20 02:50:48 2011
New Revision: 1072481
URL: http://svn.apache.org/viewvc?rev=1072481&view=rev
Log:
HBASE-3550 FilterList reports false positives
Modified:
hbase/trunk/CHANGES.txt
hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1072481&r1=1072480&r2=1072481&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Sun Feb 20 02:50:48 2011
@@ -49,7 +49,8 @@ Release 0.91.0 - Unreleased
HBASE-3525 mvn assembly is over-filling the hbase lib dir
HBASE-3531 When under global memstore pressure, dont try to flush
unflushable regions.
-
+ HBASE-3550 FilterList reports false positives (Bill Graham via Andrew
+ Purtell)
IMPROVEMENTS
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=1072481&r1=1072480&r2=1072481&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java Sun Feb 20 02:50:48 2011
@@ -168,12 +168,15 @@ public class FilterList implements Filte
if (filter.filterAllRemaining()) {
return ReturnCode.NEXT_ROW;
}
- switch (filter.filterKeyValue(v)) {
+ ReturnCode code = filter.filterKeyValue(v);
+ switch (code) {
case INCLUDE:
continue;
case NEXT_ROW:
case SKIP:
return ReturnCode.SKIP;
+ default:
+ return code;
}
} else if (operator == Operator.MUST_PASS_ONE) {
if (filter.filterAllRemaining()) {
Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java?rev=1072481&r1=1072480&r2=1072481&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java Sun Feb 20 02:50:48 2011
@@ -74,7 +74,68 @@ public class TestColumnPrefixFilter {
scan.setMaxVersions();
for (String s: prefixMap.keySet()) {
filter = new ColumnPrefixFilter(Bytes.toBytes(s));
+
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);
List<KeyValue> results = new ArrayList<KeyValue>();
while(scanner.next(results));