From 8410eaa295bdc1902e19e26fbb4967c52f542e4a Mon Sep 17 00:00:00 2001 From: Jan Hentschel Date: Wed, 17 Apr 2019 20:25:07 +0200 Subject: [PATCH] HBASE-22262 Removed deprecated methods from Filter class --- .../hbase/filter/ColumnCountGetFilter.java | 6 --- .../hbase/filter/ColumnPaginationFilter.java | 6 --- .../hbase/filter/ColumnPrefixFilter.java | 6 --- .../hbase/filter/ColumnRangeFilter.java | 6 --- .../hbase/filter/DependentColumnFilter.java | 10 ---- .../hadoop/hbase/filter/FamilyFilter.java | 6 --- .../apache/hadoop/hbase/filter/Filter.java | 50 +------------------ .../hadoop/hbase/filter/FilterBase.java | 18 +------ .../hadoop/hbase/filter/FilterList.java | 11 ---- .../hadoop/hbase/filter/FilterListBase.java | 5 -- .../hbase/filter/FilterListWithAND.java | 15 ------ .../hadoop/hbase/filter/FilterListWithOR.java | 15 ------ .../hbase/filter/FirstKeyOnlyFilter.java | 6 --- ...FirstKeyValueMatchingQualifiersFilter.java | 6 --- .../hadoop/hbase/filter/FuzzyRowFilter.java | 6 --- .../hbase/filter/InclusiveStopFilter.java | 6 --- .../hadoop/hbase/filter/KeyOnlyFilter.java | 6 --- .../hbase/filter/MultiRowRangeFilter.java | 6 --- .../filter/MultipleColumnPrefixFilter.java | 6 --- .../hadoop/hbase/filter/PageFilter.java | 6 --- .../hadoop/hbase/filter/PrefixFilter.java | 6 --- .../hadoop/hbase/filter/QualifierFilter.java | 6 --- .../hadoop/hbase/filter/RandomRowFilter.java | 6 --- .../apache/hadoop/hbase/filter/RowFilter.java | 6 --- .../hbase/filter/SingleColumnValueFilter.java | 6 --- .../hadoop/hbase/filter/SkipFilter.java | 6 --- .../hadoop/hbase/filter/TimestampsFilter.java | 6 --- .../hadoop/hbase/filter/ValueFilter.java | 6 --- .../hadoop/hbase/filter/WhileMatchFilter.java | 15 +----- .../hadoop/hbase/rest/TestTableScan.java | 6 ++- .../hadoop/hbase/filter/FilterWrapper.java | 13 ----- .../client/TestConnectionImplementation.java | 12 ----- .../TestUserScanQueryMatcher.java | 5 +- 33 files changed, 10 insertions(+), 291 deletions(-) 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 4ecf2d7bf4da..44a2b0fd1bc7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java @@ -63,12 +63,6 @@ public boolean filterAllRemaining() { return this.count > this.limit; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { this.count++; 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 5d5321b6909e..f11ddbd27922 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java @@ -111,12 +111,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Override - @Deprecated - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { 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 a016b8c29f42..3ad0f1783c56 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,12 +58,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell cell) { if (this.prefix == null) { 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 ea4d7e910bfa..9937a663d453 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java @@ -122,12 +122,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Override - @Deprecated - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { int cmpMin = 1; 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 857bfacdce68..ee79ac391ff1 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 @@ -133,12 +133,6 @@ public boolean filterAllRemaining() { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { // Check if the column and qualifier match @@ -173,10 +167,6 @@ public boolean filterRow() { return false; } - @Override - public boolean filterRowKey(byte[] buffer, int offset, int length) { - return false; - } @Override public void reset() { stampSet.clear(); 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 74c6d293bda0..f1406cd4c945 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 @@ -56,12 +56,6 @@ public FamilyFilter(final CompareOperator op, super(op, familyComparator); } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { int familyLength = c.getFamilyLength(); 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 dec8e061b549..8fba32c34294 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 @@ -64,24 +64,6 @@ public abstract class Filter { */ abstract public void reset() throws IOException; - /** - * 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 #filterCell(Cell)} below. - * - * Concrete implementers can signal a failure condition in their code by throwing an - * {@link IOException}. - * - * @param buffer buffer containing row key - * @param offset offset into buffer where row key starts - * @param length length of the row key - * @return true, remove entire row, false, include the row (maybe). - * @throws IOException in case an I/O or an filter specific failure needs to be signaled. - * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. - * Instead use {@link #filterRowKey(Cell)} - */ - @Deprecated - abstract public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException; - /** * 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 #filterCell(Cell)} below. @@ -108,34 +90,6 @@ public abstract class Filter { */ abstract public boolean filterAllRemaining() throws IOException; - /** - * A way to filter based on the column family, column qualifier and/or the column value. Return - * code is described below. This allows filters to filter only certain number of columns, then - * terminate without matching ever column. - * - * If filterRowKey returns true, filterKeyValue needs to be consistent with it. - * - * filterKeyValue can assume that filterRowKey has already been called for the row. - * - * If your filter returns ReturnCode.NEXT_ROW, it should return - * ReturnCode.NEXT_ROW until {@link #reset()} is called just in case the caller calls - * for the next row. - * - * 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, 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 @@ -157,8 +111,8 @@ public ReturnCode filterKeyValue(final Cell c) throws IOException { * @throws IOException in case an I/O or an filter specific failure needs to be signaled. * @see Filter.ReturnCode */ - public ReturnCode filterCell(final Cell c) throws IOException{ - return filterKeyValue(c); + public ReturnCode filterCell(final Cell c) throws IOException { + return ReturnCode.INCLUDE; } /** 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 7401e4cc38fb..9fb796af45b7 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 @@ -47,25 +47,9 @@ public abstract class FilterBase extends Filter { public void reset() throws IOException { } - /** - * Filters that do not filter by row key can inherit this implementation that - * never filters anything. (ie: returns false). - * - * {@inheritDoc} - * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. - * Instead use {@link #filterRowKey(Cell)} - */ - @Override - @Deprecated - public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { - if (filterAllRemaining()) return true; - return false; - } - @Override public boolean filterRowKey(Cell cell) throws IOException { - if (filterAllRemaining()) return true; - return filterRowKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); + return filterAllRemaining(); } /** 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 1f14a45279ee..32fa799e8749 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 @@ -146,11 +146,6 @@ public void reset() throws IOException { filterListBase.reset(); } - @Override - public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException { - return filterListBase.filterRowKey(rowKey, offset, length); - } - @Override public boolean filterRowKey(Cell firstRowCell) throws IOException { return filterListBase.filterRowKey(firstRowCell); @@ -166,12 +161,6 @@ public Cell transformCell(Cell c) throws IOException { return filterListBase.transformCell(c); } - @Override - @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); 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 ad24d0e47d9e..a9defbf0240f 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 @@ -112,11 +112,6 @@ public Cell transformCell(Cell c) throws IOException { return transformed; } - @Override - public ReturnCode filterKeyValue(final Cell c) throws IOException { - return filterCell(c); - } - /** * Filters that never filter by modifying the returned List of Cells can inherit this * implementation that does nothing. {@inheritDoc} 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 76cda02932c0..04aad2ce5195 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 @@ -199,21 +199,6 @@ public void reset() throws IOException { seekHintFilters.clear(); } - @Override - public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException { - if (isEmpty()) { - return super.filterRowKey(rowKey, offset, length); - } - boolean retVal = false; - for (int i = 0, n = filters.size(); i < n; i++) { - Filter filter = filters.get(i); - if (filter.filterAllRemaining() || filter.filterRowKey(rowKey, offset, length)) { - retVal = true; - } - } - return retVal; - } - @Override public boolean filterRowKey(Cell firstRowCell) throws IOException { if (isEmpty()) { 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 28540a4fa13f..725260ef7e51 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 @@ -307,21 +307,6 @@ public void reset() throws IOException { } } - @Override - public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException { - if (isEmpty()) { - return super.filterRowKey(rowKey, offset, length); - } - boolean retVal = true; - for (int i = 0, n = filters.size(); i < n; i++) { - Filter filter = filters.get(i); - if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey, offset, length)) { - retVal = false; - } - } - return retVal; - } - @Override public boolean filterRowKey(Cell firstRowCell) throws IOException { if (isEmpty()) { 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 a258ec17c317..dc4207d821cf 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 @@ -53,12 +53,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if(foundKV) return ReturnCode.NEXT_ROW; 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 4cde2bfcdeef..c59aa6ee6c0b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java @@ -61,12 +61,6 @@ public FirstKeyValueMatchingQualifiersFilter(Set qualifiers) { this.qualifiers = qualifiers; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (hasFoundKV()) { 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 abfa7e1d6e8c..f6811f607bde 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,12 +147,6 @@ private boolean isPreprocessedMask(byte[] mask) { return true; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0; 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 cfaf18df21b1..7c0966856d83 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,12 +51,6 @@ public byte[] getStopRowKey() { return this.stopRowKey; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (done) return ReturnCode.NEXT_ROW; 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 9603304520e6..26e53cff39d1 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 @@ -74,12 +74,6 @@ private Cell createKeyOnlyCell(Cell c) { } } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell ignored) throws IOException { - return filterCell(ignored); - } - @Override public ReturnCode filterCell(final Cell ignored) throws IOException { return ReturnCode.INCLUDE; 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 32e22d8a6c67..1e25d6d70312 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 @@ -137,12 +137,6 @@ public boolean filterRowKey(Cell firstRowCell) { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell ignored) { - return filterCell(ignored); - } - @Override public ReturnCode filterCell(final 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 5fd64451a45c..357f19c8831d 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 @@ -68,12 +68,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (sortedPrefixes.isEmpty()) { 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 b1228d9daed6..98831c6f5baf 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 @@ -67,12 +67,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) throws IOException { - return filterCell(c); - } - @Override public ReturnCode filterCell(final 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 15c18fcb248c..8300005edb35 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -76,12 +76,6 @@ public boolean filterRowKey(Cell firstRowCell) { return filterRow; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (filterRow) return ReturnCode.NEXT_ROW; 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 7b6167f0746e..497f63314e60 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 @@ -55,12 +55,6 @@ public QualifierFilter(final CompareOperator op, super(op, qualifierComparator); } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (compareQualifier(getCompareOperator(), this.comparator, c)) { 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 2d81878f77ad..e57cff54539c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java @@ -70,12 +70,6 @@ public boolean filterAllRemaining() { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (filterOutRow) { 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 fcda1e09ee84..19c934808ea8 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 @@ -61,12 +61,6 @@ public void reset() { this.filterOutRow = false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell v) { if(this.filterOutRow) { 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 92422c9a853f..5563ae70ec4d 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 @@ -178,12 +178,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue())); 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 fe5caad1b27d..03e1f2cabe1b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java @@ -79,12 +79,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - 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); 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 c2cdf9975c51..5e7fb5c490fb 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 @@ -110,12 +110,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return false; } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (this.timestamps.contains(c.getTimestamp())) { 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 38fe45160e71..5b5ec6f619da 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 @@ -54,12 +54,6 @@ public ValueFilter(final CompareOperator valueCompareOp, super(valueCompareOp, valueComparator); } - @Deprecated - @Override - public ReturnCode filterKeyValue(final Cell c) { - return filterCell(c); - } - @Override public ReturnCode filterCell(final Cell c) { if (compareValue(getCompareOperator(), this.comparator, c)) { 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 e053886f6319..5bfdaa35574b 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 @@ /** * A wrapper filter that returns true from {@link #filterAllRemaining()} as soon - * as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)}, + * as the wrapped filters {@link Filter#filterRowKey(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 @@ -64,13 +64,6 @@ public boolean filterAllRemaining() throws IOException { return this.filterAllRemaining || this.filter.filterAllRemaining(); } - @Override - public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { - boolean value = filter.filterRowKey(buffer, offset, length); - changeFAR(value); - return value; - } - @Override public boolean filterRowKey(Cell cell) throws IOException { if (filterAllRemaining()) return true; @@ -79,12 +72,6 @@ public boolean filterRowKey(Cell cell) throws IOException { return value; } - @Deprecated - @Override - 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); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java index e048550dbdfb..16eeda0a2dad 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java @@ -48,6 +48,7 @@ import javax.xml.parsers.SAXParserFactory; import javax.xml.stream.XMLStreamException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -649,8 +650,9 @@ public CustomFilter(byte[] key) { } @Override - public boolean filterRowKey(byte[] buffer, int offset, int length) { - int cmp = Bytes.compareTo(buffer, offset, length, this.key, 0, this.key.length); + public boolean filterRowKey(Cell cell) { + int cmp = Bytes.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), + this.key, 0, this.key.length); return cmp != 0; } 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 index 9bc072a048ec..74afc69118a7 100644 --- 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 @@ -100,25 +100,12 @@ 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); - } - @Override public ReturnCode filterCell(final Cell c) throws IOException { return this.filter.filterCell(c); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java index f27c0102a1b2..5a7554a265ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java @@ -503,18 +503,6 @@ public void run() { protected static final AtomicBoolean syncBlockingFilter = new AtomicBoolean(false); public static class BlockingFilter extends FilterBase { - @Override - public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException { - int i = 0; - while (i++ < 1000 && !syncBlockingFilter.get()) { - synchronized (syncBlockingFilter) { - syncBlockingFilter.notifyAll(); - } - Threads.sleep(100); - } - syncBlockingFilter.set(true); - return false; - } @Override public ReturnCode filterCell(final Cell ignored) throws IOException { return ReturnCode.INCLUDE; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java index 861b83e2fe59..7be1db31e154 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java @@ -248,9 +248,8 @@ public void testMatch_ExpiredWildcard() throws IOException { } private static class AlwaysIncludeAndSeekNextRowFilter extends FilterBase { - @Override - public ReturnCode filterKeyValue(final Cell c) throws IOException { + public ReturnCode filterCell(final Cell c) { return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW; } } @@ -294,7 +293,7 @@ scanWithFilter, new ScanInfo(this.conf, fam2, 0, 1, ttl, KeepDeletedCells.FALSE, private static class AlwaysIncludeFilter extends FilterBase { @Override - public ReturnCode filterKeyValue(final Cell c) throws IOException { + public ReturnCode filterCell(final Cell c) { return ReturnCode.INCLUDE; } }