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 2017/11/07 23:30:36 UTC
[7/8] hbase git commit: HBASE-18993 Backport patches in HBASE-18410
to branch-1.x branches
http://git-wip-us.apache.org/repos/asf/hbase/blob/264cf0d4/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index 6ddc422..77fc47e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -18,44 +18,37 @@
*/
package org.apache.hadoop.hbase.filter;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Lists;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNull;
-
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.testclassification.FilterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import com.google.common.collect.Lists;
-
-/**
- * Tests filter sets
- *
- */
-@Category(SmallTests.class)
+@Category({FilterTests.class, SmallTests.class})
public class TestFilterList {
static final int MAX_PAGES = 2;
- static final char FIRST_CHAR = 'a';
- static final char LAST_CHAR = 'e';
- static byte[] GOOD_BYTES = Bytes.toBytes("abc");
- static byte[] BAD_BYTES = Bytes.toBytes("def");
-
@Test
public void testAddFilter() throws Exception {
@@ -74,9 +67,49 @@ public class TestFilterList {
filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(filter1, filter2));
filterList.addFilter(new FirstKeyOnlyFilter());
+ filterList.setReversed(false);
+ FirstKeyOnlyFilter f = new FirstKeyOnlyFilter();
+ f.setReversed(true);
+ try {
+ filterList.addFilter(f);
+ fail("The IllegalArgumentException should be thrown because the added filter is reversed");
+ } catch (IllegalArgumentException e) {
+ }
+
}
+ @Test
+ public void testConstruction() {
+ FirstKeyOnlyFilter f1 = new FirstKeyOnlyFilter();
+ FirstKeyOnlyFilter f2 = new FirstKeyOnlyFilter();
+ f1.setReversed(true);
+ f2.setReversed(false);
+
+ try {
+ FilterList ff = new FilterList(f1, f2);
+ fail("The IllegalArgumentException should be thrown");
+ } catch (IllegalArgumentException e) {
+ }
+
+ try {
+ FilterList ff = new FilterList(Arrays.asList((Filter) f1, (Filter) f2));
+ fail("The IllegalArgumentException should be thrown because the added filter is reversed");
+ } catch (IllegalArgumentException e) {
+ }
+
+ try {
+ FilterList ff = new FilterList(FilterList.Operator.MUST_PASS_ALL,
+ Arrays.asList((Filter) f1, (Filter) f2));
+ fail("The IllegalArgumentException should be thrown because the added filter is reversed");
+ } catch (IllegalArgumentException e) {
+ }
+ try {
+ FilterList ff = new FilterList(FilterList.Operator.MUST_PASS_ALL, f1, f2);
+ fail("The IllegalArgumentException should be thrown because the added filter is reversed");
+ } catch (IllegalArgumentException e) {
+ }
+ }
/**
* Test "must pass one"
* @throws Exception
@@ -90,9 +123,7 @@ public class TestFilterList {
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
- Filter filterMPONE =
- new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
- return filterMPONE;
+ return new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
}
private void mpOneTest(Filter filterMPONE) throws Exception {
@@ -112,11 +143,10 @@ public class TestFilterList {
assertFalse(filterMPONE.filterAllRemaining());
/* Will pass both */
- byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+ byte[] rowkey = Bytes.toBytes("yyyyyyyyy");
for (int i = 0; i < MAX_PAGES - 1; i++) {
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
- KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
- Bytes.toBytes(i));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(filterMPONE.filterRow());
}
@@ -124,16 +154,14 @@ public class TestFilterList {
/* Only pass PageFilter */
rowkey = Bytes.toBytes("z");
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
- KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
- Bytes.toBytes(0));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(filterMPONE.filterRow());
/* reach MAX_PAGES already, should filter any rows */
rowkey = Bytes.toBytes("yyy");
assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
- kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
- Bytes.toBytes(0));
+ kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0));
assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(filterMPONE.filterRow());
@@ -156,8 +184,7 @@ public class TestFilterList {
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
- Filter filterMPALL =
- new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+ Filter filterMPALL = new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
return filterMPALL;
}
@@ -176,11 +203,10 @@ public class TestFilterList {
*/
filterMPALL.reset();
assertFalse(filterMPALL.filterAllRemaining());
- byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+ byte[] rowkey = Bytes.toBytes("yyyyyyyyy");
for (int i = 0; i < MAX_PAGES - 1; i++) {
assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
- KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
- Bytes.toBytes(i));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
}
filterMPALL.reset();
@@ -204,8 +230,7 @@ public class TestFilterList {
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
filters.add(new PageFilter(MAX_PAGES));
- Filter filterMPONE =
- new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+ Filter filterMPONE = new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
return filterMPONE;
}
@@ -226,12 +251,11 @@ public class TestFilterList {
assertFalse(filterMPONE.filterAllRemaining());
/* We should be able to fill MAX_PAGES without incrementing page counter */
- byte [] rowkey = Bytes.toBytes("yyyyyyyy");
+ byte[] rowkey = Bytes.toBytes("yyyyyyyy");
for (int i = 0; i < MAX_PAGES; i++) {
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
- KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
- Bytes.toBytes(i));
- assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(filterMPONE.filterRow());
}
@@ -239,9 +263,8 @@ public class TestFilterList {
rowkey = Bytes.toBytes("xxxxxxx");
for (int i = 0; i < MAX_PAGES; i++) {
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
- KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
- Bytes.toBytes(i));
- assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(filterMPONE.filterRow());
}
@@ -249,9 +272,8 @@ public class TestFilterList {
rowkey = Bytes.toBytes("yyy");
for (int i = 0; i < MAX_PAGES; i++) {
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
- KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
- Bytes.toBytes(i));
- assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(filterMPONE.filterRow());
}
}
@@ -266,27 +288,27 @@ public class TestFilterList {
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);
+ assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
+ assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
flist.reset();
flist.filterRowKey(r2, 0, r2.length);
- assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
-
+ assertEquals(ReturnCode.SKIP, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
+
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);
+ assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
+ assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
flist.reset();
flist.filterRowKey(r2, 0, r2.length);
- assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
+ assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
}
/**
@@ -295,32 +317,34 @@ public class TestFilterList {
* @throws Exception
*/
@Test
- public void testFilterListWithInclusiveStopFilteMustPassOne() throws Exception {
+ public void testFilterListWithInclusiveStopFilterMustPassOne() 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);
+ assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
+ assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
flist.reset();
flist.filterRowKey(r2, 0, r2.length);
- assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
+ assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
}
- public static class AlwaysNextColFilter extends FilterBase {
+ static class AlwaysNextColFilter extends FilterBase {
public AlwaysNextColFilter() {
super();
}
+
@Override
public ReturnCode filterKeyValue(Cell v) {
return ReturnCode.NEXT_COL;
}
- public static AlwaysNextColFilter parseFrom(final byte [] pbBytes)
+
+ public static AlwaysNextColFilter parseFrom(final byte[] pbBytes)
throws DeserializationException {
return new AlwaysNextColFilter();
}
@@ -335,14 +359,13 @@ public class TestFilterList {
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
- Filter filterMPALL =
- new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+ Filter filterMPALL = new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
// Decompose filterMPALL to bytes.
byte[] buffer = filterMPALL.toByteArray();
// Recompose filterMPALL.
- FilterList newFilter = FilterList.parseFrom(buffer);
+ FilterList.parseFrom(buffer);
// Run tests
mpOneTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getFilterMPONE())));
@@ -368,8 +391,8 @@ public class TestFilterList {
@Override
public Filter.ReturnCode filterKeyValue(Cell v) {
- Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
- Filter.ReturnCode.SKIP;
+ Filter.ReturnCode returnCode =
+ returnInclude ? Filter.ReturnCode.INCLUDE : Filter.ReturnCode.SKIP;
returnInclude = !returnInclude;
return returnCode;
}
@@ -380,8 +403,8 @@ public class TestFilterList {
@Override
public Filter.ReturnCode filterKeyValue(Cell v) {
- Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
- Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
+ Filter.ReturnCode returnCode =
+ returnIncludeOnly ? Filter.ReturnCode.INCLUDE : Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
returnIncludeOnly = !returnIncludeOnly;
return returnCode;
}
@@ -391,8 +414,8 @@ public class TestFilterList {
FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
// INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
- assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpOnefilterList.filterKeyValue(null));
- // INCLUDE, SKIP, INCLUDE.
+ assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
+ // INCLUDE, SKIP, INCLUDE.
assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
// Check must pass all filter.
@@ -400,7 +423,7 @@ public class TestFilterList {
Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
// INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
- // INCLUDE, SKIP, INCLUDE.
+ // INCLUDE, SKIP, INCLUDE.
assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
}
@@ -411,15 +434,14 @@ public class TestFilterList {
public void testHintPassThru() throws Exception {
final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
- final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
- null, null);
+ final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
Filter filterNoHint = new FilterBase() {
@Override
- public byte [] toByteArray() {
+ public byte[] toByteArray() {
return null;
}
-
+
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
@@ -438,7 +460,9 @@ public class TestFilterList {
}
@Override
- public byte [] toByteArray() {return null;}
+ public byte[] toByteArray() {
+ return null;
+ }
};
Filter filterMaxHint = new FilterBase() {
@@ -448,111 +472,483 @@ public class TestFilterList {
}
@Override
- public Cell getNextCellHint(Cell currentKV) {
+ public Cell getNextCellHint(Cell cell) {
return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
}
@Override
- public byte [] toByteArray() {return null;}
+ public byte[] toByteArray() {
+ return null;
+ }
};
// MUST PASS ONE
// Should take the min if given two hints
FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
- Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- minKeyValue));
+ Arrays.asList(new Filter[] { filterMinHint, filterMaxHint }));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
// Should have no hint if any filter has no hint
filterList = new FilterList(Operator.MUST_PASS_ONE,
- Arrays.asList(
- new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
- assertNull(filterList.getNextKeyHint(null));
+ Arrays.asList(new Filter[] { filterMinHint, filterMaxHint, filterNoHint }));
+ assertNull(filterList.getNextCellHint(null));
filterList = new FilterList(Operator.MUST_PASS_ONE,
- Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
- assertNull(filterList.getNextKeyHint(null));
+ Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
+ assertNull(filterList.getNextCellHint(null));
// Should give max hint if its the only one
filterList = new FilterList(Operator.MUST_PASS_ONE,
- Arrays.asList(new Filter [] { filterMaxHint, filterMaxHint } ));
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- maxKeyValue));
+ Arrays.asList(new Filter[] { filterMaxHint, filterMaxHint }));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
// MUST PASS ALL
// Should take the first hint
filterList = new FilterList(Operator.MUST_PASS_ALL,
- Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
+ Arrays.asList(new Filter[] { filterMinHint, filterMaxHint }));
filterList.filterKeyValue(null);
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- minKeyValue));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
filterList = new FilterList(Operator.MUST_PASS_ALL,
- Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
+ Arrays.asList(new Filter[] { filterMaxHint, filterMinHint }));
filterList.filterKeyValue(null);
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- maxKeyValue));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
// Should have first hint even if a filter has no hint
filterList = new FilterList(Operator.MUST_PASS_ALL,
- Arrays.asList(
- new Filter [] { filterNoHint, filterMinHint, filterMaxHint } ));
+ Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint }));
filterList.filterKeyValue(null);
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- minKeyValue));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
filterList = new FilterList(Operator.MUST_PASS_ALL,
- Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
+ Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
filterList.filterKeyValue(null);
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- maxKeyValue));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
filterList = new FilterList(Operator.MUST_PASS_ALL,
- Arrays.asList(new Filter [] { filterNoHint, filterMinHint } ));
+ Arrays.asList(new Filter[] { filterNoHint, filterMinHint }));
filterList.filterKeyValue(null);
- assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextKeyHint(null),
- minKeyValue));
+ assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
}
/**
- * Tests the behavior of transform() in a hierarchical filter.
- *
- * transform() only applies after a filterKeyValue() whose return-code includes the KeyValue.
- * Lazy evaluation of AND
+ * Tests the behavior of transform() in a hierarchical filter. transform() only applies after a
+ * filterKeyValue() whose return-code includes the KeyValue. Lazy evaluation of AND
*/
@Test
public void testTransformMPO() throws Exception {
// Apply the following filter:
- // (family=fam AND qualifier=qual1 AND KeyOnlyFilter)
- // OR (family=fam AND qualifier=qual2)
- final FilterList flist = new FilterList(Operator.MUST_PASS_ONE, Lists.<Filter>newArrayList(
- new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
- new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
- new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("qual1"))),
- new KeyOnlyFilter())),
- new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
- new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
- new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("qual2")))))));
-
- final KeyValue kvQual1 = new KeyValue(
- Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual1"), Bytes.toBytes("value"));
- final KeyValue kvQual2 = new KeyValue(
- Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual2"), Bytes.toBytes("value"));
- final KeyValue kvQual3 = new KeyValue(
- Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value"));
+ // (family=fam AND qualifier=qual1 AND KeyOnlyFilter)
+ // OR (family=fam AND qualifier=qual2)
+ final FilterList flist =
+ new FilterList(Operator.MUST_PASS_ONE,
+ Lists.<Filter> newArrayList(
+ new FilterList(Operator.MUST_PASS_ALL,
+ Lists.<Filter> newArrayList(
+ new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
+ new QualifierFilter(CompareOp.EQUAL,
+ new BinaryComparator(Bytes.toBytes("qual1"))),
+ new KeyOnlyFilter())),
+ new FilterList(Operator.MUST_PASS_ALL,
+ Lists.<Filter> newArrayList(
+ new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
+ new QualifierFilter(CompareOp.EQUAL,
+ new BinaryComparator(Bytes.toBytes("qual2")))))));
+
+ final KeyValue kvQual1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
+ Bytes.toBytes("qual1"), Bytes.toBytes("value"));
+ final KeyValue kvQual2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
+ Bytes.toBytes("qual2"), Bytes.toBytes("value"));
+ final KeyValue kvQual3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
+ Bytes.toBytes("qual3"), Bytes.toBytes("value"));
// Value for fam:qual1 should be stripped:
assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1));
- final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transform(kvQual1));
- assertEquals(0, transformedQual1.getValue().length);
+ final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual1));
+ assertEquals(0, transformedQual1.getValueLength());
// Value for fam:qual2 should not be stripped:
assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2));
- final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transform(kvQual2));
- assertEquals("value", Bytes.toString(transformedQual2.getValue()));
+ final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual2));
+ assertEquals("value", Bytes.toString(transformedQual2.getValueArray(),
+ transformedQual2.getValueOffset(), transformedQual2.getValueLength()));
// Other keys should be skipped:
assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
}
-}
+ @Test
+ public void testWithMultiVersionsInSameRow() throws Exception {
+ FilterList filterList01 =
+ new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 0));
+
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+ 1, Bytes.toBytes("value"));
+ KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+ 2, Bytes.toBytes("value"));
+ KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+ 3, Bytes.toBytes("value"));
+
+ assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterKeyValue(kv1));
+ assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv2));
+ assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv3));
+
+ FilterList filterList11 =
+ new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
+
+ assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv1));
+ assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv2));
+ assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv3));
+ }
+ @Test
+ public void testMPONEWithSeekNextUsingHint() throws Exception {
+ byte[] col = Bytes.toBytes("c");
+ FilterList filterList =
+ new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, col));
+
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 2,
+ Bytes.toBytes("value"));
+ KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 3,
+ Bytes.toBytes("value"));
+ KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
+ Bytes.toBytes("value"));
+
+ assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+ assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv2));
+ assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv3));
+ assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv4));
+ }
+
+ private static class MockFilter extends FilterBase {
+ private ReturnCode targetRetCode;
+ public boolean didCellPassToTheFilter = false;
+
+ public MockFilter(ReturnCode targetRetCode) {
+ this.targetRetCode = targetRetCode;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
+ this.didCellPassToTheFilter = true;
+ return targetRetCode;
+ }
+ }
+
+ @Test
+ public void testShouldPassCurrentCellToFilter() throws IOException {
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 2,
+ Bytes.toBytes("value"));
+ KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 3,
+ Bytes.toBytes("value"));
+ KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
+ Bytes.toBytes("value"));
+
+ MockFilter mockFilter = new MockFilter(ReturnCode.NEXT_COL);
+ FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+
+ filter.filterKeyValue(kv1);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv2);
+ assertFalse(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv3);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
+ filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+
+ filter.filterKeyValue(kv1);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv2);
+ assertFalse(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv3);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter = new MockFilter(ReturnCode.NEXT_ROW);
+ filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+ filter.filterKeyValue(kv1);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv2);
+ assertFalse(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv3);
+ assertFalse(mockFilter.didCellPassToTheFilter);
+
+ filter.reset();
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv4);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
+ filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
+ filter.filterKeyValue(kv1);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv2);
+ assertFalse(mockFilter.didCellPassToTheFilter);
+
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv3);
+ assertFalse(mockFilter.didCellPassToTheFilter);
+
+ filter.reset();
+ mockFilter.didCellPassToTheFilter = false;
+ filter.filterKeyValue(kv4);
+ assertTrue(mockFilter.didCellPassToTheFilter);
+ }
+
+ @Test
+ public void testTheMaximalRule() throws IOException {
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ MockFilter filter1 = new MockFilter(ReturnCode.INCLUDE);
+ MockFilter filter2 = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
+ MockFilter filter3 = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
+ MockFilter filter4 = new MockFilter(ReturnCode.NEXT_COL);
+ MockFilter filter5 = new MockFilter(ReturnCode.SKIP);
+ MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
+ MockFilter filter7 = new MockFilter(ReturnCode.NEXT_ROW);
+
+ FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
+ assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, filter3);
+ assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6);
+ assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6);
+ assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1);
+ assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter2, filter1, filter5);
+ assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter2,
+ new FilterList(Operator.MUST_PASS_ALL, filter3, filter4));
+ assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter7);
+ assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
+ }
+
+ @Test
+ public void testTheMinimalRule() throws IOException {
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ MockFilter filter1 = new MockFilter(ReturnCode.INCLUDE);
+ MockFilter filter2 = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
+ MockFilter filter3 = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
+ MockFilter filter4 = new MockFilter(ReturnCode.NEXT_COL);
+ MockFilter filter5 = new MockFilter(ReturnCode.SKIP);
+ MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
+ FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
+ assertEquals(filterList.filterKeyValue(kv1), ReturnCode.INCLUDE);
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, filter3);
+ assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter5, filter6);
+ assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter6);
+ assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter1);
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter2, filter1, filter5);
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
+ new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
+ assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
+ new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
+ assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6);
+ assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+ }
+
+ static class MockSeekHintFilter extends FilterBase {
+ private Cell returnCell;
+
+ public MockSeekHintFilter(Cell returnCell) {
+ this.returnCell = returnCell;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
+ return ReturnCode.SEEK_NEXT_USING_HINT;
+ }
+
+ @Override
+ public Cell getNextCellHint(Cell currentCell) throws IOException {
+ return this.returnCell;
+ }
+ }
+
+ @Test
+ public void testReversedFilterListWithMockSeekHintFilter() throws IOException {
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ KeyValue kv3 = new KeyValue(Bytes.toBytes("row3"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ Filter filter1 = new MockSeekHintFilter(kv1);
+ filter1.setReversed(true);
+ Filter filter2 = new MockSeekHintFilter(kv2);
+ filter2.setReversed(true);
+ Filter filter3 = new MockSeekHintFilter(kv3);
+ filter3.setReversed(true);
+
+ FilterList filterList = new FilterList(Operator.MUST_PASS_ONE);
+ filterList.setReversed(true);
+ filterList.addFilter(filter1);
+ filterList.addFilter(filter2);
+ filterList.addFilter(filter3);
+
+ Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+ Assert.assertEquals(kv3, filterList.getNextCellHint(kv1));
+
+ filterList = new FilterList(Operator.MUST_PASS_ALL);
+ filterList.setReversed(true);
+ filterList.addFilter(filter1);
+ filterList.addFilter(filter2);
+ filterList.addFilter(filter3);
+
+ Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
+ Assert.assertEquals(kv1, filterList.getNextCellHint(kv1));
+ }
+
+ @Test
+ public void testReversedFilterListWithOR() throws IOException {
+ byte[] r22 = Bytes.toBytes("Row22");
+ byte[] r2 = Bytes.toBytes("Row2");
+ byte[] r1 = Bytes.toBytes("Row1");
+
+ FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+ filterList.setReversed(true);
+ PrefixFilter prefixFilter = new PrefixFilter(r2);
+ prefixFilter.setReversed(true);
+ filterList.addFilter(prefixFilter);
+ filterList.filterRowKey(r22, 0, r22.length);
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
+
+ filterList.reset();
+ filterList.filterRowKey(r1, 0, r1.length);
+ assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(new KeyValue(r1, r1, r1)));
+
+ filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+ filterList.setReversed(true);
+ AlwaysNextColFilter alwaysNextColFilter = new AlwaysNextColFilter();
+ alwaysNextColFilter.setReversed(true);
+ prefixFilter = new PrefixFilter(r2);
+ prefixFilter.setReversed(true);
+ filterList.addFilter(alwaysNextColFilter);
+ filterList.addFilter(prefixFilter);
+ filterList.filterRowKey(r22, 0, r22.length);
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
+
+ filterList.reset();
+ filterList.filterRowKey(r1, 0, r1.length);
+ assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(new KeyValue(r1, r1, r1)));
+ }
+
+ @Test
+ public void testKeyOnlyFilterTransformCell() throws IOException {
+ Cell c;
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
+ 1, Bytes.toBytes("value1"));
+ KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
+ 2, Bytes.toBytes("value2"));
+
+ Filter filter1 = new SingleColumnValueFilter(Bytes.toBytes("cf"), Bytes.toBytes("column1"),
+ CompareOp.EQUAL, Bytes.toBytes("value1"));
+ Filter filter2 = new SingleColumnValueFilter(Bytes.toBytes("cf"), Bytes.toBytes("column1"),
+ CompareOp.EQUAL, Bytes.toBytes("value2"));
+ FilterList internalFilterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
+
+ FilterList keyOnlyFilterFirst =
+ new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
+
+ assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv1));
+ c = keyOnlyFilterFirst.transformCell(kv1);
+ assertEquals(0, c.getValueLength());
+ assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv2));
+ c = keyOnlyFilterFirst.transformCell(kv2);
+ assertEquals(0, c.getValueLength());
+
+ internalFilterList.reset();
+ FilterList keyOnlyFilterLast =
+ new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
+ assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv1));
+ c = keyOnlyFilterLast.transformCell(kv1);
+ assertEquals(0, c.getValueLength());
+ assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv2));
+ c = keyOnlyFilterLast.transformCell(kv2);
+ assertEquals(0, c.getValueLength());
+ }
+
+ @Test
+ public void testEmptyFilterListTransformCell() throws IOException {
+ KeyValue kv = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
+ 1, Bytes.toBytes("value"));
+ FilterList filterList = new FilterList(Operator.MUST_PASS_ALL);
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
+ assertEquals(kv, filterList.transformCell(kv));
+
+ filterList = new FilterList(Operator.MUST_PASS_ONE);
+ assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
+ assertEquals(kv, filterList.transformCell(kv));
+ }
+
+ private static class MockNextRowFilter extends FilterBase {
+ private int hitCount = 0;
+
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
+ hitCount++;
+ return ReturnCode.NEXT_ROW;
+ }
+
+ public int getHitCount() {
+ return hitCount;
+ }
+ }
+
+ @Test
+ public void testRowCountFilter() throws IOException {
+ KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam1"), Bytes.toBytes("a"), 1,
+ Bytes.toBytes("value"));
+ KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam2"), Bytes.toBytes("a"), 2,
+ Bytes.toBytes("value"));
+ MockNextRowFilter mockNextRowFilter = new MockNextRowFilter();
+ FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockNextRowFilter);
+ filter.filterKeyValue(kv1);
+ filter.filterKeyValue(kv2);
+ assertEquals(2, mockNextRowFilter.getHitCount());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/264cf0d4/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
index dd2399f..2a13ac8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java
@@ -36,10 +36,10 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/**
- * Tests filter Lists in ways that rely on a MiniCluster.
- * Where possible, favor tests in TestFilterList and TestFilterFromRegionSide instead.
+ * Tests filter Lists in ways that rely on a MiniCluster. Where possible, favor tests in
+ * TestFilterList and TestFilterFromRegionSide instead.
*/
-@Category({MediumTests.class, FilterTests.class})
+@Category({ MediumTests.class, FilterTests.class })
public class TestFilterListOnMini {
private static final Log LOG = LogFactory.getLog(TestFilterListOnMini.class);