diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 2f11472e634..3493082eacb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -67,6 +67,14 @@ final public class FilterList extends FilterBase { private final List filters; private Collection seekHintFilters = new ArrayList(); + /** + * Save previous return code and previous cell for every filter in filter list. For MUST_PASS_ONE, + * we use the previous return code to decide whether we should pass current cell encountered to + * the filter. For MUST_PASS_ALL, the two list are meaningless. + */ + private List prevFilterRCList = null; + private List prevCellList = null; + /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */ private Cell referenceCell = null; @@ -88,6 +96,7 @@ final public class FilterList extends FilterBase { public FilterList(final List rowFilters) { reversed = getReversed(rowFilters, reversed); this.filters = new ArrayList<>(rowFilters); + initPrevListForMustPassOne(rowFilters.size()); } /** @@ -107,6 +116,7 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator) { this.operator = operator; this.filters = new ArrayList<>(); + initPrevListForMustPassOne(filters.size()); } /** @@ -118,6 +128,7 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator, final List rowFilters) { this(rowFilters); this.operator = operator; + initPrevListForMustPassOne(rowFilters.size()); } /** @@ -129,8 +140,21 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator, final Filter... rowFilters) { this(rowFilters); this.operator = operator; + initPrevListForMustPassOne(rowFilters.length); } + public void initPrevListForMustPassOne(int size) { + if (operator == Operator.MUST_PASS_ONE) { + if (this.prevCellList == null) { + prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); + } + if (this.prevCellList == null) { + prevCellList = new ArrayList<>(Collections.nCopies(size, null)); + } + } + } + + /** * Get the operator. * @@ -185,6 +209,10 @@ final public class FilterList extends FilterBase { public void addFilter(List filters) { checkReversed(filters, isReversed()); this.filters.addAll(filters); + if (operator == Operator.MUST_PASS_ONE) { + this.prevFilterRCList.addAll(Collections.nCopies(filters.size(), null)); + this.prevCellList.addAll(Collections.nCopies(filters.size(), null)); + } } /** @@ -201,6 +229,10 @@ final public class FilterList extends FilterBase { int listize = filters.size(); for (int i = 0; i < listize; i++) { filters.get(i).reset(); + if (operator == Operator.MUST_PASS_ONE) { + prevFilterRCList.set(i, null); + prevCellList.set(i, null); + } } seekHintFilters.clear(); } @@ -283,6 +315,41 @@ final public class FilterList extends FilterBase { return this.transformedCell; } + /** + * For MUST_PASS_ONE, we cannot make sure that when filter-A in filter list return NEXT_COL then + * the next cell passing to filterList will be the first cell in next column, because if filter-B + * in filter list return SKIP, then the filter list will return SKIP. In this case, we should pass + * the cell following the previous cell, and it's possible that the next cell has the same column + * as the previous cell even if filter-A has NEXT_COL returned for the previous cell. So we should + * save the previous cell and the return code list when checking previous cell for every filter in + * filter list, and verify if currentCell fit the previous return code, if fit then pass the currentCell + * to the corresponding filter. (HBASE-17678) + */ + private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell currentCell, int filterIdx) + throws IOException { + ReturnCode prevCode = this.prevFilterRCList.get(filterIdx); + if (prevCell == null || prevCode == null) { + return true; + } + switch (prevCode) { + case INCLUDE: + case SKIP: + return true; + case SEEK_NEXT_USING_HINT: + Cell nextHintCell = getNextCellHint(prevCell); + return nextHintCell == null + || CellComparatorImpl.COMPARATOR.compare(currentCell, nextHintCell) >= 0; + case NEXT_COL: + case INCLUDE_AND_NEXT_COL: + return !CellUtil.matchingRowColumn(prevCell, currentCell); + case NEXT_ROW: + case INCLUDE_AND_SEEK_NEXT_ROW: + return !CellUtil.matchingRows(prevCell, currentCell); + default: + throw new IllegalStateException("Received code is not valid."); + } + } + @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH", justification="Intentional") @@ -331,12 +398,17 @@ final public class FilterList extends FilterBase { } } } else if (operator == Operator.MUST_PASS_ONE) { - if (filter.filterAllRemaining()) { + Cell prevCell = this.prevCellList.get(i); + if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, i)) { seenNonHintReturnCode = true; continue; } ReturnCode localRC = filter.filterKeyValue(c); + // Update previous cell and return code we encountered. + prevFilterRCList.set(i, localRC); + prevCellList.set(i, c); + if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { seenNonHintReturnCode = true; } 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 9ad09623952..46d44debad3 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,32 +18,34 @@ */ package org.apache.hadoop.hbase.filter; -import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparatorImpl; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; -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.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.testclassification.FilterTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.util.Bytes; 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 java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparatorImpl; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.filter.Filter.ReturnCode; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.testclassification.FilterTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + /** * Tests filter sets * @@ -593,5 +595,121 @@ public class TestFilterList { 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(filterList01.filterKeyValue(kv1), ReturnCode.INCLUDE_AND_NEXT_COL); + assertEquals(filterList01.filterKeyValue(kv2), ReturnCode.SKIP); + assertEquals(filterList01.filterKeyValue(kv3), ReturnCode.SKIP); + + FilterList filterList11 = + new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1)); + + assertEquals(filterList11.filterKeyValue(kv1), ReturnCode.SKIP); + assertEquals(filterList11.filterKeyValue(kv2), ReturnCode.SKIP); + assertEquals(filterList11.filterKeyValue(kv3), ReturnCode.SKIP); + } + + @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(filterList.filterKeyValue(kv1), ReturnCode.SEEK_NEXT_USING_HINT); + assertEquals(filterList.filterKeyValue(kv2), ReturnCode.SKIP); + assertEquals(filterList.filterKeyValue(kv3), ReturnCode.INCLUDE_AND_NEXT_COL); + assertEquals(filterList.filterKeyValue(kv4), ReturnCode.SKIP); + } + + 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); + + mockFilter.didCellPassToTheFilter = false; + filter.filterKeyValue(kv4); + assertTrue(mockFilter.didCellPassToTheFilter); + } }