hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1030692 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/filter/FilterList.java src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
Date Wed, 03 Nov 2010 21:16:40 GMT
Author: stack
Date: Wed Nov  3 21:16:38 2010
New Revision: 1030692

URL: http://svn.apache.org/viewvc?rev=1030692&view=rev
Log:
HBASE-3191 FilterList with MUST_PASS_ONE and SCVF isn't working

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/TestFilter.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1030692&r1=1030691&r2=1030692&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed Nov  3 21:16:38 2010
@@ -643,6 +643,8 @@ Release 0.21.0 - Unreleased
    HBASE-1932  Encourage use of 'lzo' compression... add the wiki page to
                getting started
    HBASE-3151  NPE when trying to read regioninfo from .META.
+   HBASE-3191  FilterList with MUST_PASS_ONE and SCVF isn't working
+               (Stefan Seelmann via Stack)
 
 
   IMPROVEMENTS

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=1030692&r1=1030691&r2=1030692&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 Wed Nov  3 21:16:38
2010
@@ -161,6 +161,8 @@ public class FilterList implements Filte
 
   @Override
   public ReturnCode filterKeyValue(KeyValue v) {
+    ReturnCode rc = operator == Operator.MUST_PASS_ONE?
+        ReturnCode.SKIP: ReturnCode.INCLUDE;
     for (Filter filter : filters) {
       if (operator == Operator.MUST_PASS_ALL) {
         if (filter.filterAllRemaining()) {
@@ -180,15 +182,15 @@ public class FilterList implements Filte
 
         switch (filter.filterKeyValue(v)) {
         case INCLUDE:
-          return ReturnCode.INCLUDE;
+          rc = ReturnCode.INCLUDE;
+          // must continue here to evaluate all filters
         case NEXT_ROW:
         case SKIP:
           // continue;
         }
       }
     }
-    return operator == Operator.MUST_PASS_ONE?
-      ReturnCode.SKIP: ReturnCode.INCLUDE;
+    return rc;
   }
 
   @Override

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1030692&r1=1030691&r2=1030692&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Wed Nov  3 21:16:38
2010
@@ -1041,6 +1041,82 @@ public class TestFilter extends HBaseTes
     verifyScanFull(s, kvs);
   }
 
+  public void testFilterListWithSingleColumnValueFilter() throws IOException {
+    // Test for HBASE-3191
+
+    // Scan using SingleColumnValueFilter
+    SingleColumnValueFilter f1 = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
+          CompareOp.EQUAL, VALUES[0]);
+    f1.setFilterIfMissing( true );
+    Scan s1 = new Scan();
+    s1.addFamily(FAMILIES[0]);
+    s1.setFilter(f1);
+    KeyValue [] kvs1 = {
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+    };
+    verifyScanNoEarlyOut(s1, 3, 3);
+    verifyScanFull(s1, kvs1);
+
+    // Scan using another SingleColumnValueFilter, expect disjoint result
+    SingleColumnValueFilter f2 = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_TWO[0],
+        CompareOp.EQUAL, VALUES[1]);
+    f2.setFilterIfMissing( true );
+    Scan s2 = new Scan();
+    s2.addFamily(FAMILIES[0]);
+    s2.setFilter(f2);
+    KeyValue [] kvs2 = {
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+    };
+    verifyScanNoEarlyOut(s2, 3, 3);
+    verifyScanFull(s2, kvs2);
+
+    // Scan, ORing the two previous filters, expect unified result
+    FilterList f = new FilterList(Operator.MUST_PASS_ONE);
+    f.addFilter(f1);
+    f.addFilter(f2);
+    Scan s = new Scan();
+    s.addFamily(FAMILIES[0]);
+    s.setFilter(f);
+    KeyValue [] kvs = {
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+    };
+    verifyScanNoEarlyOut(s, 6, 3);
+    verifyScanFull(s, kvs);
+  }
+
   public void testSingleColumnValueFilter() throws IOException {
 
     // From HBASE-1821



Mime
View raw message