From 36a55ce93f99180a1f3f81b4b25ff19696e94543 Mon Sep 17 00:00:00 2001 From: Peter Somogyi Date: Fri, 6 Oct 2017 09:26:42 -0500 Subject: [PATCH 1/2] 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 741346ee4567d5ff95e344b649ca597b16102a85 Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 6 Oct 2017 14:26:50 -0500 Subject: [PATCH 2/2] Revert "HBASE-17678 FilterList with MUST_PASS_ONE may lead to redundant cells returned" This reverts commit 256fc63007aecb63028b71ad1383d896f11db701. Backing out filterlist regression, see HBASE-18957. Work continuing branch for HBASE-18410. --- .../org/apache/hadoop/hbase/filter/FilterList.java | 82 +-------------- .../java/org/apache/hadoop/hbase/KeyValueUtil.java | 15 --- .../apache/hadoop/hbase/filter/TestFilterList.java | 117 +-------------------- 3 files changed, 3 insertions(+), 211 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 ddbd4a76aa..be22e5db06 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 @@ -21,10 +21,8 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; -import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; @@ -71,14 +69,6 @@ final public class FilterList extends Filter { private List filters = new ArrayList(); 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 referenceKV = null; @@ -103,7 +93,6 @@ final public class FilterList extends Filter { } else { this.filters = new ArrayList(rowFilters); } - initPrevListForMustPassOne(rowFilters.size()); } /** @@ -113,7 +102,6 @@ final public class FilterList extends Filter { */ public FilterList(final Filter... rowFilters) { this.filters = new ArrayList(Arrays.asList(rowFilters)); - initPrevListForMustPassOne(filters.size()); } /** @@ -123,7 +111,6 @@ final public class FilterList extends Filter { */ public FilterList(final Operator operator) { this.operator = operator; - initPrevListForMustPassOne(filters.size()); } /** @@ -135,7 +122,6 @@ final public class FilterList extends Filter { public FilterList(final Operator operator, final List rowFilters) { this.filters = new ArrayList(rowFilters); this.operator = operator; - initPrevListForMustPassOne(filters.size()); } /** @@ -147,18 +133,6 @@ final public class FilterList extends Filter { public FilterList(final Operator operator, final Filter... rowFilters) { this.filters = new ArrayList(Arrays.asList(rowFilters)); this.operator = operator; - initPrevListForMustPassOne(filters.size()); - } - - public void initPrevListForMustPassOne(int size) { - if (operator == Operator.MUST_PASS_ONE) { - if (this.prevFilterRCList == null) { - prevFilterRCList = new ArrayList(Collections.nCopies(size, (ReturnCode) null)); - } - if (this.prevCellList == null) { - prevCellList = new ArrayList(Collections.nCopies(size, (Cell) null)); - } - } } /** @@ -191,10 +165,6 @@ final public class FilterList extends Filter { + this.isReversed()); } this.filters.add(filter); - if (operator == Operator.MUST_PASS_ONE) { - this.prevFilterRCList.add((ReturnCode) null); - this.prevCellList.add((Cell) null); - } } @Override @@ -202,10 +172,6 @@ final public class FilterList extends Filter { 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; } @@ -276,38 +242,6 @@ final public class FilterList extends Filter { return KeyValueUtil.ensureKeyValue(this.transformedKV); } - /** - * 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 || KeyValue.COMPARATOR.compare(currentCell, nextHintCell) >= 0; - case NEXT_COL: - case INCLUDE_AND_NEXT_COL: - return !CellUtil.matchingRowColumn(prevCell, currentCell); - case NEXT_ROW: - return !CellUtil.matchingRow(prevCell, currentCell); - default: - throw new IllegalStateException("Received code is not valid."); - } - } @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH", @@ -351,24 +285,12 @@ final public class FilterList extends Filter { return code; } } else if (operator == Operator.MUST_PASS_ONE) { - Cell prevCell = this.prevCellList.get(i); - if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, v, i)) { + if (filter.filterAllRemaining()) { seenNonHintReturnCode = true; continue; } ReturnCode localRC = filter.filterKeyValue(v); - // Update previous cell and return code we encountered. - prevFilterRCList.set(i, localRC); - if (v == 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(v)); - } - if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) { seenNonHintReturnCode = true; } @@ -522,7 +444,7 @@ final public class FilterList extends Filter { public Cell getNextCellHint(Cell currentKV) throws IOException { Cell keyHint = null; if (operator == Operator.MUST_PASS_ALL) { - if (seekHintFilter != null) keyHint = seekHintFilter.getNextCellHint(currentKV); + keyHint = seekHintFilter.getNextCellHint(currentKV); return keyHint; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java index 12b19f63a5..6d502fd9c1 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java @@ -117,21 +117,6 @@ public class KeyValueUtil { return buffer; } - /** - * Copies the key to a new KeyValue - * @param cell - * @return the KeyValue that consists only the key part of the incoming cell - */ - public static KeyValue toNewKeyCell(final Cell cell) { - byte[] bytes = new byte[keyLength(cell)]; - appendKeyTo(cell, bytes, 0); - KeyValue kv = new KeyValue.KeyOnlyKeyValue(bytes, 0, bytes.length); - // Set the seq id. The new key cell could be used in comparisons so it - // is important that it uses the seqid also. If not the comparsion would fail - kv.setSequenceId(cell.getSequenceId()); - return kv; - } - public static byte[] copyToNewByteArray(final Cell cell) { int v1Length = length(cell); byte[] backingBytes = new byte[v1Length]; 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 994adffbc5..6ddc422a57 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 @@ -25,6 +25,7 @@ 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; @@ -553,121 +554,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