Skip to content

Commit 8410eaa

Browse files
committed
HBASE-22262 Removed deprecated methods from Filter class
1 parent 184568c commit 8410eaa

33 files changed

+10
-291
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -63,12 +63,6 @@ public boolean filterAllRemaining() {
6363
return this.count > this.limit;
6464
}
6565

66-
@Deprecated
67-
@Override
68-
public ReturnCode filterKeyValue(final Cell c) {
69-
return filterCell(c);
70-
}
71-
7266
@Override
7367
public ReturnCode filterCell(final Cell c) {
7468
this.count++;

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -111,12 +111,6 @@ public boolean filterRowKey(Cell cell) throws IOException {
111111
return false;
112112
}
113113

114-
@Override
115-
@Deprecated
116-
public ReturnCode filterKeyValue(final Cell c) {
117-
return filterCell(c);
118-
}
119-
120114
@Override
121115
public ReturnCode filterCell(final Cell c)
122116
{

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -58,12 +58,6 @@ public boolean filterRowKey(Cell cell) throws IOException {
5858
return false;
5959
}
6060

61-
@Deprecated
62-
@Override
63-
public ReturnCode filterKeyValue(final Cell c) {
64-
return filterCell(c);
65-
}
66-
6761
@Override
6862
public ReturnCode filterCell(final Cell cell) {
6963
if (this.prefix == null) {

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -122,12 +122,6 @@ public boolean filterRowKey(Cell cell) throws IOException {
122122
return false;
123123
}
124124

125-
@Override
126-
@Deprecated
127-
public ReturnCode filterKeyValue(final Cell c) {
128-
return filterCell(c);
129-
}
130-
131125
@Override
132126
public ReturnCode filterCell(final Cell c) {
133127
int cmpMin = 1;

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -133,12 +133,6 @@ public boolean filterAllRemaining() {
133133
return false;
134134
}
135135

136-
@Deprecated
137-
@Override
138-
public ReturnCode filterKeyValue(final Cell c) {
139-
return filterCell(c);
140-
}
141-
142136
@Override
143137
public ReturnCode filterCell(final Cell c) {
144138
// Check if the column and qualifier match
@@ -173,10 +167,6 @@ public boolean filterRow() {
173167
return false;
174168
}
175169

176-
@Override
177-
public boolean filterRowKey(byte[] buffer, int offset, int length) {
178-
return false;
179-
}
180170
@Override
181171
public void reset() {
182172
stampSet.clear();

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -56,12 +56,6 @@ public FamilyFilter(final CompareOperator op,
5656
super(op, familyComparator);
5757
}
5858

59-
@Deprecated
60-
@Override
61-
public ReturnCode filterKeyValue(final Cell c) {
62-
return filterCell(c);
63-
}
64-
6559
@Override
6660
public ReturnCode filterCell(final Cell c) {
6761
int familyLength = c.getFamilyLength();

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java

Lines changed: 2 additions & 48 deletions
Original file line numberDiff line numberDiff line change
@@ -64,24 +64,6 @@ public abstract class Filter {
6464
*/
6565
abstract public void reset() throws IOException;
6666

67-
/**
68-
* Filters a row based on the row key. If this returns true, the entire row will be excluded. If
69-
* false, each KeyValue in the row will be passed to {@link #filterCell(Cell)} below.
70-
*
71-
* Concrete implementers can signal a failure condition in their code by throwing an
72-
* {@link IOException}.
73-
*
74-
* @param buffer buffer containing row key
75-
* @param offset offset into buffer where row key starts
76-
* @param length length of the row key
77-
* @return true, remove entire row, false, include the row (maybe).
78-
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
79-
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
80-
* Instead use {@link #filterRowKey(Cell)}
81-
*/
82-
@Deprecated
83-
abstract public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException;
84-
8567
/**
8668
* Filters a row based on the row key. If this returns true, the entire row will be excluded. If
8769
* false, each KeyValue in the row will be passed to {@link #filterCell(Cell)} below.
@@ -108,34 +90,6 @@ public abstract class Filter {
10890
*/
10991
abstract public boolean filterAllRemaining() throws IOException;
11092

111-
/**
112-
* A way to filter based on the column family, column qualifier and/or the column value. Return
113-
* code is described below. This allows filters to filter only certain number of columns, then
114-
* terminate without matching ever column.
115-
*
116-
* If filterRowKey returns true, filterKeyValue needs to be consistent with it.
117-
*
118-
* filterKeyValue can assume that filterRowKey has already been called for the row.
119-
*
120-
* If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
121-
* <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called just in case the caller calls
122-
* for the next row.
123-
*
124-
* Concrete implementers can signal a failure condition in their code by throwing an
125-
* {@link IOException}.
126-
*
127-
* @param c the Cell in question
128-
* @return code as described below, Filter.ReturnCode.INCLUDE by default
129-
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
130-
* @see Filter.ReturnCode
131-
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
132-
* Instead use filterCell(Cell)
133-
*/
134-
@Deprecated
135-
public ReturnCode filterKeyValue(final Cell c) throws IOException {
136-
return Filter.ReturnCode.INCLUDE;
137-
}
138-
13993
/**
14094
* A way to filter based on the column family, column qualifier and/or the column value. Return
14195
* 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 {
157111
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
158112
* @see Filter.ReturnCode
159113
*/
160-
public ReturnCode filterCell(final Cell c) throws IOException{
161-
return filterKeyValue(c);
114+
public ReturnCode filterCell(final Cell c) throws IOException {
115+
return ReturnCode.INCLUDE;
162116
}
163117

164118
/**

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java

Lines changed: 1 addition & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -47,25 +47,9 @@ public abstract class FilterBase extends Filter {
4747
public void reset() throws IOException {
4848
}
4949

50-
/**
51-
* Filters that do not filter by row key can inherit this implementation that
52-
* never filters anything. (ie: returns false).
53-
*
54-
* {@inheritDoc}
55-
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
56-
* Instead use {@link #filterRowKey(Cell)}
57-
*/
58-
@Override
59-
@Deprecated
60-
public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
61-
if (filterAllRemaining()) return true;
62-
return false;
63-
}
64-
6550
@Override
6651
public boolean filterRowKey(Cell cell) throws IOException {
67-
if (filterAllRemaining()) return true;
68-
return filterRowKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
52+
return filterAllRemaining();
6953
}
7054

7155
/**

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java

Lines changed: 0 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -146,11 +146,6 @@ public void reset() throws IOException {
146146
filterListBase.reset();
147147
}
148148

149-
@Override
150-
public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
151-
return filterListBase.filterRowKey(rowKey, offset, length);
152-
}
153-
154149
@Override
155150
public boolean filterRowKey(Cell firstRowCell) throws IOException {
156151
return filterListBase.filterRowKey(firstRowCell);
@@ -166,12 +161,6 @@ public Cell transformCell(Cell c) throws IOException {
166161
return filterListBase.transformCell(c);
167162
}
168163

169-
@Override
170-
@Deprecated
171-
public ReturnCode filterKeyValue(final Cell c) throws IOException {
172-
return filterCell(c);
173-
}
174-
175164
@Override
176165
public ReturnCode filterCell(final Cell c) throws IOException {
177166
return filterListBase.filterCell(c);

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -112,11 +112,6 @@ public Cell transformCell(Cell c) throws IOException {
112112
return transformed;
113113
}
114114

115-
@Override
116-
public ReturnCode filterKeyValue(final Cell c) throws IOException {
117-
return filterCell(c);
118-
}
119-
120115
/**
121116
* Filters that never filter by modifying the returned List of Cells can inherit this
122117
* implementation that does nothing. {@inheritDoc}

0 commit comments

Comments
 (0)