diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java index 88f18100bbd9..bfc693c7f22a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java @@ -38,6 +38,10 @@ * can be used for row-based indexing, where references to other tables are stored across many * columns, in order to efficient lookups and paginated results for end users. Only most recent * versions are considered for pagination. + * @apiNote This filter is in awkward place, as even though it can return SEEK_NEXT_USING_HINT, it + * also maintains an internal row state, so it is not marked as HintingFilter. Hinted seek + * information may be lost when used in a MUST_PASS_ALL FilterList, which can result in + * suboptimal performance. */ @InterfaceAudience.Public public class ColumnPaginationFilter extends FilterBase { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java index 3b8df1d15c68..9b477ec06cc7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java @@ -39,7 +39,7 @@ * with columns like 'ball', 'act'. */ @InterfaceAudience.Public -public class ColumnPrefixFilter extends FilterBase { +public class ColumnPrefixFilter extends FilterBase implements HintingFilter { protected byte[] prefix = null; public ColumnPrefixFilter(final byte[] prefix) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java index c9a7902d1e46..bbfec008c2c5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java @@ -43,7 +43,7 @@ * maxColumnInclusive specify if the ranges are inclusive or not. */ @InterfaceAudience.Public -public class ColumnRangeFilter extends FilterBase { +public class ColumnRangeFilter extends FilterBase implements HintingFilter { protected byte[] minColumn = null; protected boolean minColumnInclusive = true; protected byte[] maxColumn = null; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java index 45e06f444547..a5e1eec45401 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java @@ -33,6 +33,7 @@ public class FilterListWithAND extends FilterListBase { private List seekHintFilters = new ArrayList<>(); + private boolean[] hintingFilters; public FilterListWithAND(List filters) { super(filters); @@ -40,6 +41,7 @@ public FilterListWithAND(List filters) { // sub-filters (because all sub-filters return INCLUDE*). So here, fill this array with true. we // keep this in FilterListWithAND for abstracting the transformCell() in FilterListBase. subFiltersIncludedCell = new ArrayList<>(Collections.nCopies(filters.size(), true)); + cacheHintingFilters(); } @Override @@ -49,6 +51,7 @@ public void addFilterLists(List filters) { } this.filters.addAll(filters); this.subFiltersIncludedCell.addAll(Collections.nCopies(filters.size(), true)); + this.cacheHintingFilters(); } @Override @@ -57,6 +60,20 @@ protected String formatLogFilters(List logFilters) { logFilters.toString()); } + /** + * As checks for this are in the hot path, we want them as fast as possible, so we are caching the + * status in an array. + */ + private void cacheHintingFilters() { + int filtersSize = filters.size(); + hintingFilters = new boolean[filtersSize]; + for (int i = 0; i < filtersSize; i++) { + if (filters.get(i) instanceof HintingFilter) { + hintingFilters[i] = true; + } + } + } + /** * 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 @@ -169,10 +186,14 @@ public ReturnCode filterCell(Cell c) throws IOException { } ReturnCode rc = ReturnCode.INCLUDE; this.seekHintFilters.clear(); - for (int i = 0, n = filters.size(); i < n; i++) { + int i = 0; + int n = filters.size(); + for (; i < n; i++) { Filter filter = filters.get(i); if (filter.filterAllRemaining()) { - return ReturnCode.NEXT_ROW; + rc = ReturnCode.NEXT_ROW; + // See comment right after this loop + break; } ReturnCode localRC; localRC = filter.filterCell(c); @@ -184,9 +205,26 @@ public ReturnCode filterCell(Cell c) throws IOException { // otherwise we may mess up the global state (such as offset, count..) in the following // sub-filters. (HBASE-20565) if (!isIncludeRelatedReturnCode(rc)) { - return rc; + // See comment right after this loop + break; + } + } + // We have the preliminary return code. However, if there are remaining uncalled hintingFilters, + // they may return hints that allow us to seek ahead and skip reading and processing a lot of + // cells. + // Process the remaining hinting filters so that we can get all seek hints. + // The farthest key is computed in getNextCellHint() + if (++i < n) { + for (; i < n; i++) { + if (hintingFilters[i]) { + Filter filter = filters.get(i); + if (filter.filterCell(c) == ReturnCode.SEEK_NEXT_USING_HINT) { + seekHintFilters.add(filter); + } + } } } + if (!seekHintFilters.isEmpty()) { return ReturnCode.SEEK_NEXT_USING_HINT; } @@ -206,17 +244,29 @@ public boolean filterRowKey(Cell firstRowCell) throws IOException { if (isEmpty()) { return super.filterRowKey(firstRowCell); } - boolean retVal = false; + boolean anyRowKeyFiltered = false; + boolean anyHintingPassed = false; for (int i = 0, n = filters.size(); i < n; i++) { Filter filter = filters.get(i); - if (filter.filterAllRemaining() || filter.filterRowKey(firstRowCell)) { + if (filter.filterAllRemaining()) { + // We don't need to care about any later filters, as we end the scan immediately. + // TODO HBASE-28633 in the normal code path, filterAllRemaining() always gets checked + // before filterRowKey(). We should be able to remove this check. + return true; + } else if (filter.filterRowKey(firstRowCell)) { // Can't just return true here, because there are some filters (such as PrefixFilter) which // will catch the row changed event by filterRowKey(). If we return early here, those // filters will have no chance to update their row state. - retVal = true; + anyRowKeyFiltered = true; + } else if (hintingFilters[i]) { + // If filterRowKey returns false and this is a hinting filter, then we must not filter this + // rowkey. + // Otherwise this sub-filter doesn't get a chance to provide a seek hint, and the scan may + // regress into a full scan. + anyHintingPassed = true; } } - return retVal; + return anyRowKeyFiltered && !anyHintingPassed; } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index fd5a81d694e3..244dc9c7ca5f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -65,7 +65,7 @@ * I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value. */ @InterfaceAudience.Public -public class FuzzyRowFilter extends FilterBase { +public class FuzzyRowFilter extends FilterBase implements HintingFilter { private static final boolean UNSAFE_UNALIGNED = HBasePlatformDependent.unaligned(); private List> fuzzyKeysData; // Used to record whether we want to skip the current row. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/HintingFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/HintingFilter.java new file mode 100644 index 000000000000..ed68bf82d20f --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/HintingFilter.java @@ -0,0 +1,29 @@ +/* + * 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.yetus.audience.InterfaceAudience; + +/** + * Marker interface for filters that may return SEEK_NEXT_USING_HINT. This marker interface + * indicates that when it's used in a MUST_PASS_ALL FilterList then filterCell() must always be + * called if filterRowKey() returned false. + */ +@InterfaceAudience.Public +public interface HintingFilter { +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java index 85fae7b0a992..032a14eb62be 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java @@ -50,7 +50,7 @@ * fast-forwarding during scan. Thus, the scan will be quite efficient. */ @InterfaceAudience.Public -public class MultiRowRangeFilter extends FilterBase { +public class MultiRowRangeFilter extends FilterBase implements HintingFilter { private static final int ROW_BEFORE_FIRST_RANGE = -1; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java index 168257cd2f71..00d84fee9e38 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java @@ -42,7 +42,7 @@ * with columns like 'ball', 'act'. */ @InterfaceAudience.Public -public class MultipleColumnPrefixFilter extends FilterBase { +public class MultipleColumnPrefixFilter extends FilterBase implements HintingFilter { private static final Logger LOG = LoggerFactory.getLogger(MultipleColumnPrefixFilter.class); protected byte[] hint = null; protected TreeSet sortedPrefixes = createTreeSet(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java index 235691ef7cb1..cf2aa688d5a8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java @@ -43,7 +43,7 @@ * {@link org.apache.hadoop.hbase.client.Scan#setTimestamp(long)}. */ @InterfaceAudience.Public -public class TimestampsFilter extends FilterBase { +public class TimestampsFilter extends FilterBase implements HintingFilter { private final boolean canHint; TreeSet timestamps; 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 6c278b362c85..f64381a8a22e 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 @@ -644,6 +644,12 @@ public int hashCode() { } } + private static class HintingMockFilter extends MockFilter implements HintingFilter { + public HintingMockFilter(ReturnCode targetRetCode) { + super(targetRetCode); + } + } + @Test public void testShouldPassCurrentCellToFilter() throws IOException { KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1, @@ -729,7 +735,7 @@ public void testTheMaximalRule() throws IOException { 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 filter6 = new HintingMockFilter(ReturnCode.SEEK_NEXT_USING_HINT); MockFilter filter7 = new MockFilter(ReturnCode.NEXT_ROW); FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2); @@ -739,10 +745,10 @@ public void testTheMaximalRule() throws IOException { assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6); - assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv1)); + assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6); - assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv1)); + assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1); assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1)); @@ -767,7 +773,7 @@ public void testTheMinimalRule() throws IOException { 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 filter6 = new HintingMockFilter(ReturnCode.SEEK_NEXT_USING_HINT); FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2); assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));