HBASE-13641 Deperecate Filter#filterRowKey(byte[] buffer, int offset, int length) in favor of filterRowKey(Cell firstRowCell).
This commit is contained in:
parent
d69807663c
commit
6c3fd34475
|
@ -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;
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
* <ul>
|
||||
* <li> {@link #reset()} : reset the filter state before filtering a new row. </li>
|
||||
* <li> {@link #filterAllRemaining()}: true means row scan is over; false means keep going. </li>
|
||||
* <li> {@link #filterRowKey(byte[],int,int)}: true means drop this row; false means include.</li>
|
||||
* <li> {@link #filterRowKey(Cell)}: true means drop this row; false means include.</li>
|
||||
* <li> {@link #filterKeyValue(Cell)}: decides whether to include or exclude this Cell.
|
||||
* See {@link ReturnCode}. </li>
|
||||
* <li> {@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.
|
||||
*
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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<Tag> putVisTags = new ArrayList<Tag>();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue