You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2014/02/11 22:43:26 UTC
svn commit: r1567402 - in /hbase/branches/0.98:
hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
Author: tedyu
Date: Tue Feb 11 21:43:26 2014
New Revision: 1567402
URL: http://svn.apache.org/r1567402
Log:
HBASE-10493 InclusiveStopFilter#filterKeyValue() should perform filtering on row key
Modified:
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java?rev=1567402&r1=1567401&r2=1567402&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java Tue Feb 11 21:43:26 2014
@@ -24,7 +24,9 @@ import java.util.ArrayList;
import com.google.protobuf.HBaseZeroCopyByteString;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
@@ -51,6 +53,12 @@ public class InclusiveStopFilter extends
return this.stopRowKey;
}
+ @Override
+ public ReturnCode filterKeyValue(Cell v) {
+ if (done) return ReturnCode.NEXT_ROW;
+ return ReturnCode.INCLUDE;
+ }
+
public boolean filterRowKey(byte[] buffer, int offset, int length) {
if (buffer == null) {
//noinspection RedundantIfStatement
Modified: hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java?rev=1567402&r1=1567401&r2=1567402&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (original)
+++ hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java Tue Feb 11 21:43:26 2014
@@ -294,6 +294,28 @@ public class TestFilterList {
assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
}
+ /**
+ * When we do a "MUST_PASS_ONE" (a logical 'OR') of the two filters
+ * we expect to get the same result as the inclusive stop result.
+ * @throws Exception
+ */
+ public void testFilterListWithInclusiveStopFilteMustPassOne() 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 AlwaysNextColFilter());
+ flist.addFilter(new InclusiveStopFilter(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);
+ }
+
public static class AlwaysNextColFilter extends FilterBase {
public AlwaysNextColFilter() {
super();