From 8cbf5d1f06fdc84fc8aa0d27a4d35e0fea99cc3e Mon Sep 17 00:00:00 2001 From: Tamas Penzes Date: Wed, 25 Oct 2017 16:22:26 +0200 Subject: [PATCH] HBASE-13346: Clean up Filter package for post 1.0 s/KeyValue/Cell/g Added filterCell method to Filter, it calls filterKeyValue by default Deprecated filterKeyValue in Filter, bud added default functionality to return Filter.ReturnCode.INCLUDE. Added filterKeyValue (calling filterCell) to Filters extending FilterBase to be backward compatible. renamed filterKeyValue to filterCell in all implementations changed all internal calls to use filterCell instead of filterKeyValue changed tests too This way the change is simple and backward compatible. Any existing custom filter should work since they override filterKeyValue and the implementation is called by Filter.filterCell. Moved FilterWrapper to hbase-server --- .../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 | 12 +- .../hadoop/hbase/filter/DependentColumnFilter.java | 8 +- .../apache/hadoop/hbase/filter/FamilyFilter.java | 12 +- .../org/apache/hadoop/hbase/filter/Filter.java | 46 ++++- .../org/apache/hadoop/hbase/filter/FilterBase.java | 2 +- .../org/apache/hadoop/hbase/filter/FilterList.java | 18 +- .../apache/hadoop/hbase/filter/FilterListBase.java | 15 +- .../hadoop/hbase/filter/FilterListWithAND.java | 8 +- .../hadoop/hbase/filter/FilterListWithOR.java | 6 +- .../hadoop/hbase/filter/FirstKeyOnlyFilter.java | 10 +- .../FirstKeyValueMatchingQualifiersFilter.java | 17 +- .../apache/hadoop/hbase/filter/FuzzyRowFilter.java | 9 +- .../hadoop/hbase/filter/InclusiveStopFilter.java | 11 +- .../apache/hadoop/hbase/filter/KeyOnlyFilter.java | 10 +- .../hadoop/hbase/filter/MultiRowRangeFilter.java | 9 +- .../hbase/filter/MultipleColumnPrefixFilter.java | 12 +- .../org/apache/hadoop/hbase/filter/PageFilter.java | 8 +- .../apache/hadoop/hbase/filter/PrefixFilter.java | 11 +- .../hadoop/hbase/filter/QualifierFilter.java | 12 +- .../hadoop/hbase/filter/RandomRowFilter.java | 11 +- .../org/apache/hadoop/hbase/filter/RowFilter.java | 8 +- .../hbase/filter/SingleColumnValueFilter.java | 8 +- .../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 +- .../apache/hadoop/hbase/filter/FilterWrapper.java | 13 +- .../hbase/regionserver/MobReferenceOnlyFilter.java | 2 +- .../regionserver/querymatcher/ColumnTracker.java | 4 +- .../regionserver/querymatcher/DeleteTracker.java | 14 +- .../querymatcher/ScanDeleteTracker.java | 6 +- .../querymatcher/UserScanQueryMatcher.java | 4 +- .../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 +- .../hadoop/hbase/filter/FilterAllFilter.java | 2 +- .../hbase/filter/TestColumnPaginationFilter.java | 4 +- .../hbase/filter/TestDependentColumnFilter.java | 6 +- .../org/apache/hadoop/hbase/filter/TestFilter.java | 8 +- .../hbase/filter/TestFilterFromRegionSide.java | 2 +- .../apache/hadoop/hbase/filter/TestFilterList.java | 204 ++++++++++----------- .../TestFirstKeyValueMatchingQualifiersFilter.java | 26 +-- .../hbase/filter/TestInvocationRecordFilter.java | 2 +- .../hbase/filter/TestMultiRowRangeFilter.java | 12 +- .../filter/TestSingleColumnValueExcludeFilter.java | 24 +-- .../hbase/filter/TestSingleColumnValueFilter.java | 136 +++++++------- .../hadoop/hbase/regionserver/TestHRegion.java | 2 +- .../hadoop/hbase/regionserver/TestHStore.java | 6 +- .../regionserver/TestScannerHeartbeatMessages.java | 2 +- .../hadoop/hbase/spark/SparkSQLPushDownFilter.java | 2 +- 59 files changed, 531 insertions(+), 341 deletions(-) rename {hbase-client => hbase-server}/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (95%) 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..c461e98985 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 @@ -62,8 +62,14 @@ public class ColumnCountGetFilter extends FilterBase { return this.count > this.limit; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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 5faf85d0bb..63385d2491 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 @@ -110,7 +110,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) { @@ -119,7 +125,7 @@ public class ColumnPaginationFilter extends FilterBase { int cmp = 0; // Only compare if no KV's have been seen so far. if (count == 0) { - cmp = CellUtil.compareQualifiers(v, this.columnOffset, 0, this.columnOffset.length); + cmp = CellUtil.compareQualifiers(c, this.columnOffset, 0, this.columnOffset.length); } if (cmp < 0) { return ReturnCode.SEEK_NEXT_USING_HINT; @@ -196,7 +202,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..25085adad4 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 @@ -58,8 +58,14 @@ public class ColumnPrefixFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell cell) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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 87057a932e..80f34fbd30 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 @@ -121,11 +121,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 = CellUtil.compareQualifiers(kv, this.minColumn, 0, this.minColumn.length); + cmpMin = CellUtil.compareQualifiers(c, this.minColumn, 0, this.minColumn.length); } if (cmpMin < 0) { @@ -140,7 +146,7 @@ public class ColumnRangeFilter extends FilterBase { return ReturnCode.INCLUDE; } - int cmpMax = CellUtil.compareQualifiers(kv, this.maxColumn, 0, this.maxColumn.length); + int cmpMax = CellUtil.compareQualifiers(c, this.maxColumn, 0, this.maxColumn.length); if (this.maxColumnInclusive && cmpMax <= 0 || !this.maxColumnInclusive && cmpMax < 0) { 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..5c116907f6 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 @@ -155,8 +155,14 @@ public class DependentColumnFilter extends CompareFilter { return false; } + @Deprecated + @Override + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell c) { + 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..a22ddd67dc 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 @@ -72,11 +72,17 @@ public class FamilyFilter extends CompareFilter { super(op, familyComparator); } + @Deprecated + @Override + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell v) { - int familyLength = v.getFamilyLength(); + 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 a92ea0be89..dec8e061b5 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 @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; *
  • {@link #reset()} : reset the filter state before filtering a new row.
  • *
  • {@link #filterAllRemaining()}: true means row scan is over; false means keep going.
  • *
  • {@link #filterRowKey(Cell)}: true means drop this row; false means include.
  • - *
  • {@link #filterKeyValue(Cell)}: decides whether to include or exclude this Cell. + *
  • {@link #filterCell(Cell)}: decides whether to include or exclude this Cell. * See {@link ReturnCode}.
  • *
  • {@link #transformCell(Cell)}: if the Cell is included, let the filter transform the * Cell.
  • @@ -66,7 +66,7 @@ public abstract class Filter { /** * Filters a row based on the row key. If this returns true, the entire row will be excluded. If - * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below. + * false, each KeyValue in the row will be passed to {@link #filterCell(Cell)} below. * * Concrete implementers can signal a failure condition in their code by throwing an * {@link IOException}. @@ -84,7 +84,7 @@ public abstract class Filter { /** * Filters a row based on the row key. If this returns true, the entire row will be excluded. If - * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below. + * false, each KeyValue in the row will be passed to {@link #filterCell(Cell)} below. * If {@link #filterAllRemaining()} returns true, then {@link #filterRowKey(Cell)} should * also return true. * @@ -120,16 +120,46 @@ public abstract class Filter { * 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. - * + * * 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, Filter.ReturnCode.INCLUDE by default + * @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) + */ + @Deprecated + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return Filter.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. + * + * If filterRowKey returns true, filterCell needs to be consistent with it. + * + * filterCell 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. + * + * 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 */ - abstract public ReturnCode filterKeyValue(final Cell v) throws IOException; + public ReturnCode filterCell(final Cell c) throws IOException{ + return filterKeyValue(c); + } /** * Give the filter a chance to transform the passed KeyValue. If the Cell is changed a new @@ -173,7 +203,7 @@ public abstract class Filter { NEXT_COL, /** * Seek to next row in current family. It may still pass a cell whose family is different but - * row is the same as previous cell to {@link #filterKeyValue(Cell)} , even if we get a NEXT_ROW + * row is the same as previous cell to {@link #filterCell(Cell)} , even if we get a NEXT_ROW * returned for previous cell. For more details see HBASE-18368.
    * Once reset() method was invoked, then we switch to the next row for all family, and you can * catch the event by invoking CellUtils.matchingRows(previousCell, currentCell).
    @@ -210,7 +240,7 @@ public abstract class Filter { abstract public boolean hasFilterRow(); /** - * Last chance to veto row based on previous {@link #filterKeyValue(Cell)} calls. The filter + * Last chance to veto row based on previous {@link #filterCell(Cell)} calls. The filter * needs to retain state then return a particular value for this call if they wish to exclude a * row if a certain column is missing (for example). * 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..86a5b39522 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 @@ -112,7 +112,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 d4242ae6d1..a3d52c889e 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 @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; * {@link Operator#MUST_PASS_ONE} (OR). Since you can use Filter Lists as children of * Filter Lists, you can create a hierarchy of filters to be evaluated.
    * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does not - * include the KeyValue.
    + * include the Cell.
    * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated.
    * Defaults to {@link Operator#MUST_PASS_ALL}. */ @@ -166,8 +166,8 @@ final public class FilterList extends FilterBase { } /** - * Internal implementation of {@link #filterKeyValue(Cell)}. Compared to the - * {@link #filterKeyValue(Cell)} method, this method accepts an additional parameter named + * Internal implementation of {@link #filterCell(Cell)}. Compared to the + * {@link #filterCell(Cell)} method, this method accepts an additional parameter named * transformedCell. This parameter indicates the initial value of transformed cell before this * filter operation.
    * For FilterList, we can consider a filter list as a node in a tree. sub-filters of the filter @@ -180,13 +180,19 @@ final public class FilterList extends FilterBase { * @return ReturnCode of this filter operation. * @throws IOException */ - ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException { - return this.filterListBase.internalFilterKeyValue(c, transformedCell); + ReturnCode internalFilterCell(Cell c, Cell transformedCell) throws IOException { + return this.filterListBase.internalFilterCell(c, transformedCell); } @Override + @Deprecated public ReturnCode filterKeyValue(Cell c) throws IOException { - return filterListBase.filterKeyValue(c); + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) throws IOException { + return filterListBase.filterCell(c); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java index f92d2e772f..65b518f26f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java @@ -43,7 +43,7 @@ public abstract class FilterListBase extends FilterBase { protected Cell referenceCell = null; /** - * When filtering a given Cell in {@link #filterKeyValue(Cell)}, this stores the transformed 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. Transformations are composed in the order specified by * {@link #filters}. @@ -108,18 +108,23 @@ public abstract class FilterListBase extends FilterBase { } /** - * Internal implementation of {@link #filterKeyValue(Cell)} + * Internal implementation of {@link #filterCell(Cell)} * @param c The cell in question. * @param transformedCell The transformed cell of previous filter(s) * @return ReturnCode of this filter operation. * @throws IOException - * @see org.apache.hadoop.hbase.filter.FilterList#internalFilterKeyValue(Cell, Cell) + * @see org.apache.hadoop.hbase.filter.FilterList#internalFilterCell(Cell, Cell) */ - abstract ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException; + abstract ReturnCode internalFilterCell(Cell c, Cell transformedCell) throws IOException; @Override public ReturnCode filterKeyValue(Cell c) throws IOException { - return internalFilterKeyValue(c, c); + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) throws IOException { + return internalFilterCell(c, c); } /** 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 9217ff92e3..3397569a6c 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 @@ -59,7 +59,7 @@ public class FilterListWithAND extends FilterListBase { * Let's call it: The Maximal Step Rule. So if filter-A in filter list return INCLUDE and filter-B * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should return * INCLUDE_AND_NEXT_COL. For SEEK_NEXT_USING_HINT, it's more special, and in method - * filterKeyValueWithMustPassAll(), if any sub-filter return SEEK_NEXT_USING_HINT, then our filter + * filterCellWithMustPassAll(), if any sub-filter return SEEK_NEXT_USING_HINT, then our filter * list will return SEEK_NEXT_USING_HINT. so we don't care about the SEEK_NEXT_USING_HINT here. *
    *
    @@ -148,7 +148,7 @@ public class FilterListWithAND extends FilterListBase { } @Override - ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException { + ReturnCode internalFilterCell(Cell c, Cell transformedCell) throws IOException { if (isEmpty()) { return ReturnCode.INCLUDE; } @@ -163,9 +163,9 @@ public class FilterListWithAND extends FilterListBase { } ReturnCode localRC; if (filter instanceof FilterList) { - localRC = ((FilterList) filter).internalFilterKeyValue(c, transformed); + localRC = ((FilterList) filter).internalFilterCell(c, transformed); } else { - localRC = filter.filterKeyValue(c); + localRC = filter.filterCell(c); } rc = mergeReturnCode(rc, localRC); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java index e51915b733..51886bc64e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java @@ -246,7 +246,7 @@ public class FilterListWithOR extends FilterListBase { } @Override - ReturnCode internalFilterKeyValue(Cell c, Cell transformCell) throws IOException { + ReturnCode internalFilterCell(Cell c, Cell transformCell) throws IOException { if (isEmpty()) { return ReturnCode.INCLUDE; } @@ -266,9 +266,9 @@ public class FilterListWithOR extends FilterListBase { ReturnCode localRC; if (filter instanceof FilterList) { - localRC = ((FilterList) filter).internalFilterKeyValue(c, transformed); + localRC = ((FilterList) filter).internalFilterCell(c, transformed); } else { - localRC = filter.filterKeyValue(c); + localRC = filter.filterCell(c); } // Update previous return code and previous cell for filter[i]. 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..28c502fb96 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 @@ -51,8 +51,14 @@ public class FirstKeyOnlyFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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..59ef3c36a2 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.Set; import java.util.TreeSet; @@ -60,19 +61,25 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { this.qualifiers = qualifiers; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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 +121,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 244e8fba59..7a5fc92fef 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; @@ -147,8 +148,14 @@ public class FuzzyRowFilter extends FilterBase { return true; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell c) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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 c467d17480..3a64b2ced4 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.ArrayList; import org.apache.hadoop.hbase.Cell; @@ -51,8 +52,14 @@ public class InclusiveStopFilter extends FilterBase { return this.stopRowKey; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (done) return ReturnCode.NEXT_ROW; return ReturnCode.INCLUDE; } @@ -105,7 +112,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..684d863145 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 @@ -67,8 +67,14 @@ public class KeyOnlyFilter extends FilterBase { } } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterKeyValue(final 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..8bb1d5360e 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -124,8 +125,14 @@ public class MultiRowRangeFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell ignored) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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..09be1285b0 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 @@ -66,12 +66,18 @@ public class MultipleColumnPrefixFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell kv) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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..5a59804d61 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 @@ -65,8 +65,14 @@ public class PageFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterKeyValue(final 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..6b8731640f 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.ArrayList; import org.apache.hadoop.hbase.ByteBufferCell; @@ -75,8 +76,14 @@ public class PrefixFilter extends FilterBase { return filterRow; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (filterRow) return ReturnCode.NEXT_ROW; return ReturnCode.INCLUDE; } @@ -128,7 +135,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..ab840dbb13 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 @@ -70,11 +70,17 @@ public class QualifierFilter extends CompareFilter { super(op, qualifierComparator); } + @Deprecated + @Override + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell v) { - int qualifierLength = v.getQualifierLength(); + 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..cbb13c2a91 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.io.IOException; import java.util.Random; import org.apache.hadoop.hbase.Cell; @@ -69,8 +70,14 @@ public class RandomRowFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) { if (filterOutRow) { return ReturnCode.NEXT_ROW; } @@ -134,7 +141,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..9533efd142 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 @@ -76,8 +76,14 @@ public class RowFilter extends CompareFilter { this.filterOutRow = false; } + @Deprecated + @Override + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell v) { + 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 468af8cf18..d69455d5ef 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 @@ -245,8 +245,14 @@ public class SingleColumnValueFilter extends FilterBase { return false; } + @Deprecated + @Override + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell c) { + 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..dd6a0bb769 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 @@ -78,11 +78,17 @@ public class SkipFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - ReturnCode c = filter.filterKeyValue(v); - changeFR(c != ReturnCode.INCLUDE); - return c; + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) throws IOException { + ReturnCode rc = filter.filterCell(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..bc8db31707 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 @@ -108,11 +108,17 @@ public class TimestampsFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { - if (this.timestamps.contains(v.getTimestamp())) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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..6eb0788c0d 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 @@ -68,9 +68,15 @@ public class ValueFilter extends CompareFilter { super(valueCompareOp, valueComparator); } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { - if (compareValue(getCompareOperator(), this.comparator, v)) { + public ReturnCode filterKeyValue(final Cell c) throws IOException { + 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..0adf998ab4 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. @@ -77,11 +77,17 @@ public class WhileMatchFilter extends FilterBase { return value; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - ReturnCode c = filter.filterKeyValue(v); - changeFAR(c != ReturnCode.INCLUDE); - return c; + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(Cell c) throws IOException { + ReturnCode code = filter.filterCell(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 acc3c62f24..3bb966ac53 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 @@ -482,16 +482,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.filterCell(c); if (LOG.isTraceEnabled()) { - LOG.trace("Filter returned:" + code + " for the key value:" + kv); + LOG.trace("Filter returned:" + code + " for the cell:" + c); } // if its not an accept type, then skip this kv if (!(code.equals(Filter.ReturnCode.INCLUDE) || code @@ -499,7 +499,7 @@ public class Import extends Configured implements Tool { return null; } } - return kv; + return c; } // helper: create a new KeyValue based on CF rename map @@ -683,7 +683,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 @@ -694,9 +694,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#filterCell(Cell) 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-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java similarity index 95% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java index c0fa6cf6fb..cb8e0599ac 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE * related hooks can be defined in this wrapper. The only way to create a * FilterWrapper instance is passing a client side Filter instance through * {@link org.apache.hadoop.hbase.client.Scan#getFilter()}. - * + * */ @InterfaceAudience.Private final public class FilterWrapper extends Filter { @@ -112,9 +112,14 @@ final public class FilterWrapper extends Filter { return this.filter.filterRowKey(cell); } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - return this.filter.filterKeyValue(v); + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + public ReturnCode filterCell(Cell c) throws IOException { + return this.filter.filterCell(c); } @Override @@ -161,7 +166,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-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/ColumnTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ColumnTracker.java index b432e9d3c9..15dea6b09e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ColumnTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ColumnTracker.java @@ -61,7 +61,7 @@ public interface ColumnTracker extends ShipperListener { * this method are {@link MatchCode#INCLUDE}, {@link MatchCode#SEEK_NEXT_COL} and * {@link MatchCode#SEEK_NEXT_ROW}. * @param cell - * @param type The type of the KeyValue + * @param type The type of the Cell * @return The match code instance. * @throws IOException in case there is an internal consistency problem caused by a data * corruption. @@ -70,7 +70,7 @@ public interface ColumnTracker extends ShipperListener { /** * Keeps track of the number of versions for the columns asked for. It assumes that the user has - * already checked if the keyvalue needs to be included by calling the + * already checked if the cell needs to be included by calling the * {@link #checkColumn(Cell, byte)} method. The enum values returned by this method * are {@link MatchCode#SKIP}, {@link MatchCode#INCLUDE}, * {@link MatchCode#INCLUDE_AND_SEEK_NEXT_COL} and {@link MatchCode#INCLUDE_AND_SEEK_NEXT_ROW}. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java index 8f0a107df7..be9c51eca8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.regionserver.ShipperListener; * This class is utilized through three methods: * */ @@ -47,7 +47,7 @@ public interface DeleteTracker extends ShipperListener { /** * Check if the specified cell buffer has been deleted by a previously seen delete. * @param cell - current cell to check if deleted by a previously seen delete - * @return deleteResult The result tells whether the KeyValue is deleted and why + * @return deleteResult The result tells whether the Cell is deleted and why */ DeleteResult isDeleted(Cell cell); @@ -76,12 +76,12 @@ public interface DeleteTracker extends ShipperListener { * and why. Based on the delete result, the ScanQueryMatcher will decide the next operation */ enum DeleteResult { - FAMILY_DELETED, // The KeyValue is deleted by a delete family. - FAMILY_VERSION_DELETED, // The KeyValue is deleted by a delete family version. - COLUMN_DELETED, // The KeyValue is deleted by a delete column. - VERSION_DELETED, // The KeyValue is deleted by a version delete. + FAMILY_DELETED, // The Cell is deleted by a delete family. + FAMILY_VERSION_DELETED, // The Cell is deleted by a delete family version. + COLUMN_DELETED, // The Cell is deleted by a delete column. + VERSION_DELETED, // The Cell is deleted by a version delete. NOT_DELETED, - VERSION_MASKED // The KeyValue is masked by max number of versions which is considered as + VERSION_MASKED // The Cell is masked by max number of versions which is considered as // deleted in strong semantics of versions(See MvccTracker) } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanDeleteTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanDeleteTracker.java index 0daeb91f60..26da698f47 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanDeleteTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanDeleteTracker.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes; * This class is utilized through three methods: * *

    @@ -63,7 +63,7 @@ public class ScanDeleteTracker implements DeleteTracker { } /** - * Add the specified KeyValue to the list of deletes to check against for this row operation. + * Add the specified Cell to the list of deletes to check against for this row operation. *

    * This is called when a Delete is encountered. * @param cell - the delete cell @@ -97,7 +97,7 @@ public class ScanDeleteTracker implements DeleteTracker { } /** - * Check if the specified KeyValue buffer has been deleted by a previously seen delete. + * Check if the specified Cell buffer has been deleted by a previously seen delete. * @param cell - current cell to check if deleted by a previously seen delete * @return deleteResult */ 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..16afa83d61 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 @@ -152,12 +152,12 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher { } return filter == null ? matchCode : mergeFilterResponse(cell, matchCode, - filter.filterKeyValue(cell)); + filter.filterCell(cell)); } /* * 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 b6742be4ff..d569096171 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 @@ -1050,7 +1050,7 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso } @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 e2bdaf42dc..9c91736be5 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 @@ -972,7 +972,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 9e7c1847fd..b5147e9b41 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 @@ -90,7 +90,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/FilterAllFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java index ab323de68b..5c1fda68a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java @@ -28,7 +28,7 @@ public class FilterAllFilter extends FilterBase { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell v) 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..fc012d25f7 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.filterCell(c) == Filter.ReturnCode.INCLUDE_AND_NEXT_COL); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java index 74503bfba6..d84fbe9481 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java @@ -234,11 +234,11 @@ public class TestDependentColumnFilter { Filter filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER); List accepted = new ArrayList<>(); for(Cell val : testVals) { - if(filter.filterKeyValue(val) == ReturnCode.INCLUDE) { + if(filter.filterCell(val) == ReturnCode.INCLUDE) { accepted.add(val); } } - assertEquals("check all values accepted from filterKeyValue", 5, accepted.size()); + assertEquals("check all values accepted from filterCell", 5, accepted.size()); filter.filterRowCells(accepted); assertEquals("check filterRow(List) dropped cell without corresponding column entry", 4, accepted.size()); @@ -247,7 +247,7 @@ public class TestDependentColumnFilter { filter = new DependentColumnFilter(FAMILIES[1], QUALIFIER, true); accepted.clear(); for(KeyValue val : testVals) { - if(filter.filterKeyValue(val) == ReturnCode.INCLUDE) { + if(filter.filterCell(val) == ReturnCode.INCLUDE) { accepted.add(val); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index d225b8d5b0..4b1864b5ad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -610,7 +610,7 @@ public class TestFilter { } @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.INCLUDE; } } @@ -666,14 +666,14 @@ public class TestFilter { /** * Tests the the {@link WhileMatchFilter} works in combination with a - * {@link Filter} that uses the {@link Filter#filterKeyValue(Cell)} method. + * {@link Filter} that uses the {@link Filter#filterCell(Cell)} method. * * See HBASE-2258. * * @throws Exception */ @Test - public void testWhileMatchFilterWithFilterKeyValue() throws Exception { + public void testWhileMatchFilterWithFilterCell() throws Exception { Scan s = new Scan(); WhileMatchFilter filter = new WhileMatchFilter( new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, Bytes.toBytes("foo")) @@ -2037,7 +2037,7 @@ public class TestFilter { public byte [] toByteArray() {return null;} @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/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java index 0a287ce818..3c0af9e23a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java @@ -146,7 +146,7 @@ public class TestFilterFromRegionSide { } @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterCell(Cell v) { if (count++ < NUM_COLS) { return ReturnCode.INCLUDE; } 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 8c83cf6a3e..7e939c1e7a 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 @@ -137,7 +137,7 @@ public class TestFilterList { *

  • {@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.
  • *
  • {@link #filterRowKey(byte[],int,int)} -> true to drop this row, * if false, we will also call
  • - *
  • {@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value
  • + *
  • {@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell
  • *
  • {@link #filterRow()} -> last chance to drop entire row based on the sequence of * filterValue() calls. Eg: filter a row if it doesn't contain a specified column. *
  • @@ -152,7 +152,7 @@ public class TestFilterList { assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i)); - assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv)); assertFalse(filterMPONE.filterRow()); } @@ -161,7 +161,7 @@ public class TestFilterList { assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0)); - assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv)); assertFalse(filterMPONE.filterRow()); /* reach MAX_PAGES already, should filter any rows */ @@ -169,7 +169,7 @@ public class TestFilterList { assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0)); - assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); + assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv)); assertFalse(filterMPONE.filterRow()); /* We should filter any row */ @@ -203,7 +203,7 @@ public class TestFilterList { *
  • {@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.
  • *
  • {@link #filterRowKey(byte[],int,int)} -> true to drop this row, * if false, we will also call
  • - *
  • {@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value
  • + *
  • {@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell
  • *
  • {@link #filterRow()} -> last chance to drop entire row based on the sequence of * filterValue() calls. Eg: filter a row if it doesn't contain a specified column. *
  • @@ -216,14 +216,14 @@ public class TestFilterList { assertFalse(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i)); - assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterCell(kv)); } filterMPALL.reset(); rowkey = Bytes.toBytes("z"); assertTrue(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); // Should fail here; row should be filtered out. KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey); - assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterCell(kv)); } /** @@ -251,7 +251,7 @@ public class TestFilterList { *
  • {@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.
  • *
  • {@link #filterRowKey(byte[],int,int)} -> true to drop this row, * if false, we will also call
  • - *
  • {@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value
  • + *
  • {@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value
  • *
  • {@link #filterRow()} -> last chance to drop entire row based on the sequence of * filterValue() calls. Eg: filter a row if it doesn't contain a specified column. *
  • @@ -266,7 +266,7 @@ public class TestFilterList { assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i)); - assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv)); assertFalse(filterMPONE.filterRow()); } @@ -276,7 +276,7 @@ public class TestFilterList { assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i)); - assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv)); assertFalse(filterMPONE.filterRow()); } @@ -286,7 +286,7 @@ public class TestFilterList { assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey))); KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i)); - assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv)); + assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv)); assertFalse(filterMPONE.filterRow()); } } @@ -305,23 +305,23 @@ public class TestFilterList { FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE); flist.addFilter(new PrefixFilter(r1)); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1))); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11))); + assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1))); + assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11))); flist.reset(); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2)); - assertEquals(ReturnCode.SKIP, flist.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(ReturnCode.SKIP, flist.filterCell(new KeyValue(r2, r2, r2))); flist = new FilterList(FilterList.Operator.MUST_PASS_ONE); flist.addFilter(new AlwaysNextColFilter()); flist.addFilter(new PrefixFilter(r1)); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1))); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11))); + assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1))); + assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11))); flist.reset(); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2)); - assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2))); } /** @@ -339,12 +339,12 @@ public class TestFilterList { flist.addFilter(new AlwaysNextColFilter()); flist.addFilter(new InclusiveStopFilter(r1)); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1))); - assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11))); + assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1))); + assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11))); flist.reset(); flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2)); - assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2))); } public static class AlwaysNextColFilter extends FilterBase { @@ -353,7 +353,7 @@ public class TestFilterList { } @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterCell(Cell v) { return ReturnCode.NEXT_COL; } @@ -388,14 +388,14 @@ public class TestFilterList { } /** - * Test filterKeyValue logic. + * Test filterCell logic. * @throws Exception */ @Test - public void testFilterKeyValue() throws Exception { + public void testFilterCell() throws Exception { Filter includeFilter = new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) { + public Filter.ReturnCode filterCell(Cell v) { return Filter.ReturnCode.INCLUDE; } }; @@ -404,7 +404,7 @@ public class TestFilterList { boolean returnInclude = true; @Override - public Filter.ReturnCode filterKeyValue(Cell v) { + public Filter.ReturnCode filterCell(Cell v) { Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE : Filter.ReturnCode.SKIP; returnInclude = !returnInclude; @@ -416,7 +416,7 @@ public class TestFilterList { boolean returnIncludeOnly = false; @Override - public Filter.ReturnCode filterKeyValue(Cell v) { + public Filter.ReturnCode filterCell(Cell v) { Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE : Filter.ReturnCode.INCLUDE_AND_NEXT_COL; returnIncludeOnly = !returnIncludeOnly; @@ -428,17 +428,17 @@ public class TestFilterList { FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE, Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter })); // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL. - assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null)); + assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterCell(null)); // INCLUDE, SKIP, INCLUDE. - assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterCell(null)); // Check must pass all filter. FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter })); // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL. - assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterCell(null)); // INCLUDE, SKIP, INCLUDE. - assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterCell(null)); } /** @@ -458,14 +458,14 @@ public class TestFilterList { } @Override - public ReturnCode filterKeyValue(Cell ignored) throws IOException { + public ReturnCode filterCell(Cell ignored) throws IOException { return ReturnCode.INCLUDE; } }; Filter filterMinHint = new FilterBase() { @Override - public ReturnCode filterKeyValue(Cell ignored) { + public ReturnCode filterCell(Cell ignored) { return ReturnCode.SEEK_NEXT_USING_HINT; } @@ -480,7 +480,7 @@ public class TestFilterList { Filter filterMaxHint = new FilterBase() { @Override - public ReturnCode filterKeyValue(Cell ignored) { + public ReturnCode filterCell(Cell ignored) { return ReturnCode.SEEK_NEXT_USING_HINT; } @@ -521,30 +521,30 @@ public class TestFilterList { // Should take the first hint filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } )); - filterList.filterKeyValue(null); + filterList.filterCell(null); assertEquals(0, CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue)); filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } )); - filterList.filterKeyValue(null); + filterList.filterCell(null); assertEquals(0, CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue)); // Should have first hint even if a filter has no hint filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint })); - filterList.filterKeyValue(null); + filterList.filterCell(null); assertEquals(0, CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue)); filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter[] { filterNoHint, filterMaxHint })); - filterList.filterKeyValue(null); + filterList.filterCell(null); assertEquals(0, CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue)); filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(new Filter[] { filterNoHint, filterMinHint })); - filterList.filterKeyValue(null); + filterList.filterCell(null); assertEquals(0, CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue)); } @@ -552,7 +552,7 @@ public class TestFilterList { /** * Tests the behavior of transform() in a hierarchical filter. * - * transform() only applies after a filterKeyValue() whose return-code includes the KeyValue. + * transform() only applies after a filterCell() whose return-code includes the KeyValue. * Lazy evaluation of AND */ @Test @@ -577,18 +577,18 @@ public class TestFilterList { Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value")); // Value for fam:qual1 should be stripped: - assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1)); + assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual1)); final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual1)); assertEquals(0, transformedQual1.getValueLength()); // Value for fam:qual2 should not be stripped: - assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2)); + assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual2)); final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual2)); assertEquals("value", Bytes.toString(transformedQual2.getValueArray(), transformedQual2.getValueOffset(), transformedQual2.getValueLength())); // Other keys should be skipped: - assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3)); + assertEquals(Filter.ReturnCode.SKIP, flist.filterCell(kvQual3)); } @Test @@ -603,16 +603,16 @@ public class TestFilterList { KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"), 3, Bytes.toBytes("value")); - assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterKeyValue(kv1)); - assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv2)); - assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv3)); + assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterCell(kv1)); + assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv2)); + assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv3)); FilterList filterList11 = new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1)); - assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv1)); - assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv2)); - assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv3)); + assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv1)); + assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv2)); + assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv3)); } @Test @@ -630,10 +630,10 @@ public class TestFilterList { KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4, Bytes.toBytes("value")); - assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1)); - assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv2)); - assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv3)); - assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv4)); + assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); + assertEquals(ReturnCode.SKIP, filterList.filterCell(kv2)); + assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3)); + assertEquals(ReturnCode.SKIP, filterList.filterCell(kv4)); } private static class MockFilter extends FilterBase { @@ -645,7 +645,7 @@ public class TestFilterList { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell v) throws IOException { this.didCellPassToTheFilter = true; return targetRetCode; } @@ -665,65 +665,65 @@ public class TestFilterList { MockFilter mockFilter = new MockFilter(ReturnCode.NEXT_COL); FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter); - filter.filterKeyValue(kv1); + filter.filterCell(kv1); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv2); + filter.filterCell(kv2); assertFalse(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv3); + filter.filterCell(kv3); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL); filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter); - filter.filterKeyValue(kv1); + filter.filterCell(kv1); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv2); + filter.filterCell(kv2); assertFalse(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv3); + filter.filterCell(kv3); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter = new MockFilter(ReturnCode.NEXT_ROW); filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter); - filter.filterKeyValue(kv1); + filter.filterCell(kv1); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv2); + filter.filterCell(kv2); assertFalse(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv3); + filter.filterCell(kv3); assertFalse(mockFilter.didCellPassToTheFilter); filter.reset(); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv4); + filter.filterCell(kv4); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW); filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter); - filter.filterKeyValue(kv1); + filter.filterCell(kv1); assertTrue(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv2); + filter.filterCell(kv2); assertFalse(mockFilter.didCellPassToTheFilter); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv3); + filter.filterCell(kv3); assertFalse(mockFilter.didCellPassToTheFilter); filter.reset(); mockFilter.didCellPassToTheFilter = false; - filter.filterKeyValue(kv4); + filter.filterCell(kv4); assertTrue(mockFilter.didCellPassToTheFilter); } @@ -740,29 +740,29 @@ public class TestFilterList { MockFilter filter7 = new MockFilter(ReturnCode.NEXT_ROW); FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2); - assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, filter3); - assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6); - assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6); - assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(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.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter2, filter1, filter5); - assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, new FilterList(Operator.MUST_PASS_ALL, filter3, filter4)); - assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter7); - assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1)); } @Test @@ -776,33 +776,33 @@ public class TestFilterList { MockFilter filter5 = new MockFilter(ReturnCode.SKIP); MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT); FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2); - assertEquals(filterList.filterKeyValue(kv1), ReturnCode.INCLUDE); + assertEquals(filterList.filterCell(kv1), ReturnCode.INCLUDE); filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, filter3); - assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter5, filter6); - assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter6); - assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter1); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter2, filter1, filter5); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, new FilterList(Operator.MUST_PASS_ONE, filter3, filter4)); - assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, new FilterList(Operator.MUST_PASS_ONE, filter3, filter4)); - assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1)); filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6); - assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1)); + assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); } static class MockSeekHintFilter extends FilterBase { @@ -813,7 +813,7 @@ public class TestFilterList { } @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell v) throws IOException { return ReturnCode.SEEK_NEXT_USING_HINT; } @@ -844,7 +844,7 @@ public class TestFilterList { filterList.addFilter(filter2); filterList.addFilter(filter3); - Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1)); + Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); Assert.assertEquals(kv3, filterList.getNextCellHint(kv1)); filterList = new FilterList(Operator.MUST_PASS_ALL); @@ -853,7 +853,7 @@ public class TestFilterList { filterList.addFilter(filter2); filterList.addFilter(filter3); - Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1)); + Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1)); Assert.assertEquals(kv1, filterList.getNextCellHint(kv1)); } @@ -869,12 +869,12 @@ public class TestFilterList { prefixFilter.setReversed(true); filterList.addFilter(prefixFilter); filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22)); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22))); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22))); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2))); filterList.reset(); filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(new KeyValue(r1, r1, r1))); + assertEquals(ReturnCode.SKIP, filterList.filterCell(new KeyValue(r1, r1, r1))); filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); filterList.setReversed(true); @@ -885,12 +885,12 @@ public class TestFilterList { filterList.addFilter(alwaysNextColFilter); filterList.addFilter(prefixFilter); filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22)); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22))); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2))); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22))); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2))); filterList.reset(); filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1)); - assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(new KeyValue(r1, r1, r1))); + assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(new KeyValue(r1, r1, r1))); } @Test @@ -910,20 +910,20 @@ public class TestFilterList { FilterList keyOnlyFilterFirst = new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList); - assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv1)); c = keyOnlyFilterFirst.transformCell(kv1); assertEquals(0, c.getValueLength()); - assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv2)); + assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv2)); c = keyOnlyFilterFirst.transformCell(kv2); assertEquals(0, c.getValueLength()); internalFilterList.reset(); FilterList keyOnlyFilterLast = new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList); - assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv1)); + assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv1)); c = keyOnlyFilterLast.transformCell(kv1); assertEquals(0, c.getValueLength()); - assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv2)); + assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv2)); c = keyOnlyFilterLast.transformCell(kv2); assertEquals(0, c.getValueLength()); } @@ -933,18 +933,18 @@ public class TestFilterList { KeyValue kv = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"), 1, Bytes.toBytes("value")); FilterList filterList = new FilterList(Operator.MUST_PASS_ALL); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv)); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv)); assertEquals(kv, filterList.transformCell(kv)); filterList = new FilterList(Operator.MUST_PASS_ONE); - assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv)); + assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv)); assertEquals(kv, filterList.transformCell(kv)); } private static class MockNextRowFilter extends FilterBase { private int hitCount = 0; - public ReturnCode filterKeyValue(Cell v) throws IOException { + public ReturnCode filterCell(Cell v) throws IOException { hitCount++; return ReturnCode.NEXT_ROW; } @@ -962,8 +962,8 @@ public class TestFilterList { Bytes.toBytes("value")); MockNextRowFilter mockNextRowFilter = new MockNextRowFilter(); FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockNextRowFilter); - filter.filterKeyValue(kv1); - filter.filterKeyValue(kv2); + filter.filterCell(kv1); + filter.filterCell(kv2); assertEquals(2, mockNextRowFilter.getHitCount()); } } 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..dfee4146a2 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 @@ -40,7 +40,7 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase { /** * Test the functionality of - * {@link FirstKeyValueMatchingQualifiersFilter#filterKeyValue(org.apache.hadoop.hbase.Cell)} + * {@link FirstKeyValueMatchingQualifiersFilter#filterCell(org.apache.hadoop.hbase.Cell)} * * @throws Exception */ @@ -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.filterCell(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.filterCell(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.filterCell(cell)); assertTrue("includeFlagIsUnset", - filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1); + filter.filterCell(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.filterCell(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.filterCell(cell) == Filter.ReturnCode.NEXT_ROW); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index b6bc2f132d..c4e352bd96 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -166,7 +166,7 @@ public class TestInvocationRecordFilter { visitedKeyValues.clear(); } - public ReturnCode filterKeyValue(Cell ignored) { + public ReturnCode filterCell(Cell ignored) { visitedKeyValues.add(ignored); return ReturnCode.INCLUDE; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java index 0b1c368ad5..a5d04d2133 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java @@ -97,7 +97,7 @@ public class TestMultiRowRangeFilter { * Expected :SEEK_NEXT_USING_HINT * Actual :INCLUDE * */ - assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterCell(null)); } @Test @@ -107,15 +107,15 @@ public class TestMultiRowRangeFilter { new MultiRowRangeFilter.RowRange(Bytes.toBytes("d"), true, Bytes.toBytes("e"), true) )); filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("a"))); - assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterCell(null)); filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("b"))); - assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null)); filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("c"))); - assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null)); filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("d"))); - assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null)); filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("e"))); - assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null)); + assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null)); } @Test 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 2d223dca70..764d0336bf 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 @@ -37,7 +37,7 @@ import java.util.ArrayList; /** * Tests for {@link SingleColumnValueExcludeFilter}. Because this filter * extends {@link SingleColumnValueFilter}, only the added functionality is - * tested. That is, method filterKeyValue(KeyValue). + * tested. That is, method filterCell(Cell). * */ @Category({FilterTests.class, SmallTests.class}) @@ -50,17 +50,17 @@ public class TestSingleColumnValueExcludeFilter { private static final byte[] VAL_2 = Bytes.toBytes("ab"); /** - * Test the overridden functionality of filterKeyValue(KeyValue) + * Test the overridden functionality of filterCell(Cell) * @throws Exception */ @Test - public void testFilterKeyValue() throws Exception { + public void testFilterCell() throws Exception { Filter filter = new SingleColumnValueExcludeFilter(COLUMN_FAMILY, COLUMN_QUALIFIER, CompareOperator.EQUAL, VAL_1); // 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", CellComparatorImpl.COMPARATOR.compare(kvs.get(0), kv) == 0); - assertTrue("leftKV2", CellComparatorImpl.COMPARATOR.compare(kvs.get(1), kv) == 0); + assertTrue("leftKV1", CellComparatorImpl.COMPARATOR.compare(kvs.get(0), c) == 0); + assertTrue("leftKV2", CellComparatorImpl.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.filterCell(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.filterCell(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.filterCell(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..c1ef1eeec1 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,143 +104,143 @@ 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.filterCell(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); + assertTrue("less than", filter.filterCell(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.filterCell(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); + assertTrue("Equals 100", filter.filterCell(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.filterCell(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); + assertTrue("include 120", filter.filterCell(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.filterCell(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(); + assertTrue("basicFilter1", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3); + assertTrue("basicFilter2", filter.filterCell(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(); + assertTrue("basicFilter2", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4); + assertTrue("basicFilter3", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); - assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); + assertTrue("basicFilter3", filter.filterCell(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.filterCell(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(); + assertTrue("basicFilter4", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); + assertTrue("basicFilter4", filter.filterCell(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); + assertTrue("basicFilter4", filter.filterCell(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.filterCell(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(); + assertTrue("basicFilter5", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2); + assertTrue("basicFilter5", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); - assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); + assertTrue("basicFilter5", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); assertFalse("basicFilterNotNull", filter.filterRow()); } 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.filterCell(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); + assertTrue("null1", filter.filterCell(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.filterCell(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("null2", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); assertTrue("null2FilterRow", filter.filterRow()); } 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.filterCell(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, + assertTrue("substrTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_2); - assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + assertTrue("substrFalse", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); - assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); + assertTrue("substrFalse", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); assertFalse("substrFilterAllRemaining", filter.filterAllRemaining()); assertFalse("substrFilterNotNull", filter.filterRow()); } 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.filterCell(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, + assertTrue("regexTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); + cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_2); - assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE); - buffer = kv.getBuffer(); + assertTrue("regexFalse", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE); + buffer = cell.getBuffer(); c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length); - assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE); + assertTrue("regexFalse", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); assertFalse("regexFilterAllRemaining", filter.filterAllRemaining()); assertFalse("regexFilterNotNull", filter.filterRow()); } 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.filterCell(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); + assertTrue("regexTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE); assertFalse("regexFilterAllRemaining", filter.filterAllRemaining()); assertFalse("regexFilterNotNull", filter.filterRow()); } 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 41bd99780b..c41a271265 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 @@ -3451,7 +3451,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/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 138260358e..1719ce6754 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -1140,7 +1140,7 @@ public class TestHStore { } }, new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) throws IOException { + public Filter.ReturnCode filterCell(Cell v) throws IOException { return ReturnCode.INCLUDE; } }, expectedSize); @@ -1164,7 +1164,7 @@ public class TestHStore { } }, new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) throws IOException { + public Filter.ReturnCode filterCell(Cell v) throws IOException { if (timeToGoNextRow.get()) { timeToGoNextRow.set(false); return ReturnCode.NEXT_ROW; @@ -1193,7 +1193,7 @@ public class TestHStore { } }, new FilterBase() { @Override - public Filter.ReturnCode filterKeyValue(Cell v) throws IOException { + public Filter.ReturnCode filterCell(Cell v) throws IOException { if (timeToGetHint.get()) { timeToGetHint.set(false); return Filter.ReturnCode.SEEK_NEXT_USING_HINT; 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 b9e77c6280..69cdaa7633 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 @@ -273,7 +273,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-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.13.5 (Apple Git-94)