From ae0e0e797443e3abc72a4a6a21fc511f11fd69ed Mon Sep 17 00:00:00 2001 From: Tamas Penzes Date: Tue, 19 Sep 2017 00:36:27 +0200 Subject: [PATCH] HBASE-13346: Clean up Filter package for post 1.0 s/KeyValue/Cell/g deprecated filterKeyValue, added filterCell, backward compatible --- .../java/org/apache/hadoop/hbase/client/Query.java | 2 +- .../hadoop/hbase/filter/ColumnCountGetFilter.java | 10 +- .../hbase/filter/ColumnPaginationFilter.java | 12 ++- .../hadoop/hbase/filter/ColumnPrefixFilter.java | 10 +- .../hadoop/hbase/filter/ColumnRangeFilter.java | 14 ++- .../hadoop/hbase/filter/DependentColumnFilter.java | 6 ++ .../apache/hadoop/hbase/filter/FamilyFilter.java | 12 ++- .../org/apache/hadoop/hbase/filter/Filter.java | 7 +- .../org/apache/hadoop/hbase/filter/FilterBase.java | 50 +++++++++- .../org/apache/hadoop/hbase/filter/FilterList.java | 10 +- .../apache/hadoop/hbase/filter/FilterWrapper.java | 20 +++- .../hadoop/hbase/filter/FirstKeyOnlyFilter.java | 10 +- .../FirstKeyValueMatchingQualifiersFilter.java | 16 +++- .../apache/hadoop/hbase/filter/FuzzyRowFilter.java | 6 ++ .../hadoop/hbase/filter/InclusiveStopFilter.java | 10 +- .../apache/hadoop/hbase/filter/KeyOnlyFilter.java | 10 +- .../hadoop/hbase/filter/MultiRowRangeFilter.java | 8 +- .../hbase/filter/MultipleColumnPrefixFilter.java | 12 ++- .../org/apache/hadoop/hbase/filter/PageFilter.java | 8 +- .../apache/hadoop/hbase/filter/PrefixFilter.java | 10 +- .../hadoop/hbase/filter/QualifierFilter.java | 12 ++- .../hadoop/hbase/filter/RandomRowFilter.java | 10 +- .../org/apache/hadoop/hbase/filter/RowFilter.java | 8 +- .../hbase/filter/SingleColumnValueFilter.java | 6 ++ .../org/apache/hadoop/hbase/filter/SkipFilter.java | 16 +++- .../hadoop/hbase/filter/TimestampsFilter.java | 16 +++- .../apache/hadoop/hbase/filter/ValueFilter.java | 10 +- .../hadoop/hbase/filter/WhileMatchFilter.java | 18 ++-- .../org/apache/hadoop/hbase/mapreduce/Import.java | 18 ++-- .../hbase/regionserver/MobReferenceOnlyFilter.java | 2 +- .../querymatcher/UserScanQueryMatcher.java | 2 +- .../hbase/security/access/AccessControlFilter.java | 2 +- .../security/visibility/VisibilityController.java | 2 +- .../security/visibility/VisibilityLabelFilter.java | 2 +- .../hbase/client/AbstractTestScanCursor.java | 4 +- .../hbase/client/ColumnCountOnRowFilter.java | 2 +- .../org/apache/hadoop/hbase/client/TestHCM.java | 2 +- .../TestRegionObserverScannerOpenHook.java | 2 +- .../hbase/filter/TestColumnPaginationFilter.java | 4 +- .../TestFirstKeyValueMatchingQualifiersFilter.java | 24 ++--- .../filter/TestSingleColumnValueExcludeFilter.java | 18 ++-- .../hbase/filter/TestSingleColumnValueFilter.java | 102 ++++++++++----------- .../hadoop/hbase/regionserver/TestHRegion.java | 2 +- .../regionserver/TestScannerHeartbeatMessages.java | 2 +- .../hadoop/hbase/regionserver/TestStore.java | 6 +- .../hadoop/hbase/spark/SparkSQLPushDownFilter.java | 2 +- 46 files changed, 373 insertions(+), 164 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java index cb09f76406..c61efa3935 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java @@ -55,7 +55,7 @@ public abstract class Query extends OperationWithAttributes { /** * Apply the specified server-side filter when performing the Query. Only - * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)} is called AFTER all tests for ttl, + * {@link Filter#filterCell(org.apache.hadoop.hbase.Cell)} is called AFTER all tests for ttl, * column match, deletes and column family's max versions have been run. * @param filter filter to run on the server * @return this for invocation chaining diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java index d41e6dfc9c..dbb640f65b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java @@ -63,7 +63,13 @@ public class ColumnCountGetFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { this.count++; return filterAllRemaining() ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL; } @@ -108,7 +114,7 @@ public class ColumnCountGetFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ 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 1565ddd24e..ccda32bc1a 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 @@ -111,7 +111,13 @@ public class ColumnPaginationFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (columnOffset != null) { if (count >= limit) { @@ -120,7 +126,7 @@ public class ColumnPaginationFilter extends FilterBase { int cmp = 0; // Only compare if no KV's have been seen so far. if (count == 0) { - cmp = CellComparator.compareQualifiers(v, this.columnOffset, 0, this.columnOffset.length); + cmp = CellComparator.compareQualifiers(c, this.columnOffset, 0, this.columnOffset.length); } if (cmp < 0) { return ReturnCode.SEEK_NEXT_USING_HINT; @@ -197,7 +203,7 @@ public class ColumnPaginationFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ 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 8a936d452f..6c507438e4 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 @@ -59,7 +59,13 @@ public class ColumnPrefixFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell cell) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell cell) { if (this.prefix == null) { return ReturnCode.INCLUDE; } else { @@ -132,7 +138,7 @@ public class ColumnPrefixFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ 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 425600c2f5..4aab83bd49 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 @@ -122,11 +122,17 @@ public class ColumnRangeFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell kv) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { int cmpMin = 1; if (this.minColumn != null) { - cmpMin = CellComparator.compareQualifiers(kv, this.minColumn, 0, this.minColumn.length); + cmpMin = CellComparator.compareQualifiers(c, this.minColumn, 0, this.minColumn.length); } if (cmpMin < 0) { @@ -141,7 +147,7 @@ public class ColumnRangeFilter extends FilterBase { return ReturnCode.INCLUDE; } - int cmpMax = CellComparator.compareQualifiers(kv, this.maxColumn, 0, this.maxColumn.length); + int cmpMax = CellComparator.compareQualifiers(c, this.maxColumn, 0, this.maxColumn.length); if (this.maxColumnInclusive && cmpMax <= 0 || !this.maxColumnInclusive && cmpMax < 0) { @@ -202,7 +208,7 @@ public class ColumnRangeFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java index 2f936e8abe..5ce897c9a1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java @@ -156,7 +156,13 @@ public class DependentColumnFilter extends CompareFilter { } @Override + @Deprecated public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { // Check if the column and qualifier match if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) { // include non-matches for the time being, they'll be discarded afterwards diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java index 2f5c33854e..f33f4c4008 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java @@ -73,10 +73,16 @@ public class FamilyFilter extends CompareFilter { } @Override - public ReturnCode filterKeyValue(Cell v) { - int familyLength = v.getFamilyLength(); + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { + int familyLength = c.getFamilyLength(); if (familyLength > 0) { - if (compareFamily(getCompareOperator(), this.comparator, v)) { + if (compareFamily(getCompareOperator(), this.comparator, c)) { return ReturnCode.NEXT_ROW; } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java index 70c68b6a5a..d808a40b68 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java @@ -124,12 +124,15 @@ public abstract class Filter { * Concrete implementers can signal a failure condition in their code by throwing an * {@link IOException}. * - * @param v the Cell in question + * @param c the Cell in question * @return code as described below * @throws IOException in case an I/O or an filter specific failure needs to be signaled. * @see Filter.ReturnCode + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. + * Instead use filterCell(Cell) */ - abstract public ReturnCode filterKeyValue(final Cell v) throws IOException; + @Deprecated + abstract public ReturnCode filterKeyValue(final Cell c) throws IOException; /** * Give the filter a chance to transform the passed KeyValue. If the Cell is changed a new diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java index e1dc4fa95b..b3f504fdab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java @@ -80,6 +80,54 @@ public abstract class FilterBase extends Filter { } /** + * A way to filter based on the column family, column qualifier and/or the column value. Return + * code is described below. This allows filters to filter only certain number of columns, then + * terminate without matching ever column. + * + * If filterRowKey returns true, filterKeyValue needs to be consistent with it. + * + * filterKeyValue can assume that filterRowKey has already been called for the row. + * + * If your filter returns ReturnCode.NEXT_ROW, it should return + * ReturnCode.NEXT_ROW until {@link #reset()} is called just in case the caller calls + * for the next row. + * + * Contains default implementation which includes all except when we filter all remaining. + * + * Concrete implementers can signal a failure condition in their code by throwing an + * {@link IOException}. + * + * @param c the Cell in question + * @return code as described below + * @throws IOException in case an I/O or an filter specific failure needs to be signaled. + * @see Filter.ReturnCode + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. + * Instead use {@link #filterCell(Cell)} + */ + @Override + @Deprecated + public ReturnCode filterKeyValue(final Cell c) throws IOException { + if(filterAllRemaining()){ + return ReturnCode.SKIP; + } + return ReturnCode.INCLUDE; + } + + /** + * A way to filter based on the column family, column qualifier and/or the column value. Return + * code is described below. This allows filters to filter only certain number of columns, then + * terminate without matching ever column. + * + * @param c the Cell in question + * @return code as described below + * @throws IOException in case an I/O or an filter specific failure needs to be signaled. + * @see Filter.ReturnCode + */ + public ReturnCode filterCell(final Cell c) throws IOException { + return filterKeyValue(c); + } + + /** * By default no transformation takes place * * {@inheritDoc} @@ -112,7 +160,7 @@ public abstract class FilterBase extends Filter { /** * Filters that never filter by rows based on previously gathered state from - * {@link #filterKeyValue(Cell)} can inherit this implementation that + * {@link #filterCell(Cell)} can inherit this implementation that * never filters a row. * * {@inheritDoc} 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..ed4c56933d 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 @@ -81,7 +81,7 @@ final public class FilterList extends FilterBase { private Cell referenceCell = null; /** - * When filtering a given Cell in {@link #filterKeyValue(Cell)}, + * When filtering a given Cell in {@link #filterCell(Cell)}, * this stores the transformed Cell to be returned by {@link #transformCell(Cell)}. * * Individual filters transformation are applied only when the filter includes the Cell. @@ -353,9 +353,15 @@ final public class FilterList extends FilterBase { } @Override + @Deprecated + public ReturnCode filterKeyValue(Cell c) throws IOException { + return filterCell(c); + } + + @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH", justification="Intentional") - public ReturnCode filterKeyValue(Cell c) throws IOException { + public ReturnCode filterCell(Cell c) throws IOException { if (isEmpty()) { return ReturnCode.INCLUDE; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java index c0fa6cf6fb..fc507ff4c4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java @@ -112,9 +112,23 @@ final public class FilterWrapper extends Filter { return this.filter.filterRowKey(cell); } + /** + * + * @param c the Cell in question + * @return code as described below + * @throws IOException in case an I/O or an filter specific failure needs to be signaled. + * @see Filter.ReturnCode + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. + * Instead use {@link #filterCell(Cell)} + */ @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - return this.filter.filterKeyValue(v); + @Deprecated + public ReturnCode filterKeyValue(Cell c) throws IOException { + return this.filter.filterKeyValue(c); + } + + public ReturnCode filterCell(Cell c) throws IOException { + return filterKeyValue(c); } @Override @@ -161,7 +175,7 @@ final public class FilterWrapper extends Filter { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java index 79e63293ff..0cc8b29d7f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java @@ -52,7 +52,13 @@ public class FirstKeyOnlyFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if(foundKV) return ReturnCode.NEXT_ROW; foundKV = true; return ReturnCode.INCLUDE; @@ -107,7 +113,7 @@ public class FirstKeyOnlyFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java index e4f1fbe5b2..aa3dea412c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java @@ -61,18 +61,24 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (hasFoundKV()) { return ReturnCode.NEXT_ROW; - } else if (hasOneMatchingQualifier(v)) { + } else if (hasOneMatchingQualifier(c)) { setFoundKV(true); } return ReturnCode.INCLUDE; } - private boolean hasOneMatchingQualifier(Cell v) { + private boolean hasOneMatchingQualifier(Cell c) { for (byte[] q : qualifiers) { - if (CellUtil.matchingQualifier(v, q)) { + if (CellUtil.matchingQualifier(c, q)) { return true; } } @@ -114,7 +120,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ 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 24d4fab907..f5e9dd4064 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 @@ -148,7 +148,13 @@ public class FuzzyRowFilter extends FilterBase { } @Override + @Deprecated public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0; final int size = fuzzyKeysData.size(); for (int i = startIndex; i < size + startIndex; i++) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java index 3c94c2c155..0aa3a1b5cf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java @@ -52,7 +52,13 @@ public class InclusiveStopFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (done) return ReturnCode.NEXT_ROW; return ReturnCode.INCLUDE; } @@ -105,7 +111,7 @@ public class InclusiveStopFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java index 0f71ca91a1..217b11b712 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java @@ -68,7 +68,13 @@ public class KeyOnlyFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + @Deprecated + public ReturnCode filterKeyValue(Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.INCLUDE; } @@ -110,7 +116,7 @@ public class KeyOnlyFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ 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 690aa3fa18..825e9a4b14 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 @@ -125,7 +125,13 @@ public class MultiRowRangeFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell ignored) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell ignored) { return currentReturnCode; } 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 78164517f9..63da5bc897 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 @@ -67,11 +67,17 @@ public class MultipleColumnPrefixFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell kv) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (sortedPrefixes.isEmpty()) { return ReturnCode.INCLUDE; } else { - return filterColumn(kv); + return filterColumn(c); } } @@ -144,7 +150,7 @@ public class MultipleColumnPrefixFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java index 2579f9dbed..d8a31f9b8a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java @@ -66,7 +66,13 @@ public class PageFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + @Deprecated + public ReturnCode filterKeyValue(Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.INCLUDE; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java index 455bd05080..ea47442467 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -76,7 +76,13 @@ public class PrefixFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (filterRow) return ReturnCode.NEXT_ROW; return ReturnCode.INCLUDE; } @@ -128,7 +134,7 @@ public class PrefixFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java index 9719a95722..c6cfe2879d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java @@ -71,10 +71,16 @@ public class QualifierFilter extends CompareFilter { } @Override - public ReturnCode filterKeyValue(Cell v) { - int qualifierLength = v.getQualifierLength(); + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { + int qualifierLength = c.getQualifierLength(); if (qualifierLength > 0) { - if (compareQualifier(getCompareOperator(), this.comparator, v)) { + if (compareQualifier(getCompareOperator(), this.comparator, c)) { return ReturnCode.SKIP; } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java index 1df18c9b19..8534c92e29 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java @@ -70,7 +70,13 @@ public class RandomRowFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (filterOutRow) { return ReturnCode.NEXT_ROW; } @@ -134,7 +140,7 @@ public class RandomRowFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java index d0ab6e92d4..d752c1862d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java @@ -77,7 +77,13 @@ public class RowFilter extends CompareFilter { } @Override - public ReturnCode filterKeyValue(Cell v) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell v) { if(this.filterOutRow) { return ReturnCode.NEXT_ROW; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index a86b257056..bd8df340fd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -247,7 +247,13 @@ public class SingleColumnValueFilter extends FilterBase { } @Override + @Deprecated public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue())); if (this.matchedColumn) { // We already found and matched the single column, all keys now pass diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java index f3d7e6bbf4..55bd22bebd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java @@ -79,10 +79,16 @@ public class SkipFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - ReturnCode c = filter.filterKeyValue(v); - changeFR(c != ReturnCode.INCLUDE); - return c; + @Deprecated + public ReturnCode filterKeyValue(Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) throws IOException { + ReturnCode rc = filter.filterKeyValue(c); + changeFR(rc != ReturnCode.INCLUDE); + return rc; } @Override @@ -130,7 +136,7 @@ public class SkipFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ 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 d77ece7a45..3576a1763f 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 @@ -109,10 +109,16 @@ public class TimestampsFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) { - if (this.timestamps.contains(v.getTimestamp())) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { + if (this.timestamps.contains(c.getTimestamp())) { return ReturnCode.INCLUDE; - } else if (v.getTimestamp() < minTimeStamp) { + } else if (c.getTimestamp() < minTimeStamp) { // The remaining versions of this column are guaranteed // to be lesser than all of the other values. return ReturnCode.NEXT_COL; @@ -140,7 +146,7 @@ public class TimestampsFilter extends FilterBase { // This should only happen if the current column's // timestamp is below the last one in the list. // - // It should never happen as the filterKeyValue should return NEXT_COL + // It should never happen as the filterCell should return NEXT_COL // but it's always better to be extra safe and protect against future // behavioral changes. @@ -193,7 +199,7 @@ public class TimestampsFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java index d4f41cff7f..e5bcc6558a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java @@ -69,8 +69,14 @@ public class ValueFilter extends CompareFilter { } @Override - public ReturnCode filterKeyValue(Cell v) { - if (compareValue(getCompareOperator(), this.comparator, v)) { + @Deprecated + public ReturnCode filterKeyValue(Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { + if (compareValue(getCompareOperator(), this.comparator, c)) { return ReturnCode.SKIP; } return ReturnCode.INCLUDE; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java index 3560178d3e..6482fd752f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE /** * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon * as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)}, - * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)}, + * {@link Filter#filterCell(org.apache.hadoop.hbase.Cell)}, * {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or * {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods * returns true. @@ -78,10 +78,16 @@ public class WhileMatchFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - ReturnCode c = filter.filterKeyValue(v); - changeFAR(c != ReturnCode.INCLUDE); - return c; + @Deprecated + public ReturnCode filterKeyValue(Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) throws IOException { + ReturnCode code = filter.filterKeyValue(c); + changeFAR(code != ReturnCode.INCLUDE); + return code; } @Override @@ -133,7 +139,7 @@ public class WhileMatchFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java index 28962bb51a..f91165d986 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java @@ -480,16 +480,16 @@ public class Import extends Configured implements Tool { /** * Attempt to filter out the keyvalue - * @param kv {@link KeyValue} on which to apply the filter + * @param c {@link Cell} on which to apply the filter * @return null if the key should not be written, otherwise returns the original - * {@link KeyValue} + * {@link Cell} */ - public static Cell filterKv(Filter filter, Cell kv) throws IOException { + public static Cell filterKv(Filter filter, Cell c) throws IOException { // apply the filter and skip this kv if the filter doesn't apply if (filter != null) { - Filter.ReturnCode code = filter.filterKeyValue(kv); + Filter.ReturnCode code = filter.filterKeyValue(c); if (LOG.isTraceEnabled()) { - LOG.trace("Filter returned:" + code + " for the key value:" + kv); + LOG.trace("Filter returned:" + code + " for the key value:" + c); } // if its not an accept type, then skip this kv if (!(code.equals(Filter.ReturnCode.INCLUDE) || code @@ -497,7 +497,7 @@ public class Import extends Configured implements Tool { return null; } } - return kv; + return c; } // helper: create a new KeyValue based on CF rename map @@ -681,7 +681,7 @@ public class Import extends Configured implements Tool { System.err.println("By default Import will load data directly into HBase. To instead generate"); System.err.println("HFiles of data to prepare for a bulk data load, pass the option:"); System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output"); - System.err.println("If there is a large result that includes too much KeyValue " + System.err.println("If there is a large result that includes too much Cell " + "whitch can occur OOME caused by the memery sort in reducer, pass the option:"); System.err.println(" -D" + HAS_LARGE_RESULT + "=true"); System.err @@ -692,9 +692,9 @@ public class Import extends Configured implements Tool { + CF_RENAME_PROP + " property. Futher, filters will only use the" + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify " + " whether the current row needs to be ignored completely for processing and " - + " Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;" + + " Filter#filterKeyValue(KeyValue) method to determine if the Cell should be added;" + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including" - + " the KeyValue."); + + " the Cell."); System.err.println("To import data exported from HBase 0.94, use"); System.err.println(" -Dhbase.import.version=0.94"); System.err.println(" -D " + JOB_NAME_CONF_KEY diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java index bddefc519c..1439198c8a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.mob.MobUtils; class MobReferenceOnlyFilter extends FilterBase { @Override - public ReturnCode filterKeyValue(Cell cell) { + public ReturnCode filterCell(Cell cell) { if (null != cell) { // If a cell with a mob reference tag, it's included. if (MobUtils.isMobReferenceCell(cell)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java index 5f76f3ec6c..3ca8c6e06d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java @@ -157,7 +157,7 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher { /* * Call this when scan has filter. Decide the desired behavior by checkVersions's MatchCode - * and filterKeyValue's ReturnCode. Cell may be skipped by filter, so the column versions + * and filterCell's ReturnCode. Cell may be skipped by filter, so the column versions * in result may be less than user need. It will check versions again after filter. * * ColumnChecker FilterResponse Desired behavior diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java index 9a3353c731..42c79f821c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java @@ -92,7 +92,7 @@ class AccessControlFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell cell) { + public ReturnCode filterCell(Cell cell) { if (isSystemTable) { return ReturnCode.INCLUDE; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index b3b1bc4212..08085a965b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -1071,7 +1071,7 @@ public class VisibilityController implements MasterObserver, RegionObserver, } @Override - public ReturnCode filterKeyValue(Cell cell) throws IOException { + public ReturnCode filterCell(Cell cell) throws IOException { List putVisTags = new ArrayList<>(); Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags); if (putVisTags.isEmpty() && deleteCellVisTags.isEmpty()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java index dcd301f15e..1b4ff4224a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java @@ -56,7 +56,7 @@ class VisibilityLabelFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell cell) throws IOException { + public ReturnCode filterCell(Cell cell) throws IOException { if (curFamily.getBytes() == null || !(CellUtil.matchingFamily(cell, curFamily.getBytes(), curFamily.getOffset(), curFamily.getLength()))) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java index ffd8c01a6b..318194f6fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java @@ -112,9 +112,9 @@ public abstract class AbstractTestScanCursor { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell c) throws IOException { Threads.sleep(TIMEOUT / 2 + 100); - return Bytes.equals(CellUtil.cloneRow(v), ROWS[reversed ? 0 : NUM_ROWS - 1]) + return Bytes.equals(CellUtil.cloneRow(c), ROWS[reversed ? 0 : NUM_ROWS - 1]) ? ReturnCode.INCLUDE : ReturnCode.SKIP; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java index 7841901b6f..96b60e4b69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java @@ -37,7 +37,7 @@ public final class ColumnCountOnRowFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell c) throws IOException { count++; return count > limit ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 1040a66b10..918556bdd0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -952,7 +952,7 @@ public class TestHCM { return false; } @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.INCLUDE; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 9c06c3ee02..bbd11e5c9a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -86,7 +86,7 @@ public class TestRegionObserverScannerOpenHook { public static class NoDataFilter extends FilterBase { @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.SKIP; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java index 828a2720c6..4f7e08964d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java @@ -76,8 +76,8 @@ public class TestColumnPaginationFilter */ private void basicFilterTests(ColumnPaginationFilter filter) throws Exception { - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1); - assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE_AND_NEXT_COL); + KeyValue c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1); + assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE_AND_NEXT_COL); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java index 60e35146ce..39f6bc2e60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFirstKeyValueMatchingQualifiersFilter.java @@ -51,26 +51,26 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase { Filter filter = new FirstKeyValueMatchingQualifiersFilter(quals); // Match in first attempt - KeyValue kv; - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1); + KeyValue cell; + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1); assertTrue("includeAndSetFlag", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); + filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); assertTrue("flagIsSetSkipToNextRow", - filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); + filter.filterKeyValue(cell) == Filter.ReturnCode.NEXT_ROW); // A mismatch in first attempt and match in second attempt. filter.reset(); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1); - System.out.println(filter.filterKeyValue(kv)); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1); + System.out.println(filter.filterKeyValue(cell)); assertTrue("includeFlagIsUnset", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); + filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); assertTrue("includeAndSetFlag", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1); + filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1); assertTrue("flagIsSetSkipToNextRow", - filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); + filter.filterKeyValue(cell) == Filter.ReturnCode.NEXT_ROW); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java index 6e9fef1f2f..269520251c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueExcludeFilter.java @@ -60,7 +60,7 @@ public class TestSingleColumnValueExcludeFilter { // A 'match' situation List kvs = new ArrayList<>(); - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); + KeyValue c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); kvs.add (new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1)); kvs.add (new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1)); @@ -69,21 +69,21 @@ public class TestSingleColumnValueExcludeFilter { filter.filterRowCells(kvs); assertEquals("resultSize", kvs.size(), 2); - assertTrue("leftKV1", CellComparator.COMPARATOR.compare(kvs.get(0), kv) == 0); - assertTrue("leftKV2", CellComparator.COMPARATOR.compare(kvs.get(1), kv) == 0); + assertTrue("leftKV1", CellComparator.COMPARATOR.compare(kvs.get(0), c) == 0); + assertTrue("leftKV2", CellComparator.COMPARATOR.compare(kvs.get(1), c) == 0); assertFalse("allRemainingWhenMatch", filter.filterAllRemaining()); // A 'mismatch' situation filter.reset(); // INCLUDE expected because test column has not yet passed - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); - assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); + c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); + assertTrue("otherColumn", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); // Test column will pass (wont match), expect NEXT_ROW - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); - assertTrue("testedMismatch", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); + c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); + assertTrue("testedMismatch", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW); // After a mismatch (at least with LatestVersionOnly), subsequent columns are EXCLUDE - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); - assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); + c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); + assertTrue("otherColumn", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java index a2167cecfa..225dc257f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java @@ -104,74 +104,74 @@ public class TestSingleColumnValueFilter { public void testLongComparator() throws IOException { Filter filter = new SingleColumnValueFilter(COLUMN_FAMILY, COLUMN_QUALIFIER, CompareOperator.GREATER, new LongComparator(100L)); - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, Bytes.toBytes(1L)); - assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); + assertTrue("less than", filter.filterKeyValue(cell) == Filter.ReturnCode.NEXT_ROW); filter.reset(); - byte[] buffer = kv.getBuffer(); + byte[] buffer = cell.getBuffer(); Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW); filter.reset(); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, Bytes.toBytes(100L)); - assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); + assertTrue("Equals 100", filter.filterKeyValue(cell) == Filter.ReturnCode.NEXT_ROW); filter.reset(); - buffer = kv.getBuffer(); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW); filter.reset(); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, Bytes.toBytes(120L)); - assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); + assertTrue("include 120", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); filter.reset(); - buffer = kv.getBuffer(); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); } private void basicFilterTests(SingleColumnValueFilter filter) throws Exception { - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); - assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - byte[] buffer = kv.getBuffer(); + KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); + assertTrue("basicFilter1", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + byte[] buffer = cell.getBuffer(); Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3); - assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3); + assertTrue("basicFilter2", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4); - assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4); + assertTrue("basicFilter3", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertFalse("basicFilterNotNull", filter.filterRow()); filter.reset(); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1); - assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1); + assertTrue("basicFilter4", filter.filterKeyValue(cell) == Filter.ReturnCode.NEXT_ROW); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); - assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); + assertTrue("basicFilter4", filter.filterKeyValue(cell) == Filter.ReturnCode.NEXT_ROW); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW); assertFalse("basicFilterAllRemaining", filter.filterAllRemaining()); assertTrue("basicFilterNotNull", filter.filterRow()); filter.reset(); filter.setLatestVersionOnly(false); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1); - assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1); + assertTrue("basicFilter5", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); - assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); + assertTrue("basicFilter5", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertFalse("basicFilterNotNull", filter.filterRow()); @@ -179,16 +179,16 @@ public class TestSingleColumnValueFilter { private void nullFilterTests(Filter filter) throws Exception { ((SingleColumnValueFilter) filter).setFilterIfMissing(true); - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1); - assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - byte[] buffer = kv.getBuffer(); + KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1); + assertTrue("null1", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + byte[] buffer = cell.getBuffer(); Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertFalse("null1FilterRow", filter.filterRow()); filter.reset(); - kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2); - assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + cell = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2); + assertTrue("null2", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertTrue("null2FilterRow", filter.filterRow()); @@ -196,17 +196,17 @@ public class TestSingleColumnValueFilter { private void substrFilterTests(Filter filter) throws Exception { - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1); assertTrue("substrTrue", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - byte[] buffer = kv.getBuffer(); + filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + byte[] buffer = cell.getBuffer(); Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_2); - assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + assertTrue("substrFalse", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertFalse("substrFilterAllRemaining", filter.filterAllRemaining()); @@ -215,17 +215,17 @@ public class TestSingleColumnValueFilter { private void regexFilterTests(Filter filter) throws Exception { - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1); assertTrue("regexTrue", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - byte[] buffer = kv.getBuffer(); + filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + byte[] buffer = cell.getBuffer(); Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_2); - assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + assertTrue("regexFalse", filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertFalse("regexFilterAllRemaining", filter.filterAllRemaining()); @@ -234,11 +234,11 @@ public class TestSingleColumnValueFilter { private void regexPatternFilterTests(Filter filter) throws Exception { - KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, + KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1); assertTrue("regexTrue", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - byte[] buffer = kv.getBuffer(); + filter.filterKeyValue(cell) == Filter.ReturnCode.INCLUDE); + byte[] buffer = cell.getBuffer(); Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); assertFalse("regexFilterAllRemaining", filter.filterAllRemaining()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 334df17e77..e0248c2f0b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -3448,7 +3448,7 @@ public class TestHRegion { scan.setLoadColumnFamiliesOnDemand(true); Filter bogusFilter = new FilterBase() { @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.INCLUDE; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java index 00e623129d..f506036f4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java @@ -272,7 +272,7 @@ public class TestScannerHeartbeatMessages { public static class SparseFilter extends FilterBase { @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell v) throws IOException { try { Thread.sleep(CLIENT_TIMEOUT / 2 + 100); } catch (InterruptedException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index 2095dcdb5f..b091e22535 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -1081,7 +1081,7 @@ public class TestStore { } }, new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) throws IOException { + public Filter.ReturnCode filterCell(Cell c) throws IOException { return ReturnCode.INCLUDE; } }, expectedSize); @@ -1105,7 +1105,7 @@ public class TestStore { } }, new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) throws IOException { + public Filter.ReturnCode filterCell(Cell c) throws IOException { if (timeToGoNextRow.get()) { timeToGoNextRow.set(false); return ReturnCode.NEXT_ROW; @@ -1134,7 +1134,7 @@ public class TestStore { } }, new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) throws IOException { + public Filter.ReturnCode filterCell(Cell c) throws IOException { if (timeToGetHint.get()) { timeToGetHint.set(false); return Filter.ReturnCode.SEEK_NEXT_USING_HINT; diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java index 6f3996b411..cfbf7ce0a6 100644 --- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java +++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java @@ -107,7 +107,7 @@ public class SparkSQLPushDownFilter extends FilterBase{ } @Override - public ReturnCode filterKeyValue(Cell c) throws IOException { + public ReturnCode filterCell(Cell c) throws IOException { //If the map RowValueMap is empty then we need to populate // the row key -- 2.11.0 (Apple Git-81)