Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 086F9200D3B for ; Fri, 27 Oct 2017 07:47:25 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0387F1609E5; Fri, 27 Oct 2017 05:47:25 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7C5C2160BF3 for ; Fri, 27 Oct 2017 07:47:22 +0200 (CEST) Received: (qmail 96052 invoked by uid 500); 27 Oct 2017 05:47:21 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 96043 invoked by uid 99); 27 Oct 2017 05:47:21 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Oct 2017 05:47:21 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 5F708DFC49; Fri, 27 Oct 2017 05:47:20 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: anoopsamjohn@apache.org To: commits@hbase.apache.org Date: Fri, 27 Oct 2017 05:47:21 -0000 Message-Id: <0795fdf1eab34ab68a8d875fe3eb5303@git.apache.org> In-Reply-To: <589164ba90074b728dff684f36c09fc4@git.apache.org> References: <589164ba90074b728dff684f36c09fc4@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/2] hbase git commit: HBASE-13346: Clean up Filter package for post 1.0 s/KeyValue/Cell/g archived-at: Fri, 27 Oct 2017 05:47:25 -0000 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 Signed-off-by: anoopsamjohn Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/82b1c320 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/82b1c320 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/82b1c320 Branch: refs/heads/master Commit: 82b1c320f072a8032c9727a93ae106b5fcdc8108 Parents: 660ead0 Author: Tamas Penzes Authored: Wed Oct 25 22:20:16 2017 +0200 Committer: anoopsamjohn Committed: Fri Oct 27 11:16:13 2017 +0530 ---------------------------------------------------------------------- .../org/apache/hadoop/hbase/client/Query.java | 2 +- .../hbase/filter/ColumnCountGetFilter.java | 10 +- .../hbase/filter/ColumnPaginationFilter.java | 12 +- .../hadoop/hbase/filter/ColumnPrefixFilter.java | 10 +- .../hadoop/hbase/filter/ColumnRangeFilter.java | 12 +- .../hbase/filter/DependentColumnFilter.java | 8 +- .../hadoop/hbase/filter/FamilyFilter.java | 12 +- .../org/apache/hadoop/hbase/filter/Filter.java | 46 ++++- .../apache/hadoop/hbase/filter/FilterBase.java | 2 +- .../apache/hadoop/hbase/filter/FilterList.java | 20 +- .../hadoop/hbase/filter/FilterListBase.java | 17 +- .../hadoop/hbase/filter/FilterListWithAND.java | 8 +- .../hadoop/hbase/filter/FilterListWithOR.java | 6 +- .../hadoop/hbase/filter/FilterWrapper.java | 175 ---------------- .../hadoop/hbase/filter/FirstKeyOnlyFilter.java | 10 +- .../FirstKeyValueMatchingQualifiersFilter.java | 16 +- .../hadoop/hbase/filter/FuzzyRowFilter.java | 8 +- .../hbase/filter/InclusiveStopFilter.java | 10 +- .../hadoop/hbase/filter/KeyOnlyFilter.java | 10 +- .../hbase/filter/MultiRowRangeFilter.java | 8 +- .../filter/MultipleColumnPrefixFilter.java | 12 +- .../apache/hadoop/hbase/filter/PageFilter.java | 8 +- .../hadoop/hbase/filter/PrefixFilter.java | 10 +- .../hadoop/hbase/filter/QualifierFilter.java | 12 +- .../hadoop/hbase/filter/RandomRowFilter.java | 10 +- .../apache/hadoop/hbase/filter/RowFilter.java | 8 +- .../hbase/filter/SingleColumnValueFilter.java | 8 +- .../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 +- .../apache/hadoop/hbase/mapreduce/Import.java | 18 +- .../hadoop/hbase/filter/FilterWrapper.java | 180 ++++++++++++++++ .../regionserver/MobReferenceOnlyFilter.java | 2 +- .../querymatcher/ColumnTracker.java | 4 +- .../querymatcher/DeleteTracker.java | 14 +- .../querymatcher/ScanDeleteTracker.java | 6 +- .../querymatcher/UserScanQueryMatcher.java | 4 +- .../security/access/AccessControlFilter.java | 2 +- .../visibility/VisibilityController.java | 2 +- .../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 +- .../filter/TestColumnPaginationFilter.java | 4 +- .../hbase/filter/TestDependentColumnFilter.java | 6 +- .../apache/hadoop/hbase/filter/TestFilter.java | 8 +- .../hbase/filter/TestFilterFromRegionSide.java | 2 +- .../hadoop/hbase/filter/TestFilterList.java | 204 +++++++++---------- ...stFirstKeyValueMatchingQualifiersFilter.java | 26 +-- .../filter/TestInvocationRecordFilter.java | 2 +- .../hbase/filter/TestMultiRowRangeFilter.java | 12 +- .../TestSingleColumnValueExcludeFilter.java | 24 +-- .../filter/TestSingleColumnValueFilter.java | 136 ++++++------- .../hadoop/hbase/regionserver/TestHRegion.java | 2 +- .../hadoop/hbase/regionserver/TestHStore.java | 6 +- .../TestScannerHeartbeatMessages.java | 2 +- .../hbase/spark/SparkSQLPushDownFilter.java | 2 +- 60 files changed, 698 insertions(+), 514 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Query.java ---------------------------------------------------------------------- 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 cb09f76..c61efa3 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java ---------------------------------------------------------------------- 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 d41e6df..120a4bb 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) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java ---------------------------------------------------------------------- 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 5faf85d..f61f418 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(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java ---------------------------------------------------------------------- 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 8a936d4..e571bb3 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) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java ---------------------------------------------------------------------- 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 87057a9..e3da170 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(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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) { http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java ---------------------------------------------------------------------- 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 2f936e8..020fae6 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) { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell c) { + public ReturnCode filterCell(final 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java ---------------------------------------------------------------------- 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 2f5c338..6745599 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) { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell v) { - int familyLength = v.getFamilyLength(); + public ReturnCode filterCell(final 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; } } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java ---------------------------------------------------------------------- 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 a92ea0b..dec8e06 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). * http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java ---------------------------------------------------------------------- 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 e1dc4fa..86a5b39 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} http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java ---------------------------------------------------------------------- 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 d4242ae..4f9a8d8 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 - public ReturnCode filterKeyValue(Cell c) throws IOException { - return filterListBase.filterKeyValue(c); + @Deprecated + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) throws IOException { + return filterListBase.filterCell(c); } /** http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java ---------------------------------------------------------------------- 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 f92d2e7..cd36974 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); + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) throws IOException { + return internalFilterCell(c, c); } /** http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java ---------------------------------------------------------------------- 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 755dd98..edb5d3e 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 @@ -58,7 +58,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. *
    *
    @@ -147,7 +147,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; } @@ -162,9 +162,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); http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java ---------------------------------------------------------------------- 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 e51915b..51886bc 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]. http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java deleted file mode 100644 index c0fa6cf..0000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Copyright The Apache Software Foundation - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.filter; - -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.hbase.Cell; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; - -/** - * This is a Filter wrapper class which is used in the server side. Some filter - * 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 { - Filter filter = null; - - public FilterWrapper( Filter filter ) { - if (null == filter) { - // ensure the filter instance is not null - throw new NullPointerException("Cannot create FilterWrapper with null Filter"); - } - this.filter = filter; - } - - /** - * @return The filter serialized using pb - */ - public byte[] toByteArray() throws IOException { - FilterProtos.FilterWrapper.Builder builder = - FilterProtos.FilterWrapper.newBuilder(); - builder.setFilter(ProtobufUtil.toFilter(this.filter)); - return builder.build().toByteArray(); - } - - /** - * @param pbBytes A pb serialized {@link FilterWrapper} instance - * @return An instance of {@link FilterWrapper} made from bytes - * @throws org.apache.hadoop.hbase.exceptions.DeserializationException - * @see #toByteArray - */ - public static FilterWrapper parseFrom(final byte [] pbBytes) - throws DeserializationException { - FilterProtos.FilterWrapper proto; - try { - proto = FilterProtos.FilterWrapper.parseFrom(pbBytes); - } catch (InvalidProtocolBufferException e) { - throw new DeserializationException(e); - } - try { - return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter())); - } catch (IOException ioe) { - throw new DeserializationException(ioe); - } - } - - @Override - public void reset() throws IOException { - this.filter.reset(); - } - - @Override - public boolean filterAllRemaining() throws IOException { - return this.filter.filterAllRemaining(); - } - - @Override - public boolean filterRow() throws IOException { - return this.filter.filterRow(); - } - - @Override - public Cell getNextCellHint(Cell currentCell) throws IOException { - return this.filter.getNextCellHint(currentCell); - } - - @Override - public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { - // No call to this. - if (filterAllRemaining()) return true; - return this.filter.filterRowKey(buffer, offset, length); - } - - @Override - public boolean filterRowKey(Cell cell) throws IOException { - if (filterAllRemaining()) return true; - return this.filter.filterRowKey(cell); - } - - @Override - public ReturnCode filterKeyValue(Cell v) throws IOException { - return this.filter.filterKeyValue(v); - } - - @Override - public Cell transformCell(Cell v) throws IOException { - return this.filter.transformCell(v); - } - - @Override - public boolean hasFilterRow() { - return this.filter.hasFilterRow(); - } - - @Override - public void filterRowCells(List kvs) throws IOException { - filterRowCellsWithRet(kvs); - } - - public enum FilterRowRetCode { - NOT_CALLED, - INCLUDE, // corresponds to filter.filterRow() returning false - EXCLUDE, // corresponds to filter.filterRow() returning true - INCLUDE_THIS_FAMILY // exclude other families - } - public FilterRowRetCode filterRowCellsWithRet(List kvs) throws IOException { - //To fix HBASE-6429, - //Filter with filterRow() returning true is incompatible with scan with limit - //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented. - //2. filterRow() is merged with filterRow(kvs), - //so that to make all those row related filtering stuff in the same function. - this.filter.filterRowCells(kvs); - if (!kvs.isEmpty()) { - if (this.filter.filterRow()) { - kvs.clear(); - return FilterRowRetCode.EXCLUDE; - } - return FilterRowRetCode.INCLUDE; - } - return FilterRowRetCode.NOT_CALLED; - } - - @Override - public boolean isFamilyEssential(byte[] name) throws IOException { - return filter.isFamilyEssential(name); - } - - /** - * @param other - * @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. - */ - boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof FilterWrapper)) return false; - - FilterWrapper other = (FilterWrapper)o; - return this.filter.areSerializedFieldsEqual(other.filter); - } -} http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java ---------------------------------------------------------------------- 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 79e6329..e0244da 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) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java ---------------------------------------------------------------------- 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 e4f1fbe..f994b15 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 @@ -60,19 +60,25 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { this.qualifiers = qualifiers; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) { if (hasFoundKV()) { return ReturnCode.NEXT_ROW; - } else if (hasOneMatchingQualifier(v)) { + } else if (hasOneMatchingQualifier(c)) { setFoundKV(true); } return ReturnCode.INCLUDE; } - private boolean hasOneMatchingQualifier(Cell v) { + private boolean hasOneMatchingQualifier(Cell c) { for (byte[] q : qualifiers) { - if (CellUtil.matchingQualifier(v, q)) { + if (CellUtil.matchingQualifier(c, q)) { return true; } } @@ -114,7 +120,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java ---------------------------------------------------------------------- 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 244e8fb..62099e4 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 @@ -147,8 +147,14 @@ public class FuzzyRowFilter extends FilterBase { return true; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell c) { + public ReturnCode filterKeyValue(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) { final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0; final int size = fuzzyKeysData.size(); for (int i = startIndex; i < size + startIndex; i++) { http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java ---------------------------------------------------------------------- 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 c467d17..79ebfe5 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 @@ -51,8 +51,14 @@ public class InclusiveStopFilter extends FilterBase { return this.stopRowKey; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) { if (done) return ReturnCode.NEXT_ROW; return ReturnCode.INCLUDE; } @@ -105,7 +111,7 @@ public class InclusiveStopFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java ---------------------------------------------------------------------- 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 0f71ca9..b23677b 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 ignored) throws IOException { + return filterCell(ignored); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java ---------------------------------------------------------------------- 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 690aa3f..a3f4159 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 @@ -124,8 +124,14 @@ public class MultiRowRangeFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell ignored) { + public ReturnCode filterKeyValue(final Cell ignored) { + return filterCell(ignored); + } + + @Override + public ReturnCode filterCell(final Cell ignored) { return currentReturnCode; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java ---------------------------------------------------------------------- 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 7816451..83d56a3 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) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java ---------------------------------------------------------------------- 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 2579f9d..8949896 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(final Cell ignored) throws IOException { return ReturnCode.INCLUDE; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java ---------------------------------------------------------------------- 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 455bd05..4fb2370 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 @@ -75,8 +75,14 @@ public class PrefixFilter extends FilterBase { return filterRow; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) { if (filterRow) return ReturnCode.NEXT_ROW; return ReturnCode.INCLUDE; } @@ -128,7 +134,7 @@ public class PrefixFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java ---------------------------------------------------------------------- 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 9719a95..8f3c859 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) { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell v) { - int qualifierLength = v.getQualifierLength(); + public ReturnCode filterCell(final Cell c) { + int qualifierLength = c.getQualifierLength(); if (qualifierLength > 0) { - if (compareQualifier(getCompareOperator(), this.comparator, v)) { + if (compareQualifier(getCompareOperator(), this.comparator, c)) { return ReturnCode.SKIP; } } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java ---------------------------------------------------------------------- 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 1df18c9..1a1248e 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 @@ -69,8 +69,14 @@ public class RandomRowFilter extends FilterBase { return false; } + @Deprecated @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterKeyValue(final Cell c) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) { if (filterOutRow) { return ReturnCode.NEXT_ROW; } @@ -134,7 +140,7 @@ public class RandomRowFilter extends FilterBase { } /** - * @param other + * @param o the other filter to compare with * @return true if and only if the fields of the filter that are serialized * are equal to the corresponding fields in other. Used for testing. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java ---------------------------------------------------------------------- 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 d0ab6e9..bfd88b9 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) { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell v) { + public ReturnCode filterCell(final Cell v) { if(this.filterOutRow) { return ReturnCode.NEXT_ROW; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java ---------------------------------------------------------------------- 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 468af8c..a66c209 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) { + return filterCell(c); + } + @Override - public ReturnCode filterKeyValue(Cell c) { + public ReturnCode filterCell(final 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java ---------------------------------------------------------------------- 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 f3d7e6b..adfe1c1 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(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java ---------------------------------------------------------------------- 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 d77ece7..491d2ed 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) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java ---------------------------------------------------------------------- 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 d4f41cf..17de4ff 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) { + return filterCell(c); + } + + @Override + public ReturnCode filterCell(final Cell c) { + if (compareValue(getCompareOperator(), this.comparator, c)) { return ReturnCode.SKIP; } return ReturnCode.INCLUDE; http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java ---------------------------------------------------------------------- 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 3560178..ab6200f 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(final 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. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java ---------------------------------------------------------------------- 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 acc3c62..3bb966a 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java new file mode 100644 index 0000000..a7607be --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java @@ -0,0 +1,180 @@ +/* + * Copyright The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.filter; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hbase.Cell; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; + +/** + * This is a Filter wrapper class which is used in the server side. Some filter + * 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 { + Filter filter = null; + + public FilterWrapper( Filter filter ) { + if (null == filter) { + // ensure the filter instance is not null + throw new NullPointerException("Cannot create FilterWrapper with null Filter"); + } + this.filter = filter; + } + + /** + * @return The filter serialized using pb + */ + public byte[] toByteArray() throws IOException { + FilterProtos.FilterWrapper.Builder builder = + FilterProtos.FilterWrapper.newBuilder(); + builder.setFilter(ProtobufUtil.toFilter(this.filter)); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link FilterWrapper} instance + * @return An instance of {@link FilterWrapper} made from bytes + * @throws org.apache.hadoop.hbase.exceptions.DeserializationException + * @see #toByteArray + */ + public static FilterWrapper parseFrom(final byte [] pbBytes) + throws DeserializationException { + FilterProtos.FilterWrapper proto; + try { + proto = FilterProtos.FilterWrapper.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + try { + return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter())); + } catch (IOException ioe) { + throw new DeserializationException(ioe); + } + } + + @Override + public void reset() throws IOException { + this.filter.reset(); + } + + @Override + public boolean filterAllRemaining() throws IOException { + return this.filter.filterAllRemaining(); + } + + @Override + public boolean filterRow() throws IOException { + return this.filter.filterRow(); + } + + @Override + public Cell getNextCellHint(Cell currentCell) throws IOException { + return this.filter.getNextCellHint(currentCell); + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { + // No call to this. + if (filterAllRemaining()) return true; + return this.filter.filterRowKey(buffer, offset, length); + } + + @Override + public boolean filterRowKey(Cell cell) throws IOException { + if (filterAllRemaining()) return true; + return this.filter.filterRowKey(cell); + } + + @Deprecated + @Override + public ReturnCode filterKeyValue(final Cell c) throws IOException { + return filterCell(c); + } + + public ReturnCode filterCell(final Cell c) throws IOException { + return this.filter.filterCell(c); + } + + @Override + public Cell transformCell(Cell v) throws IOException { + return this.filter.transformCell(v); + } + + @Override + public boolean hasFilterRow() { + return this.filter.hasFilterRow(); + } + + @Override + public void filterRowCells(List kvs) throws IOException { + filterRowCellsWithRet(kvs); + } + + public enum FilterRowRetCode { + NOT_CALLED, + INCLUDE, // corresponds to filter.filterRow() returning false + EXCLUDE, // corresponds to filter.filterRow() returning true + INCLUDE_THIS_FAMILY // exclude other families + } + public FilterRowRetCode filterRowCellsWithRet(List kvs) throws IOException { + //To fix HBASE-6429, + //Filter with filterRow() returning true is incompatible with scan with limit + //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented. + //2. filterRow() is merged with filterRow(kvs), + //so that to make all those row related filtering stuff in the same function. + this.filter.filterRowCells(kvs); + if (!kvs.isEmpty()) { + if (this.filter.filterRow()) { + kvs.clear(); + return FilterRowRetCode.EXCLUDE; + } + return FilterRowRetCode.INCLUDE; + } + return FilterRowRetCode.NOT_CALLED; + } + + @Override + public boolean isFamilyEssential(byte[] name) throws IOException { + return filter.isFamilyEssential(name); + } + + /** + * @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. + */ + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof FilterWrapper)) return false; + + FilterWrapper other = (FilterWrapper)o; + return this.filter.areSerializedFieldsEqual(other.filter); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java ---------------------------------------------------------------------- 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 bddefc5..b544ea4 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(final Cell cell) { if (null != cell) { // If a cell with a mob reference tag, it's included. if (MobUtils.isMobReferenceCell(cell)) { http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ColumnTracker.java ---------------------------------------------------------------------- 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 b432e9d..15dea6b 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}. http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DeleteTracker.java ---------------------------------------------------------------------- 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 8f0a107..be9c51e 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: *
      *
    • {@link #add} when encountering a Delete
    • - *
    • {@link #isDeleted} when checking if a Put KeyValue has been deleted
    • + *
    • {@link #isDeleted} when checking if a Put Cell has been deleted
    • *
    • {@link #update} when reaching the end of a StoreFile
    • *
    */ @@ -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) } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanDeleteTracker.java ---------------------------------------------------------------------- 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 0daeb91..26da698 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: *
      *
    • {@link #add} when encountering a Delete or DeleteColumn
    • - *
    • {@link #isDeleted} when checking if a Put KeyValue has been deleted
    • + *
    • {@link #isDeleted} when checking if a Put Cell has been deleted
    • *
    • {@link #update} when reaching the end of a StoreFile or row for scans
    • *
    *

    @@ -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 */ http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/UserScanQueryMatcher.java ---------------------------------------------------------------------- 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 5f76f3e..16afa83 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java ---------------------------------------------------------------------- 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 9a3353c..9fc63e1 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(final Cell cell) { if (isSystemTable) { return ReturnCode.INCLUDE; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java ---------------------------------------------------------------------- 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 b6742be..a3841aa 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(final Cell cell) throws IOException { List putVisTags = new ArrayList<>(); Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags); if (putVisTags.isEmpty() && deleteCellVisTags.isEmpty()) { http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java ---------------------------------------------------------------------- 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 dcd301f..f9a04cc 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(final Cell cell) throws IOException { if (curFamily.getBytes() == null || !(CellUtil.matchingFamily(cell, curFamily.getBytes(), curFamily.getOffset(), curFamily.getLength()))) { http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java ---------------------------------------------------------------------- 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 ffd8c01..14f2caa 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(final 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; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java ---------------------------------------------------------------------- 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 7841901..5bede2a 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(final Cell c) { count++; return count > limit ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java ---------------------------------------------------------------------- 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 e2bdaf4..6f20118 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(final Cell ignored) throws IOException { return ReturnCode.INCLUDE; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java ---------------------------------------------------------------------- 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 9e7c184..52b0e55 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(final Cell ignored) { return ReturnCode.SKIP; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java ---------------------------------------------------------------------- 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 ab323de..40628db 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(final Cell c) throws IOException { return ReturnCode.SKIP; } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java ---------------------------------------------------------------------- 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 828a272..fc012d2 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); } /** http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java ---------------------------------------------------------------------- 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 74503bf..d84fbe9 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); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/82b1c320/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java ---------------------------------------------------------------------- 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 d225b8d..070a26c 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(final 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(final Cell ignored) throws IOException { return ReturnCode.INCLUDE; }