You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2012/01/18 04:14:40 UTC

svn commit: r1232739 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/filter/FilterList.java test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java

Author: nspiegelberg
Date: Wed Jan 18 03:14:39 2012
New Revision: 1232739

URL: http://svn.apache.org/viewvc?rev=1232739&view=rev
Log:
[HBASE-5126] Pulling in HBASE-3550.

Summary: FilterList for MUST_PASS_ALL was incorrectly including kvs that were
tagged with SEEK_NEXT_USING_HINT.

Test Plan: Ran unit test.

Reviewers: kannan, aaiyer

Reviewed By: kannan

CC: hbase-eng@lists, liujiakai, aaiyer, madhuvaidya, pkhemani, kannan

Differential Revision: https://phabricator.fb.com/D384773

Task ID: 865824

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=1232739&r1=1232738&r2=1232739&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java Wed Jan 18 03:14:39 2012
@@ -166,12 +166,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()) {
@@ -250,4 +253,4 @@ public class FilterList implements Filte
   public KeyValue getNextKeyHint(KeyValue currentKV) {
     return null;
   }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java?rev=1232739&r1=1232738&r2=1232739&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java Wed Jan 18 03:14:39 2012
@@ -113,7 +113,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));