From 2e16a865a46871e894efd5786f4d5671955ef6d9 Mon Sep 17 00:00:00 2001 From: Peter Somogyi Date: Fri, 6 Oct 2017 09:26:42 -0500 Subject: [PATCH 1/6] HBASE-18957 establish filter test that shows current behavior of OR operator on FamilyFilters. Amending-Author: Sean Busbey --- .../hadoop/hbase/filter/TestFilterListOnMini.java | 88 ++++++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 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 new file mode 100644 index 0000000000..dd2399f910 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOnMini.java @@ -0,0 +1,88 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.testclassification.FilterTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +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. + */ +@Category({MediumTests.class, FilterTests.class}) +public class TestFilterListOnMini { + + private static final Log LOG = LogFactory.getLog(TestFilterListOnMini.class); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + @Rule + public TestName name = new TestName(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.startMiniCluster(1); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testFiltersWithOR() throws Exception { + TableName tn = TableName.valueOf(name.getMethodName()); + Table table = TEST_UTIL.createTable(tn, new String[] { "cf1", "cf2" }); + byte[] CF1 = Bytes.toBytes("cf1"); + byte[] CF2 = Bytes.toBytes("cf2"); + Put put1 = new Put(Bytes.toBytes("0")); + put1.addColumn(CF1, Bytes.toBytes("col_a"), Bytes.toBytes(0)); + table.put(put1); + Put put2 = new Put(Bytes.toBytes("0")); + put2.addColumn(CF2, Bytes.toBytes("col_b"), Bytes.toBytes(0)); + table.put(put2); + FamilyFilter filterCF1 = + new FamilyFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator(CF1)); + FamilyFilter filterCF2 = + new FamilyFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator(CF2)); + FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); + filterList.addFilter(filterCF1); + filterList.addFilter(filterCF2); + Scan scan = new Scan(); + scan.setFilter(filterList); + ResultScanner scanner = table.getScanner(scan); + LOG.info("Filter list: " + filterList); + for (Result rr = scanner.next(); rr != null; rr = scanner.next()) { + Assert.assertEquals(2, rr.size()); + } + } +} -- 2.14.1 From fa0df08dd8c9be45037ae3f2b38a8f6824a150ce Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 6 Oct 2017 11:19:00 -0500 Subject: [PATCH 2/6] Revert "HBASE-18904 Missing break in NEXT_ROW case of FilterList#mergeReturnCodeForOrOperator()" This reverts commit 3bd824facadd40dabb06e19ccad911be0ea6915a. Backing out filterlist regression, see HBASE-18957. Work continuing branch for HBASE-18410. --- .../src/main/java/org/apache/hadoop/hbase/filter/FilterList.java | 2 -- 1 file changed, 2 deletions(-) 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 033ca83ff4..7f2405d665 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 @@ -566,7 +566,6 @@ final public class FilterList extends FilterBase { if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) { return ReturnCode.NEXT_ROW; } - break; case SEEK_NEXT_USING_HINT: if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { @@ -578,7 +577,6 @@ final public class FilterList extends FilterBase { if (isInReturnCodes(rc, ReturnCode.SEEK_NEXT_USING_HINT)) { return ReturnCode.SEEK_NEXT_USING_HINT; } - break; } throw new IllegalStateException( "Received code is not valid. rc: " + rc + ", localRC: " + localRC); -- 2.14.1 From ce2db9ab54f517baffc65e38042c5840b0b6ed25 Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 6 Oct 2017 11:19:39 -0500 Subject: [PATCH 3/6] Revert "HBASE-18160 Fix incorrect logic in FilterList.filterKeyValue" This reverts commit f54cc1ca51440ee0f445a008f2f31697730fd7fe. Backing out filterlist regression, see HBASE-18957. Work continuing branch for HBASE-18410. --- .../org/apache/hadoop/hbase/filter/FilterList.java | 542 +++++++-------------- .../apache/hadoop/hbase/filter/TestFilterList.java | 146 ++---- 2 files changed, 219 insertions(+), 469 deletions(-) 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 7f2405d665..1e80a7e6e5 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 @@ -90,53 +90,62 @@ final public class FilterList extends FilterBase { private Cell transformedCell = null; /** - * Constructor that takes a set of {@link Filter}s and an operator. - * @param operator Operator to process filter set with. - * @param rowFilters Set of row filters. - */ - public FilterList(final Operator operator, final List rowFilters) { - reversed = checkAndGetReversed(rowFilters, reversed); - this.filters = new ArrayList<>(rowFilters); - this.operator = operator; - initPrevListForMustPassOne(rowFilters.size()); - } - - /** - * Constructor that takes a set of {@link Filter}s. The default operator MUST_PASS_ALL is assumed. + * Constructor that takes a set of {@link Filter}s. The default operator + * MUST_PASS_ALL is assumed. * All filters are cloned to internal list. * @param rowFilters list of filters */ public FilterList(final List rowFilters) { - this(Operator.MUST_PASS_ALL, rowFilters); + reversed = getReversed(rowFilters, reversed); + this.filters = new ArrayList<>(rowFilters); + initPrevListForMustPassOne(rowFilters.size()); } /** - * Constructor that takes a var arg number of {@link Filter}s. The default operator MUST_PASS_ALL - * is assumed. + * Constructor that takes a var arg number of {@link Filter}s. The fefault operator + * MUST_PASS_ALL is assumed. * @param rowFilters */ public FilterList(final Filter... rowFilters) { - this(Operator.MUST_PASS_ALL, Arrays.asList(rowFilters)); + this(Arrays.asList(rowFilters)); } /** * Constructor that takes an operator. + * * @param operator Operator to process filter set with. */ public FilterList(final Operator operator) { - this(operator, new ArrayList<>()); + this.operator = operator; + this.filters = new ArrayList<>(); + initPrevListForMustPassOne(filters.size()); + } + + /** + * Constructor that takes a set of {@link Filter}s and an operator. + * + * @param operator Operator to process filter set with. + * @param rowFilters Set of row filters. + */ + public FilterList(final Operator operator, final List rowFilters) { + this(rowFilters); + this.operator = operator; + initPrevListForMustPassOne(rowFilters.size()); } /** * Constructor that takes a var arg number of {@link Filter}s and an operator. + * * @param operator Operator to process filter set with. * @param rowFilters Filters to use */ public FilterList(final Operator operator, final Filter... rowFilters) { - this(operator, Arrays.asList(rowFilters)); + this(rowFilters); + this.operator = operator; + initPrevListForMustPassOne(rowFilters.length); } - private void initPrevListForMustPassOne(int size) { + public void initPrevListForMustPassOne(int size) { if (operator == Operator.MUST_PASS_ONE) { if (this.prevFilterRCList == null) { prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); @@ -147,8 +156,10 @@ final public class FilterList extends FilterBase { } } + /** * Get the operator. + * * @return operator */ public Operator getOperator() { @@ -157,6 +168,7 @@ final public class FilterList extends FilterBase { /** * Get the filters. + * * @return filters */ public List getFilters() { @@ -171,22 +183,33 @@ final public class FilterList extends FilterBase { return filters.isEmpty(); } - private static boolean checkAndGetReversed(List rowFilters, boolean defaultValue) { - if (rowFilters.isEmpty()) { - return defaultValue; + private static boolean getReversed(List rowFilters, boolean defaultValue) { + boolean rval = defaultValue; + boolean isFirst = true; + for (Filter f : rowFilters) { + if (isFirst) { + rval = f.isReversed(); + isFirst = false; + continue; + } + if (rval != f.isReversed()) { + throw new IllegalArgumentException("Filters in the list must have the same reversed flag"); + } } - Boolean retValue = rowFilters.get(0).isReversed(); - boolean allEqual = rowFilters.stream().map(Filter::isReversed).allMatch(retValue::equals); - if (!allEqual) { - throw new IllegalArgumentException("Filters in the list must have the same reversed flag"); + return rval; + } + private static void checkReversed(List rowFilters, boolean expected) { + for (Filter filter : rowFilters) { + if (expected != filter.isReversed()) { + throw new IllegalArgumentException( + "Filters in the list must have the same reversed flag, expected=" + + expected); + } } - return retValue; } public void addFilter(List filters) { - if (checkAndGetReversed(filters, isReversed()) != isReversed()) { - throw new IllegalArgumentException("Filters in the list must have the same reversed flag"); - } + checkReversed(filters, isReversed()); this.filters.addAll(filters); if (operator == Operator.MUST_PASS_ONE) { this.prevFilterRCList.addAll(Collections.nCopies(filters.size(), null)); @@ -196,6 +219,7 @@ final public class FilterList extends FilterBase { /** * Add a filter. + * * @param filter another filter */ public void addFilter(Filter filter) { @@ -204,7 +228,8 @@ final public class FilterList extends FilterBase { @Override public void reset() throws IOException { - for (int i = 0, n = filters.size(); i < n; i++) { + 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); @@ -220,15 +245,18 @@ final public class FilterList extends FilterBase { return super.filterRowKey(rowKey, offset, length); } boolean flag = this.operator == Operator.MUST_PASS_ONE; - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { Filter filter = filters.get(i); if (this.operator == Operator.MUST_PASS_ALL) { - if (filter.filterAllRemaining() || filter.filterRowKey(rowKey, offset, length)) { - flag = true; + if (filter.filterAllRemaining() || + filter.filterRowKey(rowKey, offset, length)) { + flag = true; } } else if (this.operator == Operator.MUST_PASS_ONE) { - if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey, offset, length)) { - flag = false; + if (!filter.filterAllRemaining() && + !filter.filterRowKey(rowKey, offset, length)) { + flag = false; } } } @@ -241,7 +269,8 @@ final public class FilterList extends FilterBase { return super.filterRowKey(firstRowCell); } boolean flag = this.operator == Operator.MUST_PASS_ONE; - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { Filter filter = filters.get(i); if (this.operator == Operator.MUST_PASS_ALL) { if (filter.filterAllRemaining() || filter.filterRowKey(firstRowCell)) { @@ -261,7 +290,8 @@ final public class FilterList extends FilterBase { if (isEmpty()) { return super.filterAllRemaining(); } - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { if (filters.get(i).filterAllRemaining()) { if (operator == Operator.MUST_PASS_ALL) { return true; @@ -281,15 +311,10 @@ final public class FilterList extends FilterBase { return super.transformCell(c); } if (!CellUtil.equals(c, referenceCell)) { - throw new IllegalStateException( - "Reference Cell: " + this.referenceCell + " does not match: " + c); + throw new IllegalStateException("Reference Cell: " + this.referenceCell + " does not match: " + + c); } - // Copy transformedCell into a new cell and reset transformedCell & referenceCell to null for - // Java GC optimization - Cell cell = KeyValueUtil.copyToNewKeyValue(this.transformedCell); - this.transformedCell = null; - this.referenceCell = null; - return cell; + return this.transformedCell; } /** @@ -327,316 +352,116 @@ final public class FilterList extends FilterBase { } } - /** - * FilterList with MUST_PASS_ALL choose the maximal forward step among sub-filters in filter list. - * Let's call it: The Maximal Step Rule. So if filter-A in filter list return INCLUDE and filter-B - * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should return - * INCLUDE_AND_NEXT_COL. For SEEK_NEXT_USING_HINT, it's more special, and in method - * filterKeyValueWithMustPassAll(), if any sub-filter return SEEK_NEXT_USING_HINT, then our filter - * list will return SEEK_NEXT_USING_HINT. so we don't care about the SEEK_NEXT_USING_HINT here.
- *
- * The jump step will be: - * - *
-   * INCLUDE < SKIP < INCLUDE_AND_NEXT_COL < NEXT_COL < INCLUDE_AND_SEEK_NEXT_ROW < NEXT_ROW < SEEK_NEXT_USING_HINT
-   * 
- * - * Here, we have the following map to describe The Maximal Step Rule. if current return code (for - * previous sub-filters in filter list) is ReturnCode, and current filter returns - * localRC, then we should return map[ReturnCode][localRC] for the merged result, - * according to The Maximal Step Rule.
- * - *
-   * LocalCode\ReturnCode       INCLUDE                    INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  SKIP                  NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * INCLUDE                    INCLUDE                    INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  SKIP                  NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * INCLUDE_AND_NEXT_COL       INCLUDE_AND_NEXT_COL       INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  NEXT_COL              NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW  NEXT_ROW              NEXT_ROW              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * SKIP                       SKIP                       NEXT_COL                  NEXT_ROW                   SKIP                  NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * NEXT_COL                   NEXT_COL                   NEXT_COL                  NEXT_ROW                   NEXT_COL              NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * NEXT_ROW                   NEXT_ROW                   NEXT_ROW                  NEXT_ROW                   NEXT_ROW              NEXT_ROW              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT      SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT
-   * 
- * @param rc Return code which is calculated by previous sub-filter(s) in filter list. - * @param localRC Return code of the current sub-filter in filter list. - * @return Return code which is merged by the return code of previous sub-filter(s) and the return - * code of current sub-filter. - */ - private ReturnCode mergeReturnCodeForAndOperator(ReturnCode rc, ReturnCode localRC) { - if (rc == ReturnCode.SEEK_NEXT_USING_HINT || localRC == ReturnCode.SEEK_NEXT_USING_HINT) { - return ReturnCode.SEEK_NEXT_USING_HINT; - } - switch (localRC) { - case INCLUDE: - return rc; - case INCLUDE_AND_NEXT_COL: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL)) { - return ReturnCode.INCLUDE_AND_NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW; - } - if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL)) { - return ReturnCode.NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) { - return ReturnCode.NEXT_ROW; - } - break; - case INCLUDE_AND_SEEK_NEXT_ROW: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, - ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW; - } - if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW)) { - return ReturnCode.NEXT_ROW; - } - break; - case SKIP: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP)) { - return ReturnCode.SKIP; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.NEXT_COL)) { - return ReturnCode.NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, ReturnCode.NEXT_ROW)) { - return ReturnCode.NEXT_ROW; - } - break; - case NEXT_COL: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.SKIP, - ReturnCode.NEXT_COL)) { - return ReturnCode.NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, ReturnCode.NEXT_ROW)) { - return ReturnCode.NEXT_ROW; - } - break; - case NEXT_ROW: - return ReturnCode.NEXT_ROW; - } - throw new IllegalStateException("Received code is not valid. rc: " + rc + ", localRC: " - + localRC); - } - - private ReturnCode filterKeyValueWithMustPassAll(Cell c) throws IOException { - ReturnCode rc = ReturnCode.INCLUDE; - Cell transformed = c; - this.seekHintFilter.clear(); - for (int i = 0, n = filters.size(); i < n; i++) { - Filter filter = filters.get(i); - if (filter.filterAllRemaining()) { - return ReturnCode.NEXT_ROW; - } - ReturnCode localRC = filter.filterKeyValue(c); - rc = mergeReturnCodeForAndOperator(rc, localRC); - - // For INCLUDE* case, we need to update the transformed cell. - if (isInReturnCodes(localRC, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, - ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - transformed = filter.transformCell(transformed); - } - if (localRC == ReturnCode.SEEK_NEXT_USING_HINT) { - seekHintFilter.add(filter); - } - } - this.transformedCell = transformed; - if (!seekHintFilter.isEmpty()) { - return ReturnCode.SEEK_NEXT_USING_HINT; - } - return rc; - } - - private void updatePrevFilterRCList(int index, ReturnCode currentRC) { - prevFilterRCList.set(index, currentRC); - } - - private void updatePrevCellList(int index, Cell currentCell, ReturnCode currentRC) { - if (currentCell == null || currentRC == ReturnCode.INCLUDE || currentRC == ReturnCode.SKIP) { - // If previous return code is INCLUDE or SKIP, we should always pass the next cell to the - // corresponding sub-filter(need not test shouldPassCurrentCellToFilter() method), So we - // need not save current cell to prevCellList for saving heap memory. - prevCellList.set(index, null); - } else { - prevCellList.set(index, KeyValueUtil.toNewKeyCell(currentCell)); - } - } - - private static boolean isInReturnCodes(ReturnCode testRC, ReturnCode... returnCodes) { - return Arrays.stream(returnCodes).anyMatch(testRC::equals); - } - - /** - * FilterList with MUST_PASS_ONE choose the minimal forward step among sub-filter in filter list. - * Let's call it: The Minimal Step Rule. So if filter-A in filter list return INCLUDE and filter-B - * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should return INCLUDE. For - * SEEK_NEXT_USING_HINT, it's more special, because we do not know how far it will forward, so we - * use SKIP by default.
- *
- * The jump step will be: - * - *
-   * INCLUDE < SKIP < INCLUDE_AND_NEXT_COL < NEXT_COL < INCLUDE_AND_SEEK_NEXT_ROW < NEXT_ROW < SEEK_NEXT_USING_HINT
-   * 
- * - * Here, we have the following map to describe The Minimal Step Rule. if current return code (for - * previous sub-filters in filter list) is ReturnCode, and current filter returns - * localRC, then we should return map[ReturnCode][localRC] for the merged result, - * according to The Minimal Step Rule.
- * - *
-   * LocalCode\ReturnCode       INCLUDE INCLUDE_AND_NEXT_COL     INCLUDE_AND_SEEK_NEXT_ROW  SKIP      NEXT_COL              NEXT_ROW                  SEEK_NEXT_USING_HINT
-   * INCLUDE                    INCLUDE INCLUDE                  INCLUDE                    INCLUDE   INCLUDE               INCLUDE                   INCLUDE
-   * INCLUDE_AND_NEXT_COL       INCLUDE INCLUDE_AND_NEXT_COL     INCLUDE_AND_NEXT_COL       INCLUDE   INCLUDE_AND_NEXT_COL  INCLUDE_AND_NEXT_COL      INCLUDE
-   * INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE INCLUDE_AND_NEXT_COL     INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE   INCLUDE_AND_NEXT_COL  INCLUDE_AND_SEEK_NEXT_ROW INCLUDE
-   * SKIP                       INCLUDE INCLUDE                  INCLUDE                    SKIP      SKIP                  SKIP                      SKIP
-   * NEXT_COL                   INCLUDE INCLUDE_AND_NEXT_COL     INCLUDE_AND_NEXT_COL       SKIP      NEXT_COL              NEXT_COL                  SKIP
-   * NEXT_ROW                   INCLUDE INCLUDE_AND_NEXT_COL     INCLUDE_AND_SEEK_NEXT_ROW  SKIP      NEXT_COL              NEXT_ROW                  SKIP
-   * SEEK_NEXT_USING_HINT       INCLUDE INCLUDE                  INCLUDE                    SKIP      SKIP                  SKIP                      SEEK_NEXT_USING_HINT
-   * 
- * @param rc Return code which is calculated by previous sub-filter(s) in filter list. - * @param localRC Return code of the current sub-filter in filter list. - * @return Return code which is merged by the return code of previous sub-filter(s) and the return - * code of current sub-filter. - */ - private ReturnCode mergeReturnCodeForOrOperator(ReturnCode rc, ReturnCode localRC) { - if (rc == null) return localRC; - switch (localRC) { - case INCLUDE: + @Override + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH", + justification="Intentional") + public ReturnCode filterKeyValue(Cell c) throws IOException { + if (isEmpty()) { return ReturnCode.INCLUDE; - case INCLUDE_AND_NEXT_COL: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP, - ReturnCode.SEEK_NEXT_USING_HINT)) { - return ReturnCode.INCLUDE; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, - ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW)) { - return ReturnCode.INCLUDE_AND_NEXT_COL; - } - break; - case INCLUDE_AND_SEEK_NEXT_ROW: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP, - ReturnCode.SEEK_NEXT_USING_HINT)) { - return ReturnCode.INCLUDE; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.NEXT_COL)) { - return ReturnCode.INCLUDE_AND_NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, ReturnCode.NEXT_ROW)) { - return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW; - } - break; - case SKIP: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, - ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - return ReturnCode.INCLUDE; - } - if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW, - ReturnCode.SEEK_NEXT_USING_HINT)) { - return ReturnCode.SKIP; - } - break; - case NEXT_COL: - if (isInReturnCodes(rc, ReturnCode.INCLUDE)) { - return ReturnCode.INCLUDE; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, - ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - return ReturnCode.INCLUDE_AND_NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.SEEK_NEXT_USING_HINT)) { - return ReturnCode.SKIP; - } - break; - case NEXT_ROW: - if (isInReturnCodes(rc, ReturnCode.INCLUDE)) { - return ReturnCode.INCLUDE; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL)) { - return ReturnCode.INCLUDE_AND_NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW; - } - if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.SEEK_NEXT_USING_HINT)) { - return ReturnCode.SKIP; - } - if (isInReturnCodes(rc, ReturnCode.NEXT_COL)) { - return ReturnCode.NEXT_COL; - } - if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) { - return ReturnCode.NEXT_ROW; - } - case SEEK_NEXT_USING_HINT: - if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, - ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - return ReturnCode.INCLUDE; - } - if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW)) { - return ReturnCode.SKIP; - } - if (isInReturnCodes(rc, ReturnCode.SEEK_NEXT_USING_HINT)) { - return ReturnCode.SEEK_NEXT_USING_HINT; - } } - throw new IllegalStateException( - "Received code is not valid. rc: " + rc + ", localRC: " + localRC); - } + this.referenceCell = c; + seekHintFilter.clear(); - private ReturnCode filterKeyValueWithMustPassOne(Cell c) throws IOException { - ReturnCode rc = null; - boolean everyFilterReturnHint = true; + // Accumulates successive transformation of every filter that includes the Cell: Cell transformed = c; - for (int i = 0, n = filters.size(); i < n; i++) { - Filter filter = filters.get(i); - - Cell prevCell = this.prevCellList.get(i); - if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, i)) { - everyFilterReturnHint = false; - continue; - } - ReturnCode localRC = filter.filterKeyValue(c); - - // Update previous return code and previous cell for filter[i]. - updatePrevFilterRCList(i, localRC); - updatePrevCellList(i, c, localRC); - - if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { - everyFilterReturnHint = false; - } + ReturnCode rc = operator == Operator.MUST_PASS_ONE? + ReturnCode.SKIP: ReturnCode.INCLUDE; + int listize = filters.size(); + /* + * When all filters in a MUST_PASS_ONE FilterList return a SEEK_USING_NEXT_HINT code, + * we should return SEEK_NEXT_USING_HINT from the FilterList to utilize the lowest seek value. + * + * The following variable tracks whether any of the Filters returns ReturnCode other than + * SEEK_NEXT_USING_HINT for MUST_PASS_ONE FilterList, in which case the optimization would + * be skipped. + */ + boolean seenNonHintReturnCode = false; + for (int i = 0; i < listize; i++) { + Filter filter = filters.get(i); + if (operator == Operator.MUST_PASS_ALL) { + if (filter.filterAllRemaining()) { + return ReturnCode.NEXT_ROW; + } + ReturnCode code = filter.filterKeyValue(c); + switch (code) { + // Override INCLUDE and continue to evaluate. + case INCLUDE_AND_NEXT_COL: + rc = ReturnCode.INCLUDE_AND_NEXT_COL; // FindBugs SF_SWITCH_FALLTHROUGH + case INCLUDE: + transformed = filter.transformCell(transformed); + continue; + case SEEK_NEXT_USING_HINT: + seekHintFilter.add(filter); + continue; + default: + if (seekHintFilter.isEmpty()) { + return code; + } + } + } else if (operator == Operator.MUST_PASS_ONE) { + Cell prevCell = this.prevCellList.get(i); + if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, i)) { + seenNonHintReturnCode = true; + continue; + } - rc = mergeReturnCodeForOrOperator(rc, localRC); + ReturnCode localRC = filter.filterKeyValue(c); + // Update previous cell and return code we encountered. + prevFilterRCList.set(i, localRC); + if (c == null || localRC == ReturnCode.INCLUDE || localRC == ReturnCode.SKIP) { + // If previous return code is INCLUDE or SKIP, we should always pass the next cell to the + // corresponding sub-filter(need not test shouldPassCurrentCellToFilter() method), So we + // need not save current cell to prevCellList for saving heap memory. + prevCellList.set(i, null); + } else { + prevCellList.set(i, KeyValueUtil.toNewKeyCell(c)); + } - // For INCLUDE* case, we need to update the transformed cell. - if (isInReturnCodes(localRC, ReturnCode.INCLUDE, ReturnCode.INCLUDE_AND_NEXT_COL, - ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) { - transformed = filter.transformCell(transformed); + if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { + seenNonHintReturnCode = true; + } + switch (localRC) { + case INCLUDE: + if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) { + rc = ReturnCode.INCLUDE; + } + transformed = filter.transformCell(transformed); + break; + case INCLUDE_AND_NEXT_COL: + rc = ReturnCode.INCLUDE_AND_NEXT_COL; + transformed = filter.transformCell(transformed); + // must continue here to evaluate all filters + break; + case NEXT_ROW: + break; + case SKIP: + break; + case NEXT_COL: + break; + case SEEK_NEXT_USING_HINT: + break; + default: + throw new IllegalStateException("Received code is not valid."); + } } } - this.transformedCell = transformed; - if (everyFilterReturnHint) { + if (!seekHintFilter.isEmpty()) { return ReturnCode.SEEK_NEXT_USING_HINT; - } else if (rc == null) { - // Each sub-filter in filter list got true for filterAllRemaining(). - return ReturnCode.SKIP; - } else { - return rc; } - } - @Override - public ReturnCode filterKeyValue(Cell c) throws IOException { - if (isEmpty()) { - return ReturnCode.INCLUDE; - } - this.referenceCell = c; + // Save the transformed Cell for transform(): + this.transformedCell = transformed; - if (operator == Operator.MUST_PASS_ALL) { - return filterKeyValueWithMustPassAll(c); - } else { - return filterKeyValueWithMustPassOne(c); + /* + * The seenNonHintReturnCode flag is intended only for Operator.MUST_PASS_ONE branch. + * If we have seen non SEEK_NEXT_USING_HINT ReturnCode, respect that ReturnCode. + */ + if (operator == Operator.MUST_PASS_ONE && !seenNonHintReturnCode) { + return ReturnCode.SEEK_NEXT_USING_HINT; } + return rc; } /** @@ -647,14 +472,16 @@ final public class FilterList extends FilterBase { */ @Override public void filterRowCells(List cells) throws IOException { - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { filters.get(i).filterRowCells(cells); } } @Override public boolean hasFilterRow() { - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { if (filters.get(i).hasFilterRow()) { return true; } @@ -667,7 +494,8 @@ final public class FilterList extends FilterBase { if (isEmpty()) { return super.filterRow(); } - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { Filter filter = filters.get(i); if (operator == Operator.MUST_PASS_ALL) { if (filter.filterRow()) { @@ -679,16 +507,18 @@ final public class FilterList extends FilterBase { } } } - return operator == Operator.MUST_PASS_ONE; + return operator == Operator.MUST_PASS_ONE; } /** * @return The filter serialized using pb */ public byte[] toByteArray() throws IOException { - FilterProtos.FilterList.Builder builder = FilterProtos.FilterList.newBuilder(); + FilterProtos.FilterList.Builder builder = + FilterProtos.FilterList.newBuilder(); builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name())); - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { builder.addFilters(ProtobufUtil.toFilter(filters.get(i))); } return builder.build().toByteArray(); @@ -701,7 +531,7 @@ final public class FilterList extends FilterBase { * @see #toByteArray */ public static FilterList parseFrom(final byte [] pbBytes) - throws DeserializationException { + throws DeserializationException { FilterProtos.FilterList proto; try { proto = FilterProtos.FilterList.parseFrom(pbBytes); @@ -712,13 +542,14 @@ final public class FilterList extends FilterBase { List rowFilters = new ArrayList<>(proto.getFiltersCount()); try { List filtersList = proto.getFiltersList(); - for (int i = 0, n = filtersList.size(); i < n; i++) { + int listSize = filtersList.size(); + for (int i = 0; i < listSize; i++) { rowFilters.add(ProtobufUtil.toFilter(filtersList.get(i))); } } catch (IOException ioe) { throw new DeserializationException(ioe); } - return new FilterList(Operator.valueOf(proto.getOperator().name()), rowFilters); + return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters); } /** @@ -758,7 +589,8 @@ final public class FilterList extends FilterBase { } // If any condition can pass, we need to keep the min hint - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { if (filters.get(i).filterAllRemaining()) { continue; } @@ -784,7 +616,8 @@ final public class FilterList extends FilterBase { if (isEmpty()) { return super.isFamilyEssential(name); } - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { if (filters.get(i).isFamilyEssential(name)) { return true; } @@ -794,7 +627,8 @@ final public class FilterList extends FilterBase { @Override public void setReversed(boolean reversed) { - for (int i = 0, n = filters.size(); i < n; i++) { + int listize = filters.size(); + for (int i = 0; i < listize; i++) { filters.get(i).setReversed(reversed); } this.reversed = reversed; 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 a57171264d..f20a9ba915 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 @@ -26,9 +26,11 @@ import java.util.Arrays; import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; 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; @@ -43,9 +45,18 @@ import static org.junit.Assert.fail; import org.junit.Test; import org.junit.experimental.categories.Category; +/** + * Tests filter sets + * + */ @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 { @@ -299,23 +310,23 @@ public class TestFilterList { FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE); flist.addFilter(new PrefixFilter(r1)); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1))); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11))); + assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE); + assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE); flist.reset(); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2)); - assertEquals(ReturnCode.SKIP, flist.filterKeyValue(new KeyValue(r2, r2, r2))); + 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(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1))); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11))); + assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE); + assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE); flist.reset(); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2)); - assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP); } /** @@ -333,12 +344,12 @@ public class TestFilterList { flist.addFilter(new AlwaysNextColFilter()); flist.addFilter(new InclusiveStopFilter(r1)); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1))); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11))); + assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE); + assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE); flist.reset(); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2)); - assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP); } public static class AlwaysNextColFilter extends FilterBase { @@ -420,7 +431,7 @@ 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(ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpOnefilterList.filterKeyValue(null)); // INCLUDE, SKIP, INCLUDE. assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null)); @@ -596,16 +607,16 @@ public class TestFilterList { 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)); + 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(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv1)); - assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv2)); - assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv3)); + assertEquals(filterList11.filterKeyValue(kv1), ReturnCode.SKIP); + assertEquals(filterList11.filterKeyValue(kv2), ReturnCode.SKIP); + assertEquals(filterList11.filterKeyValue(kv3), ReturnCode.SKIP); } @Test @@ -623,10 +634,10 @@ public class TestFilterList { 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)); + 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 { @@ -699,101 +710,6 @@ public class TestFilterList { 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); - - 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)); } } -- 2.14.1 From 690494ff03c71961923fe978ba852154a610ab0a Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 6 Oct 2017 11:19:53 -0500 Subject: [PATCH 4/6] Revert "HBASE-15410 Utilize the max seek value when all Filters in MUST_PASS_ALL FilterList return SEEK_NEXT_USING_HINT" This reverts commit df34300cd3f89c1efdea43b0b2ecb64c317e1a34. Backing out filterlist regression, see HBASE-18957. Work continuing branch for HBASE-18410. --- .../org/apache/hadoop/hbase/filter/FilterList.java | 31 +++++----------------- .../apache/hadoop/hbase/filter/TestFilterList.java | 6 ++--- 2 files changed, 9 insertions(+), 28 deletions(-) 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 1e80a7e6e5..33d5b451c8 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 @@ -22,9 +22,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.Set; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -67,7 +65,7 @@ final public class FilterList extends FilterBase { private static final int MAX_LOG_FILTERS = 5; private Operator operator = Operator.MUST_PASS_ALL; private final List filters; - private Set seekHintFilter = new HashSet<>(); + private Filter seekHintFilter = null; /** * Save previous return code and previous cell for every filter in filter list. For MUST_PASS_ONE, @@ -236,7 +234,7 @@ final public class FilterList extends FilterBase { prevCellList.set(i, null); } } - seekHintFilter.clear(); + seekHintFilter = null; } @Override @@ -360,7 +358,6 @@ final public class FilterList extends FilterBase { return ReturnCode.INCLUDE; } this.referenceCell = c; - seekHintFilter.clear(); // Accumulates successive transformation of every filter that includes the Cell: Cell transformed = c; @@ -392,12 +389,10 @@ final public class FilterList extends FilterBase { transformed = filter.transformCell(transformed); continue; case SEEK_NEXT_USING_HINT: - seekHintFilter.add(filter); - continue; + seekHintFilter = filter; + return code; default: - if (seekHintFilter.isEmpty()) { - return code; - } + return code; } } else if (operator == Operator.MUST_PASS_ONE) { Cell prevCell = this.prevCellList.get(i); @@ -447,10 +442,6 @@ final public class FilterList extends FilterBase { } } - if (!seekHintFilter.isEmpty()) { - return ReturnCode.SEEK_NEXT_USING_HINT; - } - // Save the transformed Cell for transform(): this.transformedCell = transformed; @@ -574,17 +565,7 @@ final public class FilterList extends FilterBase { } Cell keyHint = null; if (operator == Operator.MUST_PASS_ALL) { - for (Filter filter : seekHintFilter) { - if (filter.filterAllRemaining()) continue; - Cell curKeyHint = filter.getNextCellHint(currentCell); - if (keyHint == null) { - keyHint = curKeyHint; - continue; - } - if (CellComparator.COMPARATOR.compare(keyHint, curKeyHint) < 0) { - keyHint = curKeyHint; - } - } + if (seekHintFilter != null) keyHint = seekHintFilter.getNextCellHint(currentCell); return keyHint; } 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 f20a9ba915..90f95a56bf 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 @@ -502,7 +502,7 @@ public class TestFilterList { FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } )); assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null), - minKeyValue)); + minKeyValue)); // Should have no hint if any filter has no hint filterList = new FilterList(Operator.MUST_PASS_ONE, @@ -526,7 +526,7 @@ public class TestFilterList { Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } )); filterList.filterKeyValue(null); assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null), - maxKeyValue)); + minKeyValue)); filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } )); @@ -540,7 +540,7 @@ public class TestFilterList { new Filter [] { filterNoHint, filterMinHint, filterMaxHint } )); filterList.filterKeyValue(null); assertEquals(0, CellComparator.COMPARATOR.compare(filterList.getNextCellHint(null), - maxKeyValue)); + minKeyValue)); filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } )); filterList.filterKeyValue(null); -- 2.14.1 From 892f370aefacfe617f2a73324d932a78b8074bea Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 6 Oct 2017 11:21:10 -0500 Subject: [PATCH 5/6] Revert "HBASE-17678 FilterList with MUST_PASS_ONE lead to redundancy cells returned - addendum" This reverts commit 0eeafa04f26f2388602a7c74b496e6682716f50a. Backing out filterlist regression, see HBASE-18957. Work continuing branch for HBASE-18410. --- .../main/java/org/apache/hadoop/hbase/filter/FilterList.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) 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 33d5b451c8..87dae1c70c 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 @@ -27,7 +27,6 @@ import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; @@ -145,7 +144,7 @@ final public class FilterList extends FilterBase { public void initPrevListForMustPassOne(int size) { if (operator == Operator.MUST_PASS_ONE) { - if (this.prevFilterRCList == null) { + if (this.prevCellList == null) { prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null)); } if (this.prevCellList == null) { @@ -404,14 +403,7 @@ final public class FilterList extends FilterBase { ReturnCode localRC = filter.filterKeyValue(c); // Update previous cell and return code we encountered. prevFilterRCList.set(i, localRC); - if (c == null || localRC == ReturnCode.INCLUDE || localRC == ReturnCode.SKIP) { - // If previous return code is INCLUDE or SKIP, we should always pass the next cell to the - // corresponding sub-filter(need not test shouldPassCurrentCellToFilter() method), So we - // need not save current cell to prevCellList for saving heap memory. - prevCellList.set(i, null); - } else { - prevCellList.set(i, KeyValueUtil.toNewKeyCell(c)); - } + prevCellList.set(i, c); if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { seenNonHintReturnCode = true; -- 2.14.1 From 07c09cc3b8a74abf83abb890ac1824ba5b42e65f Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 6 Oct 2017 11:21:30 -0500 Subject: [PATCH 6/6] Revert "HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned" This reverts commit 0d0c330401ade938bf934aafd79ec23705edcc60. Backing out filterlist regression, see HBASE-18957. Work continuing branch for HBASE-18410. --- .../org/apache/hadoop/hbase/filter/FilterList.java | 76 +------------ .../apache/hadoop/hbase/filter/TestFilterList.java | 117 --------------------- 2 files changed, 2 insertions(+), 191 deletions(-) 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 87dae1c70c..9c4da419b5 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 @@ -66,14 +66,6 @@ final public class FilterList extends FilterBase { private final List filters; private Filter seekHintFilter = null; - /** - * 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; @@ -95,7 +87,6 @@ final public class FilterList extends FilterBase { public FilterList(final List rowFilters) { reversed = getReversed(rowFilters, reversed); this.filters = new ArrayList<>(rowFilters); - initPrevListForMustPassOne(rowFilters.size()); } /** @@ -115,7 +106,6 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator) { this.operator = operator; this.filters = new ArrayList<>(); - initPrevListForMustPassOne(filters.size()); } /** @@ -127,7 +117,6 @@ final public class FilterList extends FilterBase { public FilterList(final Operator operator, final List rowFilters) { this(rowFilters); this.operator = operator; - initPrevListForMustPassOne(rowFilters.size()); } /** @@ -139,21 +128,8 @@ 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. * @@ -208,10 +184,6 @@ 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)); - } } /** @@ -228,10 +200,6 @@ 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); - } } seekHintFilter = null; } @@ -314,41 +282,6 @@ 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 - || CellComparator.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") @@ -394,17 +327,12 @@ final public class FilterList extends FilterBase { return code; } } else if (operator == Operator.MUST_PASS_ONE) { - Cell prevCell = this.prevCellList.get(i); - if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, i)) { + if (filter.filterAllRemaining()) { 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; } @@ -557,7 +485,7 @@ final public class FilterList extends FilterBase { } Cell keyHint = null; if (operator == Operator.MUST_PASS_ALL) { - if (seekHintFilter != null) keyHint = seekHintFilter.getNextCellHint(currentCell); + keyHint = seekHintFilter.getNextCellHint(currentCell); return keyHint; } 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 90f95a56bf..885c01ea64 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 @@ -26,7 +26,6 @@ import java.util.Arrays; import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -595,121 +594,5 @@ 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); - } } -- 2.14.1