You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2014/02/12 00:34:24 UTC
svn commit: r1567461 - in /hbase/branches/0.94/src:
main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
Author: larsh
Date: Tue Feb 11 23:34:23 2014
New Revision: 1567461
URL: http://svn.apache.org/r1567461
Log:
HBASE-10485 PrefixFilter#filterKeyValue() should perform filtering on row key (Ted Yu and LarsH)
Modified:
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java?rev=1567461&r1=1567460&r2=1567461&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java Tue Feb 11 23:34:23 2014
@@ -25,6 +25,7 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
+import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.base.Preconditions;
@@ -66,6 +67,11 @@ public class PrefixFilter extends Filter
return filterRow;
}
+ @Override
+ public ReturnCode filterKeyValue(KeyValue ignored) {
+ return filterRow ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE;
+ }
+
public boolean filterRow() {
return filterRow;
}
Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java?rev=1567461&r1=1567460&r2=1567461&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java Tue Feb 11 23:34:23 2014
@@ -207,7 +207,7 @@ public class TestFilterList extends Test
}
}
- public static class AlwaysNextColFilter extends FilterBase {
+ private static class AlwaysNextColFilter extends FilterBase {
public AlwaysNextColFilter() {
super();
}
@@ -231,7 +231,7 @@ public class TestFilterList extends Test
byte[] r1 = Bytes.toBytes("Row1");
byte[] r11 = Bytes.toBytes("Row11");
byte[] r2 = Bytes.toBytes("Row2");
-
+
FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
flist.addFilter(new AlwaysNextColFilter());
flist.addFilter(new InclusiveStopFilter(r1));
@@ -245,6 +245,38 @@ public class TestFilterList extends Test
}
/**
+ * When we do a "MUST_PASS_ONE" (a logical 'OR') of the above two filters
+ * we expect to get the same result as the 'prefix' only result.
+ * @throws Exception
+ */
+ public void testFilterListTwoFiltersMustPassOne() throws Exception {
+ byte[] r1 = Bytes.toBytes("Row1");
+ byte[] r11 = Bytes.toBytes("Row11");
+ byte[] r2 = Bytes.toBytes("Row2");
+
+ FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+ flist.addFilter(new PrefixFilter(r1));
+ flist.filterRowKey(r1, 0, r1.length);
+ assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
+ assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
+
+ flist.reset();
+ flist.filterRowKey(r2, 0, r2.length);
+ assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
+
+ flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+ flist.addFilter(new AlwaysNextColFilter());
+ flist.addFilter(new PrefixFilter(r1));
+ flist.filterRowKey(r1, 0, r1.length);
+ assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
+ assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
+
+ flist.reset();
+ flist.filterRowKey(r2, 0, r2.length);
+ assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
+ }
+
+ /**
* Test serialization
* @throws Exception
*/