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 572de9f5c7a..fd651307bfc 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
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.Cell;
@@ -51,6 +52,12 @@ public class ColumnCountGetFilter extends FilterBase {
return limit;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public boolean filterAllRemaining() {
return this.count > this.limit;
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 673ca6e37c1..2b5f128b5b4 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
@@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.Cell;
@@ -40,8 +41,8 @@ import com.google.protobuf.InvalidProtocolBufferException;
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
-public class ColumnPaginationFilter extends FilterBase
-{
+public class ColumnPaginationFilter extends FilterBase {
+
private int limit = 0;
private int offset = -1;
private byte[] columnOffset = null;
@@ -104,6 +105,12 @@ public class ColumnPaginationFilter extends FilterBase
return columnOffset;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell v)
{
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 d2f058a59cd..a89a1f391ca 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
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.Cell;
@@ -51,6 +52,12 @@ public class ColumnPrefixFilter extends FilterBase {
return prefix;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell kv) {
if (this.prefix == null || kv.getQualifierArray() == 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 d8ea0949f68..c386ad687f9 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter;
import static org.apache.hadoop.hbase.util.Bytes.len;
+import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.Cell;
@@ -115,6 +116,12 @@ public class ColumnRangeFilter extends FilterBase {
return this.maxColumnInclusive;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell kv) {
// TODO have a column compare method in Cell
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
index 319e1238b55..f6415244c59 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
@@ -19,8 +19,10 @@
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -97,6 +99,12 @@ public abstract class CompareFilter extends FilterBase {
return comparator;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
protected boolean doCompare(final CompareOp compareOp,
final ByteArrayComparable comparator, final byte [] data,
final int offset, final int length) {
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 88bf842a6f8..a2f90152f7e 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(byte[],int,int)}: true means drop this row; false means include.
+ * - {@link #filterRowKey(Cell)}: true means drop this row; false means include.
* - {@link #filterKeyValue(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
@@ -78,9 +78,25 @@ public abstract class Filter {
* @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 #filterKeyValue(Cell)} below.
+ *
+ * Concrete implementers can signal a failure condition in their code by throwing an
+ * {@link IOException}.
+ *
+ * @param firstRowCell The first cell coming in the new row
+ * @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.
+ */
+ abstract public boolean filterRowKey(Cell firstRowCell) throws IOException;
+
/**
* If this returns true, the scan will terminate.
*
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 a04dd897f42..1bcd00aa761 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
@@ -52,12 +52,22 @@ public abstract class FilterBase extends Filter {
* 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 {
return false;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // TODO when cell is backed by DirectByteBuffer, we would need to copy row bytes to temp byte[]
+ // and call old method for BC.
+ return filterRowKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
+ }
+
/**
* Filters that never filter all remaining can inherit this implementation that
* never stops the filter early.
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 a7856cdda78..2f89251d897 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
@@ -197,6 +197,25 @@ final public class FilterList extends Filter {
return flag;
}
+ @Override
+ public boolean filterRowKey(Cell firstRowCell) throws IOException {
+ boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
+ int listize = filters.size();
+ for (int i = 0; i < listize; i++) {
+ Filter filter = filters.get(i);
+ if (this.operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining() || filter.filterRowKey(firstRowCell)) {
+ flag = true;
+ }
+ } else if (this.operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining() && !filter.filterRowKey(firstRowCell)) {
+ flag = false;
+ }
+ }
+ }
+ return flag;
+ }
+
@Override
public boolean filterAllRemaining() throws IOException {
int listize = filters.size();
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
index 517611525a7..ff0e43b7410 100644
--- 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
@@ -102,9 +102,15 @@ final public class FilterWrapper extends Filter {
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
+ // No call to this.
return this.filter.filterRowKey(buffer, offset, length);
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ return this.filter.filterRowKey(cell);
+ }
+
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
return this.filter.filterKeyValue(v);
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 77ed7d99697..80a1deb5935 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
@@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.Cell;
@@ -46,6 +47,12 @@ public class FirstKeyOnlyFilter extends FilterBase {
foundKV = false;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell v) {
if(foundKV) return ReturnCode.NEXT_ROW;
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 bb5edf42cf5..9b234dea0d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -83,6 +84,12 @@ public class FuzzyRowFilter extends FilterBase {
this.fuzzyKeysData = fuzzyKeysData;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
// TODO: possible improvement: save which fuzzy row key to use when providing a hint
@Override
public ReturnCode filterKeyValue(Cell c) {
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 cf2d1533ce8..b487e9868ba 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
@@ -58,17 +58,10 @@ public class InclusiveStopFilter extends FilterBase {
return ReturnCode.INCLUDE;
}
- public boolean filterRowKey(byte[] buffer, int offset, int length) {
- if (buffer == null) {
- //noinspection RedundantIfStatement
- if (this.stopRowKey == null) {
- return true; //filter...
- }
- return false;
- }
+ public boolean filterRowKey(Cell firstRowCell) {
// if stopRowKey is <= buffer, then true, filter row.
int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
- buffer, offset, length);
+ firstRowCell.getRowArray(), firstRowCell.getRowOffset(), firstRowCell.getRowLength());
if(cmp < 0) {
done = true;
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 2a2b525efe5..2fd5aba1d30 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
@@ -49,6 +49,12 @@ public class KeyOnlyFilter extends FilterBase {
public KeyOnlyFilter() { this(false); }
public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public Cell transformCell(Cell cell) {
return createKeyOnlyCell(cell);
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 24c81356fad..e2f159b76c7 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
@@ -83,14 +83,17 @@ public class MultiRowRangeFilter extends FilterBase {
}
@Override
- public boolean filterRowKey(byte[] buffer, int offset, int length) {
+ public boolean filterRowKey(Cell firstRowCell) {
// If it is the first time of running, calculate the current range index for
// the row key. If index is out of bound which happens when the start row
// user sets is after the largest stop row of the ranges, stop the scan.
// If row key is after the current range, find the next range and update index.
- if (!initialized || !range.contains(buffer, offset, length)) {
+ int length = firstRowCell.getRowLength();
+ int offset = firstRowCell.getRowOffset();
+ if (!initialized
+ || !range.contains(firstRowCell.getRowArray(), offset, length)) {
byte[] rowkey = new byte[length];
- System.arraycopy(buffer, offset, rowkey, 0, length);
+ System.arraycopy(firstRowCell.getRowArray(), firstRowCell.getRowOffset(), rowkey, 0, length);
index = getNextRangeIndex(rowkey);
if (index >= rangeList.size()) {
done = true;
@@ -115,7 +118,7 @@ public class MultiRowRangeFilter extends FilterBase {
}
initialized = true;
} else {
- if (range.contains(buffer, offset, length)) {
+ if (range.contains(firstRowCell.getRowArray(), offset, length)) {
currentReturnCode = ReturnCode.INCLUDE;
} else currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT;
}
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 b7ec11a20e1..79191a30792 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
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
@@ -63,6 +64,12 @@ public class MultipleColumnPrefixFilter extends FilterBase {
return temp;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell kv) {
if (sortedPrefixes.size() == 0 || kv.getQualifierArray() == null) {
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 0dbd97b3b06..7c68dd23704 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
@@ -60,6 +60,12 @@ public class PageFilter extends FilterBase {
return pageSize;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(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 5b56748c068..7a031df7228 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
@@ -50,16 +50,16 @@ public class PrefixFilter extends FilterBase {
return prefix;
}
- public boolean filterRowKey(byte[] buffer, int offset, int length) {
- if (buffer == null || this.prefix == null)
- return true;
- if (length < prefix.length)
+ public boolean filterRowKey(Cell firstRowCell) {
+ if (firstRowCell == null || this.prefix == null)
return true;
+ int length = firstRowCell.getRowLength();
+ if (length < prefix.length) return true;
// if they are equal, return false => pass row
// else return true, filter row
// if we are passed the prefix, set flag
- int cmp = Bytes.compareTo(buffer, offset, this.prefix.length, this.prefix, 0,
- this.prefix.length);
+ int cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),
+ this.prefix.length, this.prefix, 0, this.prefix.length);
if ((!isReversed() && cmp > 0) || (isReversed() && cmp < 0)) {
passedPrefix = true;
}
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 2a25b32b58f..decdc78bafe 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
@@ -89,7 +89,7 @@ public class RandomRowFilter extends FilterBase {
}
@Override
- public boolean filterRowKey(byte[] buffer, int offset, int length) {
+ public boolean filterRowKey(Cell firstRowCell) {
if (chance < 0) {
// with a zero chance, the rows is always excluded
filterOutRow = true;
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 cb4337e28df..e508b8b3097 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
@@ -74,8 +74,9 @@ public class RowFilter extends CompareFilter {
}
@Override
- public boolean filterRowKey(byte[] data, int offset, int length) {
- if(doCompare(this.compareOp, this.comparator, data, offset, length)) {
+ public boolean filterRowKey(Cell firstRowCell) {
+ if (doCompare(this.compareOp, this.comparator, firstRowCell.getRowArray(),
+ firstRowCell.getRowOffset(), firstRowCell.getRowLength())) {
this.filterOutRow = true;
}
return 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 2fcbf4f57bd..09fc90e384d 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
@@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.filter;
import java.io.IOException;
import java.util.ArrayList;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -71,7 +69,6 @@ import com.google.protobuf.InvalidProtocolBufferException;
@InterfaceAudience.Public
@InterfaceStability.Stable
public class SingleColumnValueFilter extends FilterBase {
- private static final Log LOG = LogFactory.getLog(SingleColumnValueFilter.class);
protected byte [] columnFamily;
protected byte [] columnQualifier;
@@ -168,6 +165,12 @@ public class SingleColumnValueFilter extends FilterBase {
return columnQualifier;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(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 ce8e5112b39..71ea3c3bf3c 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
@@ -73,6 +73,12 @@ public class SkipFilter extends FilterBase {
filterRow = filterRow || value;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
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 32a3d731177..0ce04470f9f 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
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.TreeSet;
@@ -88,6 +89,12 @@ public class TimestampsFilter extends FilterBase {
return minTimeStamp;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell v) {
if (this.timestamps.contains(v.getTimestamp())) {
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 31d4f77e323..e75ca49bdca 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
@@ -72,6 +72,13 @@ public class WhileMatchFilter extends FilterBase {
return value;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ boolean value = filter.filterRowKey(cell);
+ changeFAR(value);
+ return value;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index 9c5b5afd9ac..e2f4ce0f283 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
@@ -108,7 +107,9 @@ public class Import extends Configured implements Tool {
LOG.trace("Considering the row."
+ Bytes.toString(row.get(), row.getOffset(), row.getLength()));
}
- if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
+ if (filter == null
+ || !filter.filterRowKey(KeyValueUtil.createFirstOnRow(row.get(), row.getOffset(),
+ (short) row.getLength()))) {
for (Cell kv : value.rawCells()) {
kv = filterKv(filter, kv);
// skip if we filtered it out
@@ -163,7 +164,9 @@ public class Import extends Configured implements Tool {
LOG.trace("Considering the row."
+ Bytes.toString(key.get(), key.getOffset(), key.getLength()));
}
- if (filter == null || !filter.filterRowKey(key.get(), key.getOffset(), key.getLength())) {
+ if (filter == null
+ || !filter.filterRowKey(KeyValueUtil.createFirstOnRow(key.get(), key.getOffset(),
+ (short) key.getLength()))) {
processKV(key, result, context, put, delete);
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 3bfb1a6bc2d..576a23e8e34 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -5555,7 +5555,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Check if rowkey filter wants to exclude this row. If so, loop to next.
// Technically, if we hit limits before on this row, we don't need this call.
- if (filterRowKey(currentRow, offset, length)) {
+ if (filterRowKey(current)) {
boolean moreRows = nextRow(current);
if (!moreRows) {
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
@@ -5707,9 +5707,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
&& filter.filterRow();
}
- private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
- return filter != null
- && filter.filterRowKey(row, offset, length);
+ private boolean filterRowKey(Cell current) throws IOException {
+ return filter != null && filter.filterRowKey(current);
}
protected boolean nextRow(Cell curRowCell) throws IOException {
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 48a982ee41f..293e60e9594 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
@@ -86,6 +86,12 @@ class AccessControlFilter extends FilterBase {
this.prevQual = new SimpleMutableByteRange();
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell cell) {
if (isSystemTable) {
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 e30344cf81c..871678fdf11 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
@@ -1071,6 +1071,12 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
this.deleteCellVisTagsFormat = deleteCellVisTagsFormat;
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell cell) throws IOException {
List putVisTags = new ArrayList();
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 eb8abbebffc..18bd9f05124 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
@@ -49,6 +49,12 @@ class VisibilityLabelFilter extends FilterBase {
this.curQualifier = new SimpleMutableByteRange();
}
+ @Override
+ public boolean filterRowKey(Cell cell) throws IOException {
+ // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+ return false;
+ }
+
@Override
public ReturnCode filterKeyValue(Cell cell) throws IOException {
if (curFamily.getBytes() == null
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 02c4c9d95c8..3601b0115d2 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
@@ -628,7 +628,7 @@ public class TestFilter {
/**
* Tests the the {@link WhileMatchFilter} works in combination with a
* {@link Filter} that uses the
- * {@link Filter#filterRowKey(byte[], int, int)} method.
+ * {@link Filter#filterRowKey(Cell)} method.
*
* See HBASE-2258.
*
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index 0e01484006d..8854efeb2b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -115,7 +115,7 @@ public class TestFilterList {
/* Will pass both */
byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
for (int i = 0; i < MAX_PAGES - 1; i++) {
- assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -124,7 +124,7 @@ public class TestFilterList {
/* Only pass PageFilter */
rowkey = Bytes.toBytes("z");
- assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
Bytes.toBytes(0));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -132,7 +132,7 @@ public class TestFilterList {
/* reach MAX_PAGES already, should filter any rows */
rowkey = Bytes.toBytes("yyy");
- assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
Bytes.toBytes(0));
assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -140,7 +140,7 @@ public class TestFilterList {
/* We should filter any row */
rowkey = Bytes.toBytes("z");
- assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
assertTrue(filterMPONE.filterAllRemaining());
}
@@ -179,14 +179,14 @@ public class TestFilterList {
assertFalse(filterMPALL.filterAllRemaining());
byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
for (int i = 0; i < MAX_PAGES - 1; i++) {
- assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+ assertFalse(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
}
filterMPALL.reset();
rowkey = Bytes.toBytes("z");
- assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+ assertTrue(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
// Should fail here; row should be filtered out.
KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
@@ -229,7 +229,7 @@ public class TestFilterList {
/* We should be able to fill MAX_PAGES without incrementing page counter */
byte [] rowkey = Bytes.toBytes("yyyyyyyy");
for (int i = 0; i < MAX_PAGES; i++) {
- assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -239,7 +239,7 @@ public class TestFilterList {
/* Now let's fill the page filter */
rowkey = Bytes.toBytes("xxxxxxx");
for (int i = 0; i < MAX_PAGES; i++) {
- assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -249,7 +249,7 @@ public class TestFilterList {
/* We should still be able to include even though page filter is at max */
rowkey = Bytes.toBytes("yyy");
for (int i = 0; i < MAX_PAGES; i++) {
- assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
Bytes.toBytes(i));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -269,23 +269,23 @@ public class TestFilterList {
FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
flist.addFilter(new PrefixFilter(r1));
- flist.filterRowKey(r1, 0, r1.length);
+ flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
flist.reset();
- flist.filterRowKey(r2, 0, r2.length);
+ flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
flist.addFilter(new AlwaysNextColFilter());
flist.addFilter(new PrefixFilter(r1));
- flist.filterRowKey(r1, 0, r1.length);
+ flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
flist.reset();
- flist.filterRowKey(r2, 0, r2.length);
+ flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
}
@@ -302,12 +302,12 @@ public class TestFilterList {
FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
flist.addFilter(new AlwaysNextColFilter());
flist.addFilter(new InclusiveStopFilter(r1));
- flist.filterRowKey(r1, 0, r1.length);
+ flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
flist.reset();
- flist.filterRowKey(r2, 0, r2.length);
+ flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
index e527ca87780..fe4e5d16a48 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
@@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.filter;
+import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -71,16 +72,14 @@ public class TestInclusiveStopFilter {
private void stopRowTests(Filter filter) throws Exception {
assertFalse("Filtering on " + Bytes.toString(GOOD_ROW),
- filter.filterRowKey(GOOD_ROW, 0, GOOD_ROW.length));
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(GOOD_ROW)));
assertFalse("Filtering on " + Bytes.toString(STOP_ROW),
- filter.filterRowKey(STOP_ROW, 0, STOP_ROW.length));
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(STOP_ROW)));
assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
- filter.filterRowKey(PAST_STOP_ROW, 0, PAST_STOP_ROW.length));
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(PAST_STOP_ROW)));
assertTrue("FilterAllRemaining", filter.filterAllRemaining());
assertFalse("FilterNotNull", filter.filterRow());
-
- assertFalse("Filter a null", filter.filterRowKey(null, 0, 0));
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
index 0c596aae4ce..9f623354dbc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@@ -75,15 +76,15 @@ public class TestMultiRowRangeFilter {
new MultiRowRangeFilter.RowRange(Bytes.toBytes("b"), true, Bytes.toBytes("c"), true),
new MultiRowRangeFilter.RowRange(Bytes.toBytes("d"), true, Bytes.toBytes("e"), true)
));
- filter.filterRowKey(Bytes.toBytes("a"), 0, 1);
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("a")));
assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null));
- filter.filterRowKey(Bytes.toBytes("b"), 0, 1);
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("b")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
- filter.filterRowKey(Bytes.toBytes("c"), 0, 1);
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("c")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
- filter.filterRowKey(Bytes.toBytes("d"), 0, 1);
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("d")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
- filter.filterRowKey(Bytes.toBytes("e"), 0, 1);
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("e")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
index 02a55baa9d2..754bdb60445 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.filter;
+import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -72,13 +73,13 @@ public class TestPrefixFilter {
for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
byte [] t = createRow(c);
assertFalse("Failed with character " + c,
- filter.filterRowKey(t, 0, t.length));
+ filter.filterRowKey(KeyValueUtil.createFirstOnRow(t)));
assertFalse(filter.filterAllRemaining());
}
String yahooSite = "com.yahoo.www";
byte [] yahooSiteBytes = Bytes.toBytes(yahooSite);
assertTrue("Failed with character " +
- yahooSite, filter.filterRowKey(yahooSiteBytes, 0, yahooSiteBytes.length));
+ yahooSite, filter.filterRowKey(KeyValueUtil.createFirstOnRow(yahooSiteBytes)));
assertEquals(filter.filterAllRemaining(), lastFilterAllRemaining);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
index 8effca5e709..ca4fa0682f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.filter;
+import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -47,8 +48,7 @@ public class TestRandomRowFilter {
int included = 0;
int max = 1000000;
for (int i = 0; i < max; i++) {
- if (!quarterChanceFilter.filterRowKey(Bytes.toBytes("row"), 0, Bytes
- .toBytes("row").length)) {
+ if (!quarterChanceFilter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("row")))) {
included++;
}
}