From c3177add7e445a04ed92084b015728ac9b62b74c Mon Sep 17 00:00:00 2001 From: Reid Chan Date: Tue, 22 May 2018 12:29:29 +0800 Subject: [PATCH] HBASE-20151 Bug with SingleColumnValueFilter and FamilyFilter --- .../apache/hadoop/hbase/filter/FamilyFilter.java | 9 +++ .../org/apache/hadoop/hbase/filter/Filter.java | 15 ++++ .../apache/hadoop/hbase/filter/FilterListBase.java | 4 ++ .../hadoop/hbase/filter/FilterListWithAND.java | 1 + .../hadoop/hbase/filter/FilterListWithOR.java | 1 + .../hbase/filter/SingleColumnValueFilter.java | 1 - .../apache/hadoop/hbase/filter/FilterWrapper.java | 5 ++ .../org/apache/hadoop/hbase/filter/TestFilter.java | 81 ++++++++++++++++++++++ .../apache/hadoop/hbase/filter/TestFilterList.java | 44 ++++++++++++ 9 files changed, 160 insertions(+), 1 deletion(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java index f114e98b7d..7ceb372f18 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java @@ -89,6 +89,15 @@ public class FamilyFilter extends CompareFilter { return ReturnCode.INCLUDE; } + @Override + public ReturnCode transformReturnCode(FilterListBase.LOGIC logic, ReturnCode originRC) { + if (logic == FilterListBase.LOGIC.AND && + originRC == ReturnCode.NEXT_ROW) { + return ReturnCode.SKIP; + } + return originRC; + } + public static Filter createFilterFromArguments(ArrayList filterArguments) { ArrayList arguments = CompareFilter.extractArguments(filterArguments); CompareOperator compareOp = (CompareOperator)arguments.get(0); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java index dec8e061b5..51724c73c3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java @@ -290,6 +290,21 @@ public abstract class Filter { */ abstract public byte[] toByteArray() throws IOException; + /** + * It will be called in {@link FilterListWithAND} and {@link FilterListWithOR}. Sometimes + * ReturnCode of one filter may advance too fast, skipping too many cells which leads to + * incorrect answers of filtering. So this method is to provide a way for transforming the + * ReturnCode after {@link #filterCell(Cell)} called. + * For more details, please refer to HBASE-20151. + * @see HBASE-20151 + * @param logic Logic.AND or Logic.OR + * @param originRC ReturnCode from {@link #filterCell(Cell)} + * @return transformed ReturnCode + */ + public ReturnCode transformReturnCode(FilterListBase.LOGIC logic, ReturnCode originRC) { + return originRC; + } + /** * * Concrete implementers can signal a failure condition in their code by throwing an diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java index d6bb75cd04..fceca35cda 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java @@ -43,6 +43,10 @@ public abstract class FilterListBase extends FilterBase { */ protected ArrayList subFiltersIncludedCell; + enum LOGIC { + AND, OR + } + public FilterListBase(List filters) { reversed = checkAndGetReversed(filters, reversed); this.filters = new ArrayList<>(filters); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java index 9f2ca21b28..9572e0fc27 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java @@ -168,6 +168,7 @@ public class FilterListWithAND extends FilterListBase { } ReturnCode localRC; localRC = filter.filterCell(c); + localRC = filter.transformReturnCode(LOGIC.AND, localRC); rc = mergeReturnCode(rc, localRC); if (localRC == ReturnCode.SEEK_NEXT_USING_HINT) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java index 064dd8387b..fdb3ba37fd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java @@ -266,6 +266,7 @@ public class FilterListWithOR extends FilterListBase { } ReturnCode localRC = filter.filterCell(c); + localRC = filter.transformReturnCode(LOGIC.OR, localRC); // Update previous return code and previous cell for filter[i]. updatePrevFilterRCList(i, localRC); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index e5c83b1d72..b377b80020 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -254,7 +254,6 @@ public class SingleColumnValueFilter extends FilterBase { @Override public ReturnCode filterCell(final Cell c) { - // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue())); if (this.matchedColumn) { // We already found and matched the single column, all keys now pass return ReturnCode.INCLUDE; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java index 9bc072a048..687bb079e1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java @@ -129,6 +129,11 @@ final public class FilterWrapper extends Filter { return this.filter.transformCell(v); } + @Override + public ReturnCode transformReturnCode(FilterListBase.LOGIC logic, ReturnCode originRC) { + return this.filter.transformReturnCode(logic, originRC); + } + @Override public boolean hasFilterRow() { return this.filter.hasFilterRow(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index a3e3359c1d..e5190dd1a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -112,6 +112,9 @@ public class TestFilter { Bytes.toBytes("testFamilyThree"), Bytes.toBytes("testFamilyFour") }; + private static final byte[] FAMILY_A = Bytes.toBytes("a"); + private static final byte[] FAMILY_B = Bytes.toBytes("b"); + private static final byte [][] QUALIFIERS_ONE = { Bytes.toBytes("testQualifierOne-0"), Bytes.toBytes("testQualifierOne-1"), Bytes.toBytes("testQualifierOne-2"), Bytes.toBytes("testQualifierOne-3") @@ -157,6 +160,9 @@ public class TestFilter { htd.addFamily(new HColumnDescriptor(FAMILIES_1[1])); htd.addFamily(new HColumnDescriptor(NEW_FAMILIES[0])); htd.addFamily(new HColumnDescriptor(NEW_FAMILIES[1])); + HColumnDescriptor familyA = new HColumnDescriptor(FAMILY_A).setVersions(3, 3); + htd.addFamily(familyA); + htd.addFamily(new HColumnDescriptor(FAMILY_B)); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd); @@ -1667,6 +1673,81 @@ public class TestFilter { } + @Test + public void testSingleColumnValueFilterAndFamilyFilter() throws Exception { + // Prepare environment, kvs are: + // r1, 'a:1', 'nil' + // r1. 'a:10', 'nil' + // r1. 'b:2', 'nil' + byte[] r1 = Bytes.toBytes("r1"); + byte[] q1 = Bytes.toBytes("1"); + byte[] q10 = Bytes.toBytes("10"); + byte[] q2 = Bytes.toBytes("2"); + byte[] vNil = Bytes.toBytes(""); + Put kv1 = new Put(r1).setDurability(Durability.SKIP_WAL); + // FAMILY_A supports multi-version + kv1.addColumn(FAMILY_A, q1, vNil) + .addColumn(FAMILY_A, q10, vNil) + .addColumn(FAMILY_B, q2, vNil); + region.put(kv1); + region.flush(true); + + // Test + // 1. SCVF, filterIfMissing:true, latestVersionOnly:true + FamilyFilter ff = new FamilyFilter(CompareOperator.EQUAL, new BinaryComparator(FAMILY_B)); + SingleColumnValueFilter scvf = new SingleColumnValueFilter( + FAMILY_A, q10, CompareOperator.EQUAL, new BinaryComparator(vNil), true, true); + FilterList fl = new FilterList(new Filter[] {scvf, ff}); + Scan scan = new Scan().setFilter(fl); + KeyValue[] expected = { new KeyValue(r1, FAMILY_B, q2, vNil) }; + verifyScanFull(scan, expected); + + // 2. SCVF, filterIfMissing:false, latestVersionOnly:true (default) + scvf.setFilterIfMissing(false); + scvf.setLatestVersionOnly(true); + fl = new FilterList(new Filter[] {scvf, ff}); + scan.setFilter(fl); + verifyScanFull(scan, expected); + + // To test multi-version, put one new kv + // r1. 'a:10', '1' + Put kv2 = new Put(r1).setDurability(Durability.SKIP_WAL); + byte[] v1 = Bytes.toBytes("1"); + kv2.addColumn(FAMILY_A, q10, v1); + region.put(kv2); + region.flush(true); + + // 3. SCVF, filterIfMissing:false, latestVersionOnly:false, + // vNil is older version. + scvf.setFilterIfMissing(false); + scvf.setLatestVersionOnly(false); + fl = new FilterList(new Filter[] {scvf, ff}); + scan.setFilter(fl); + verifyScanFull(scan, expected); + + // 4. SCVF, filterIfMissing:true, latestVersionOnly:false + scvf.setFilterIfMissing(true); + scvf.setLatestVersionOnly(false); + fl = new FilterList(new Filter[] {scvf, ff}); + scan.setFilter(fl); + verifyScanFull(scan, expected); + + // 5. Test scan multi-version, and set FamilyFilter('a') to see effects + ff = new FamilyFilter(CompareOperator.EQUAL, new BinaryComparator(FAMILY_A)); + fl = new FilterList(new Filter[] {scvf, ff}); + scan.setFilter(fl); // without multi-version + KeyValue[] expected2 = { + new KeyValue(r1, FAMILY_A, q1, vNil), + new KeyValue(r1, FAMILY_A, q10, v1) }; + verifyScanFull(scan, expected2); + scan.setFilter(fl).readVersions(3); // with multi-version + KeyValue[] expected3 = { + new KeyValue(r1, FAMILY_A, q1, vNil), + new KeyValue(r1, FAMILY_A, q10, v1), + new KeyValue(r1, FAMILY_A, q10, vNil)}; + verifyScanFull(scan, expected3); + } + @Test public void testColumnValueFilter() throws Exception { // Prepare test rows: 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 b2fe9d0c9f..7a90204b63 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 @@ -1019,5 +1019,49 @@ public class TestFilterList { Assert.assertEquals(true, filter2.getTransformed()); Assert.assertEquals(true, filter3.getTransformed()); } + + @Test + public void testTransformReturnCode() throws IOException { + KeyValue kv1 = new KeyValue( + Bytes.toBytes("1"), Bytes.toBytes("a"), Bytes.toBytes("1"), Bytes.toBytes("")); + KeyValue kv2 = new KeyValue( + Bytes.toBytes("1"), Bytes.toBytes("a"), Bytes.toBytes("10"), Bytes.toBytes("")); + KeyValue kv3 = new KeyValue( + Bytes.toBytes("1"), Bytes.toBytes("b"), Bytes.toBytes("2"), Bytes.toBytes("")); + + FilterList andFilterList = new FilterList(Operator.MUST_PASS_ALL); + SingleColumnValueFilter scvf = new SingleColumnValueFilter( + Bytes.toBytes("a"), Bytes.toBytes("10"), + CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes(""))); + FamilyFilter ff = new FamilyFilter( + CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("b"))); + andFilterList.addFilter(scvf); + andFilterList.addFilter(ff); + + // Against kv1. + Assert.assertEquals(ReturnCode.INCLUDE, scvf.filterCell(kv1)); + Assert.assertEquals(ReturnCode.INCLUDE, + scvf.transformReturnCode(FilterListBase.LOGIC.AND, ReturnCode.INCLUDE)); + Assert.assertEquals(ReturnCode.NEXT_ROW, ff.filterCell(kv1)); + Assert.assertEquals(ReturnCode.SKIP, + ff.transformReturnCode(FilterListBase.LOGIC.AND, ReturnCode.NEXT_ROW)); + Assert.assertEquals(ReturnCode.SKIP, andFilterList.filterCell(kv1)); + // Against kv2. + Assert.assertEquals(ReturnCode.INCLUDE, scvf.filterCell(kv2)); + Assert.assertEquals(ReturnCode.INCLUDE, + scvf.transformReturnCode(FilterListBase.LOGIC.AND, ReturnCode.INCLUDE)); + Assert.assertEquals(ReturnCode.NEXT_ROW, ff.filterCell(kv2)); + Assert.assertEquals(ReturnCode.SKIP, + ff.transformReturnCode(FilterListBase.LOGIC.AND, ReturnCode.NEXT_ROW)); + Assert.assertEquals(ReturnCode.SKIP, andFilterList.filterCell(kv2)); + // Against kv3. + Assert.assertEquals(ReturnCode.INCLUDE, scvf.filterCell(kv3)); + Assert.assertEquals(ReturnCode.INCLUDE, + scvf.transformReturnCode(FilterListBase.LOGIC.AND, ReturnCode.INCLUDE)); + Assert.assertEquals(ReturnCode.INCLUDE, ff.filterCell(kv3)); + Assert.assertEquals(ReturnCode.INCLUDE, + ff.transformReturnCode(FilterListBase.LOGIC.AND, ReturnCode.INCLUDE)); + Assert.assertEquals(ReturnCode.INCLUDE, andFilterList.filterCell(kv3)); + } } -- 2.15.0