You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ja...@apache.org on 2019/06/09 22:47:20 UTC
[hbase] branch master updated: HBASE-22262 Removed deprecated
methods from Filter class
This is an automated email from the ASF dual-hosted git repository.
janh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new f620285 HBASE-22262 Removed deprecated methods from Filter class
f620285 is described below
commit f62028593f3facf5244195806c97f048c72845e7
Author: Jan Hentschel <ja...@ultratendency.com>
AuthorDate: Wed Apr 17 20:25:07 2019 +0200
HBASE-22262 Removed deprecated methods from Filter class
---
.../hadoop/hbase/filter/ColumnCountGetFilter.java | 6 ---
.../hbase/filter/ColumnPaginationFilter.java | 6 ---
.../hadoop/hbase/filter/ColumnPrefixFilter.java | 6 ---
.../hadoop/hbase/filter/ColumnRangeFilter.java | 6 ---
.../hadoop/hbase/filter/DependentColumnFilter.java | 10 -----
.../apache/hadoop/hbase/filter/FamilyFilter.java | 6 ---
.../org/apache/hadoop/hbase/filter/Filter.java | 50 +---------------------
.../org/apache/hadoop/hbase/filter/FilterBase.java | 18 +-------
.../org/apache/hadoop/hbase/filter/FilterList.java | 11 -----
.../apache/hadoop/hbase/filter/FilterListBase.java | 5 ---
.../hadoop/hbase/filter/FilterListWithAND.java | 15 -------
.../hadoop/hbase/filter/FilterListWithOR.java | 15 -------
.../hadoop/hbase/filter/FirstKeyOnlyFilter.java | 6 ---
.../FirstKeyValueMatchingQualifiersFilter.java | 6 ---
.../apache/hadoop/hbase/filter/FuzzyRowFilter.java | 6 ---
.../hadoop/hbase/filter/InclusiveStopFilter.java | 6 ---
.../apache/hadoop/hbase/filter/KeyOnlyFilter.java | 6 ---
.../hadoop/hbase/filter/MultiRowRangeFilter.java | 6 ---
.../hbase/filter/MultipleColumnPrefixFilter.java | 6 ---
.../org/apache/hadoop/hbase/filter/PageFilter.java | 6 ---
.../apache/hadoop/hbase/filter/PrefixFilter.java | 6 ---
.../hadoop/hbase/filter/QualifierFilter.java | 6 ---
.../hadoop/hbase/filter/RandomRowFilter.java | 6 ---
.../org/apache/hadoop/hbase/filter/RowFilter.java | 6 ---
.../hbase/filter/SingleColumnValueFilter.java | 6 ---
.../org/apache/hadoop/hbase/filter/SkipFilter.java | 6 ---
.../hadoop/hbase/filter/TimestampsFilter.java | 6 ---
.../apache/hadoop/hbase/filter/ValueFilter.java | 6 ---
.../hadoop/hbase/filter/WhileMatchFilter.java | 15 +------
.../apache/hadoop/hbase/rest/TestTableScan.java | 6 ++-
.../apache/hadoop/hbase/filter/FilterWrapper.java | 13 ------
.../hbase/client/TestConnectionImplementation.java | 12 ------
.../querymatcher/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 4ecf2d7..44a2b0f 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 class ColumnCountGetFilter extends FilterBase {
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 5d5321b..f11ddbd 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
@@ -112,12 +112,6 @@ public class ColumnPaginationFilter extends FilterBase {
}
@Override
- @Deprecated
- public ReturnCode filterKeyValue(final Cell c) {
- return filterCell(c);
- }
-
- @Override
public ReturnCode filterCell(final Cell c)
{
if (columnOffset != null) {
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 a016b8c..3ad0f17 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 class ColumnPrefixFilter extends FilterBase {
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 ea4d7e9..9937a66 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
@@ -123,12 +123,6 @@ public class ColumnRangeFilter extends FilterBase {
}
@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 857bfac..ee79ac3 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 class DependentColumnFilter extends CompareFilter {
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
@@ -174,10 +168,6 @@ public class DependentColumnFilter extends CompareFilter {
}
@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 74c6d29..f1406cd 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 class FamilyFilter extends CompareFilter {
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 dec8e06..8fba32c 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
@@ -67,24 +67,6 @@ 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 #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.
* If {@link #filterAllRemaining()} returns true, then {@link #filterRowKey(Cell)} should
* also return true.
*
@@ -112,34 +94,6 @@ public abstract class Filter {
* A way to filter based on the column family, column qualifier and/or the column value. Return
* code is described below. This allows filters to filter only certain number of columns, then
* terminate without matching ever column.
- *
- * If filterRowKey returns true, filterKeyValue needs to be consistent with it.
- *
- * filterKeyValue can assume that filterRowKey has already been called for the row.
- *
- * If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
- * <code>ReturnCode.NEXT_ROW</code> 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
- * terminate without matching ever column.
*
* If filterRowKey returns true, filterCell needs to be consistent with it.
*
@@ -157,8 +111,8 @@ public abstract class Filter {
* @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 7401e4c..9fb796a 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 1f14a45..32fa799 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
@@ -147,11 +147,6 @@ final public class FilterList extends FilterBase {
}
@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);
}
@@ -167,12 +162,6 @@ final public class FilterList extends FilterBase {
}
@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 ad24d0e..a9defbf 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 abstract class FilterListBase extends FilterBase {
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 76cda02..04aad2c 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
@@ -200,21 +200,6 @@ public class FilterListWithAND extends FilterListBase {
}
@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()) {
return super.filterRowKey(firstRowCell);
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 28540a4..725260e 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
@@ -308,21 +308,6 @@ public class FilterListWithOR extends FilterListBase {
}
@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()) {
return super.filterRowKey(firstRowCell);
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 a258ec1..dc4207d 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 class FirstKeyOnlyFilter extends FilterBase {
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 4cde2bf..c59aa6e 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 class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
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 abfa7e1..f6811f6 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 @@ public class FuzzyRowFilter extends FilterBase {
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 cfaf18d..7c09668 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 class InclusiveStopFilter extends FilterBase {
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 9603304..26e53cf 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 @@ public class KeyOnlyFilter extends FilterBase {
}
}
- @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 32e22d8..1e25d6d 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 class MultiRowRangeFilter extends FilterBase {
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 5fd6445..357f19c 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 class MultipleColumnPrefixFilter extends FilterBase {
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 b1228d9..98831c6 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 class PageFilter extends FilterBase {
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 15c18fc..8300005 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 class PrefixFilter extends FilterBase {
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 7b6167f..497f633 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 class QualifierFilter extends CompareFilter {
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 2d81878..e57cff5 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 class RandomRowFilter extends FilterBase {
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 fcda1e0..19c9348 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 class RowFilter extends CompareFilter {
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 92422c9..5563ae7 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 class SingleColumnValueFilter extends FilterBase {
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 fe5caad..03e1f2c 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 class SkipFilter extends FilterBase {
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 c2cdf99..5e7fb5c 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 class TimestampsFilter extends FilterBase {
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 38fe451..5b5ec6f 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 class ValueFilter extends CompareFilter {
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 e053886..5bfdaa3 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.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferExce
/**
* 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
@@ -65,13 +65,6 @@ public class WhileMatchFilter extends FilterBase {
}
@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;
boolean value = filter.filterRowKey(cell);
@@ -79,12 +72,6 @@ public class WhileMatchFilter extends FilterBase {
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 e048550..16eeda0 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.bind.annotation.XmlRootElement;
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 class TestTableScan {
}
@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 9bc072a..74afc69 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
@@ -101,24 +101,11 @@ final public class FilterWrapper extends Filter {
}
@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 b1ad866..1a4be8a 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 class TestConnectionImplementation {
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 861b83e..7be1db3 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 class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher {
}
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 @@ public class TestUserScanQueryMatcher extends AbstractTestScanQueryMatcher {
private static class AlwaysIncludeFilter extends FilterBase {
@Override
- public ReturnCode filterKeyValue(final Cell c) throws IOException {
+ public ReturnCode filterCell(final Cell c) {
return ReturnCode.INCLUDE;
}
}