HBASE-13346: Clean up Filter package for post 1.0 s/KeyValue/Cell/g

Added filterCell method to Filter, it calls filterKeyValue by default
Deprecated filterKeyValue in Filter, bud added default functionality to return Filter.ReturnCode.INCLUDE.
Added filterKeyValue (calling filterCell) to Filters extending FilterBase to be backward compatible.
renamed filterKeyValue to filterCell in all implementations
changed all internal calls to use filterCell instead of filterKeyValue
changed tests too

This way the change is simple and backward compatible.
Any existing custom filter should work since they override filterKeyValue
and the implementation is called by Filter.filterCell.

Moved FilterWrapper to hbase-server

Signed-off-by: anoopsamjohn <anoopsamjohn@gmail.com>
This commit is contained in:
Tamas Penzes 2017-10-25 22:20:16 +02:00 committed by anoopsamjohn
parent e1152afdd4
commit fc581270a8
59 changed files with 527 additions and 343 deletions

View File

@ -55,7 +55,7 @@ public abstract class Query extends OperationWithAttributes {
/**
* Apply the specified server-side filter when performing the Query. Only
* {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)} is called AFTER all tests for ttl,
* {@link Filter#filterCell(org.apache.hadoop.hbase.Cell)} is called AFTER all tests for ttl,
* column match, deletes and column family's max versions have been run.
* @param filter filter to run on the server
* @return this for invocation chaining

View File

@ -62,8 +62,14 @@ public class ColumnCountGetFilter extends FilterBase {
return this.count > this.limit;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
this.count++;
return filterAllRemaining() ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL;
}
@ -108,7 +114,7 @@ public class ColumnCountGetFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -110,7 +110,13 @@ public class ColumnPaginationFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(Cell v)
@Deprecated
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c)
{
if (columnOffset != null) {
if (count >= limit) {
@ -119,7 +125,7 @@ public class ColumnPaginationFilter extends FilterBase {
int cmp = 0;
// Only compare if no KV's have been seen so far.
if (count == 0) {
cmp = CellUtil.compareQualifiers(v, this.columnOffset, 0, this.columnOffset.length);
cmp = CellUtil.compareQualifiers(c, this.columnOffset, 0, this.columnOffset.length);
}
if (cmp < 0) {
return ReturnCode.SEEK_NEXT_USING_HINT;
@ -196,7 +202,7 @@ public class ColumnPaginationFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -58,8 +58,14 @@ public class ColumnPrefixFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell cell) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell cell) {
if (this.prefix == null) {
return ReturnCode.INCLUDE;
} else {
@ -132,7 +138,7 @@ public class ColumnPrefixFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -121,11 +121,17 @@ public class ColumnRangeFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(Cell kv) {
@Deprecated
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
int cmpMin = 1;
if (this.minColumn != null) {
cmpMin = CellUtil.compareQualifiers(kv, this.minColumn, 0, this.minColumn.length);
cmpMin = CellUtil.compareQualifiers(c, this.minColumn, 0, this.minColumn.length);
}
if (cmpMin < 0) {
@ -140,7 +146,7 @@ public class ColumnRangeFilter extends FilterBase {
return ReturnCode.INCLUDE;
}
int cmpMax = CellUtil.compareQualifiers(kv, this.maxColumn, 0, this.maxColumn.length);
int cmpMax = CellUtil.compareQualifiers(c, this.maxColumn, 0, this.maxColumn.length);
if (this.maxColumnInclusive && cmpMax <= 0 ||
!this.maxColumnInclusive && cmpMax < 0) {

View File

@ -155,8 +155,14 @@ public class DependentColumnFilter extends CompareFilter {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell c) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
// Check if the column and qualifier match
if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) {
// include non-matches for the time being, they'll be discarded afterwards

View File

@ -72,11 +72,17 @@ public class FamilyFilter extends CompareFilter {
super(op, familyComparator);
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
int familyLength = v.getFamilyLength();
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
int familyLength = c.getFamilyLength();
if (familyLength > 0) {
if (compareFamily(getCompareOperator(), this.comparator, v)) {
if (compareFamily(getCompareOperator(), this.comparator, c)) {
return ReturnCode.NEXT_ROW;
}
}

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
* <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(Cell)}: true means drop this row; false means include.</li>
* <li> {@link #filterKeyValue(Cell)}: decides whether to include or exclude this Cell.
* <li> {@link #filterCell(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
* Cell. </li>
@ -66,7 +66,7 @@ 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 #filterKeyValue(Cell)} below.
* 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}.
@ -84,7 +84,7 @@ 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 #filterKeyValue(Cell)} below.
* 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.
*
@ -124,12 +124,42 @@ public abstract class Filter {
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param v the Cell in question
* @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.
*
* filterCell 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
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
* @see Filter.ReturnCode
*/
abstract public ReturnCode filterKeyValue(final Cell v) throws IOException;
public ReturnCode filterCell(final Cell c) throws IOException{
return filterKeyValue(c);
}
/**
* Give the filter a chance to transform the passed KeyValue. If the Cell is changed a new
@ -173,7 +203,7 @@ public abstract class Filter {
NEXT_COL,
/**
* Seek to next row in current family. It may still pass a cell whose family is different but
* row is the same as previous cell to {@link #filterKeyValue(Cell)} , even if we get a NEXT_ROW
* row is the same as previous cell to {@link #filterCell(Cell)} , even if we get a NEXT_ROW
* returned for previous cell. For more details see HBASE-18368. <br>
* Once reset() method was invoked, then we switch to the next row for all family, and you can
* catch the event by invoking CellUtils.matchingRows(previousCell, currentCell). <br>
@ -210,7 +240,7 @@ public abstract class Filter {
abstract public boolean hasFilterRow();
/**
* Last chance to veto row based on previous {@link #filterKeyValue(Cell)} calls. The filter
* Last chance to veto row based on previous {@link #filterCell(Cell)} calls. The filter
* needs to retain state then return a particular value for this call if they wish to exclude a
* row if a certain column is missing (for example).
*

View File

@ -112,7 +112,7 @@ public abstract class FilterBase extends Filter {
/**
* Filters that never filter by rows based on previously gathered state from
* {@link #filterKeyValue(Cell)} can inherit this implementation that
* {@link #filterCell(Cell)} can inherit this implementation that
* never filters a row.
*
* {@inheritDoc}

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
* {@link Operator#MUST_PASS_ONE} (<code>OR</code>). Since you can use Filter Lists as children of
* Filter Lists, you can create a hierarchy of filters to be evaluated. <br>
* {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does not
* include the KeyValue. <br>
* include the Cell. <br>
* {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated. <br>
* Defaults to {@link Operator#MUST_PASS_ALL}.
*/
@ -166,8 +166,8 @@ final public class FilterList extends FilterBase {
}
/**
* Internal implementation of {@link #filterKeyValue(Cell)}. Compared to the
* {@link #filterKeyValue(Cell)} method, this method accepts an additional parameter named
* Internal implementation of {@link #filterCell(Cell)}. Compared to the
* {@link #filterCell(Cell)} method, this method accepts an additional parameter named
* transformedCell. This parameter indicates the initial value of transformed cell before this
* filter operation. <br/>
* For FilterList, we can consider a filter list as a node in a tree. sub-filters of the filter
@ -180,13 +180,19 @@ final public class FilterList extends FilterBase {
* @return ReturnCode of this filter operation.
* @throws IOException
*/
ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException {
return this.filterListBase.internalFilterKeyValue(c, transformedCell);
ReturnCode internalFilterCell(Cell c, Cell transformedCell) throws IOException {
return this.filterListBase.internalFilterCell(c, transformedCell);
}
@Override
public ReturnCode filterKeyValue(Cell c) throws IOException {
return filterListBase.filterKeyValue(c);
@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);
}
/**

View File

@ -43,7 +43,7 @@ public abstract class FilterListBase extends FilterBase {
protected Cell referenceCell = null;
/**
* When filtering a given Cell in {@link #filterKeyValue(Cell)}, this stores the transformed Cell
* When filtering a given Cell in {@link #filterCell(Cell)}, this stores the transformed Cell
* to be returned by {@link #transformCell(Cell)}. Individual filters transformation are applied
* only when the filter includes the Cell. Transformations are composed in the order specified by
* {@link #filters}.
@ -108,18 +108,23 @@ public abstract class FilterListBase extends FilterBase {
}
/**
* Internal implementation of {@link #filterKeyValue(Cell)}
* Internal implementation of {@link #filterCell(Cell)}
* @param c The cell in question.
* @param transformedCell The transformed cell of previous filter(s)
* @return ReturnCode of this filter operation.
* @throws IOException
* @see org.apache.hadoop.hbase.filter.FilterList#internalFilterKeyValue(Cell, Cell)
* @see org.apache.hadoop.hbase.filter.FilterList#internalFilterCell(Cell, Cell)
*/
abstract ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException;
abstract ReturnCode internalFilterCell(Cell c, Cell transformedCell) throws IOException;
@Override
public ReturnCode filterKeyValue(Cell c) throws IOException {
return internalFilterKeyValue(c, c);
public ReturnCode filterKeyValue(final Cell c) throws IOException {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) throws IOException {
return internalFilterCell(c, c);
}
/**

View File

@ -58,7 +58,7 @@ public class FilterListWithAND extends FilterListBase {
* Let's call it: The Maximal Step Rule. So if filter-A in filter list return INCLUDE and filter-B
* in filter list return INCLUDE_AND_NEXT_COL, then the filter list should return
* INCLUDE_AND_NEXT_COL. For SEEK_NEXT_USING_HINT, it's more special, and in method
* filterKeyValueWithMustPassAll(), if any sub-filter return SEEK_NEXT_USING_HINT, then our filter
* filterCellWithMustPassAll(), if any sub-filter return SEEK_NEXT_USING_HINT, then our filter
* list will return SEEK_NEXT_USING_HINT. so we don't care about the SEEK_NEXT_USING_HINT here.
* <br/>
* <br/>
@ -147,7 +147,7 @@ public class FilterListWithAND extends FilterListBase {
}
@Override
ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException {
ReturnCode internalFilterCell(Cell c, Cell transformedCell) throws IOException {
if (isEmpty()) {
return ReturnCode.INCLUDE;
}
@ -162,9 +162,9 @@ public class FilterListWithAND extends FilterListBase {
}
ReturnCode localRC;
if (filter instanceof FilterList) {
localRC = ((FilterList) filter).internalFilterKeyValue(c, transformed);
localRC = ((FilterList) filter).internalFilterCell(c, transformed);
} else {
localRC = filter.filterKeyValue(c);
localRC = filter.filterCell(c);
}
rc = mergeReturnCode(rc, localRC);

View File

@ -246,7 +246,7 @@ public class FilterListWithOR extends FilterListBase {
}
@Override
ReturnCode internalFilterKeyValue(Cell c, Cell transformCell) throws IOException {
ReturnCode internalFilterCell(Cell c, Cell transformCell) throws IOException {
if (isEmpty()) {
return ReturnCode.INCLUDE;
}
@ -266,9 +266,9 @@ public class FilterListWithOR extends FilterListBase {
ReturnCode localRC;
if (filter instanceof FilterList) {
localRC = ((FilterList) filter).internalFilterKeyValue(c, transformed);
localRC = ((FilterList) filter).internalFilterCell(c, transformed);
} else {
localRC = filter.filterKeyValue(c);
localRC = filter.filterCell(c);
}
// Update previous return code and previous cell for filter[i].

View File

@ -51,8 +51,14 @@ public class FirstKeyOnlyFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if(foundKV) return ReturnCode.NEXT_ROW;
foundKV = true;
return ReturnCode.INCLUDE;
@ -107,7 +113,7 @@ public class FirstKeyOnlyFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -60,19 +60,25 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
this.qualifiers = qualifiers;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (hasFoundKV()) {
return ReturnCode.NEXT_ROW;
} else if (hasOneMatchingQualifier(v)) {
} else if (hasOneMatchingQualifier(c)) {
setFoundKV(true);
}
return ReturnCode.INCLUDE;
}
private boolean hasOneMatchingQualifier(Cell v) {
private boolean hasOneMatchingQualifier(Cell c) {
for (byte[] q : qualifiers) {
if (CellUtil.matchingQualifier(v, q)) {
if (CellUtil.matchingQualifier(c, q)) {
return true;
}
}
@ -114,7 +120,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -147,8 +147,14 @@ public class FuzzyRowFilter extends FilterBase {
return true;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell c) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0;
final int size = fuzzyKeysData.size();
for (int i = startIndex; i < size + startIndex; i++) {

View File

@ -51,8 +51,14 @@ public class InclusiveStopFilter extends FilterBase {
return this.stopRowKey;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (done) return ReturnCode.NEXT_ROW;
return ReturnCode.INCLUDE;
}
@ -105,7 +111,7 @@ public class InclusiveStopFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -67,8 +67,14 @@ public class KeyOnlyFilter extends FilterBase {
}
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterKeyValue(final Cell ignored) throws IOException {
return filterCell(ignored);
}
@Override
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}
@ -110,7 +116,7 @@ public class KeyOnlyFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -124,8 +124,14 @@ public class MultiRowRangeFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell ignored) {
public ReturnCode filterKeyValue(final Cell ignored) {
return filterCell(ignored);
}
@Override
public ReturnCode filterCell(final Cell ignored) {
return currentReturnCode;
}

View File

@ -66,12 +66,18 @@ public class MultipleColumnPrefixFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell kv) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (sortedPrefixes.isEmpty()) {
return ReturnCode.INCLUDE;
} else {
return filterColumn(kv);
return filterColumn(c);
}
}
@ -144,7 +150,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -65,8 +65,14 @@ public class PageFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterKeyValue(final Cell c) throws IOException {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}

View File

@ -75,8 +75,14 @@ public class PrefixFilter extends FilterBase {
return filterRow;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (filterRow) return ReturnCode.NEXT_ROW;
return ReturnCode.INCLUDE;
}
@ -128,7 +134,7 @@ public class PrefixFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -70,11 +70,17 @@ public class QualifierFilter extends CompareFilter {
super(op, qualifierComparator);
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
int qualifierLength = v.getQualifierLength();
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
int qualifierLength = c.getQualifierLength();
if (qualifierLength > 0) {
if (compareQualifier(getCompareOperator(), this.comparator, v)) {
if (compareQualifier(getCompareOperator(), this.comparator, c)) {
return ReturnCode.SKIP;
}
}

View File

@ -69,8 +69,14 @@ public class RandomRowFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (filterOutRow) {
return ReturnCode.NEXT_ROW;
}
@ -134,7 +140,7 @@ public class RandomRowFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -76,8 +76,14 @@ public class RowFilter extends CompareFilter {
this.filterOutRow = false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell v) {
if(this.filterOutRow) {
return ReturnCode.NEXT_ROW;
}

View File

@ -245,8 +245,14 @@ public class SingleColumnValueFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell c) {
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()));
if (this.matchedColumn) {
// We already found and matched the single column, all keys now pass

View File

@ -78,11 +78,17 @@ public class SkipFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFR(c != ReturnCode.INCLUDE);
return c;
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);
changeFR(rc != ReturnCode.INCLUDE);
return rc;
}
@Override
@ -130,7 +136,7 @@ public class SkipFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -108,11 +108,17 @@ public class TimestampsFilter extends FilterBase {
return false;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
if (this.timestamps.contains(v.getTimestamp())) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (this.timestamps.contains(c.getTimestamp())) {
return ReturnCode.INCLUDE;
} else if (v.getTimestamp() < minTimeStamp) {
} else if (c.getTimestamp() < minTimeStamp) {
// The remaining versions of this column are guaranteed
// to be lesser than all of the other values.
return ReturnCode.NEXT_COL;
@ -140,7 +146,7 @@ public class TimestampsFilter extends FilterBase {
// This should only happen if the current column's
// timestamp is below the last one in the list.
//
// It should never happen as the filterKeyValue should return NEXT_COL
// It should never happen as the filterCell should return NEXT_COL
// but it's always better to be extra safe and protect against future
// behavioral changes.
@ -193,7 +199,7 @@ public class TimestampsFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -68,9 +68,15 @@ public class ValueFilter extends CompareFilter {
super(valueCompareOp, valueComparator);
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) {
if (compareValue(getCompareOperator(), this.comparator, v)) {
public ReturnCode filterKeyValue(final Cell c) {
return filterCell(c);
}
@Override
public ReturnCode filterCell(final Cell c) {
if (compareValue(getCompareOperator(), this.comparator, c)) {
return ReturnCode.SKIP;
}
return ReturnCode.INCLUDE;

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferE
/**
* A wrapper filter that returns true from {@link #filterAllRemaining()} as soon
* as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)},
* {@link Filter#filterKeyValue(org.apache.hadoop.hbase.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
* returns true.
@ -77,11 +77,17 @@ public class WhileMatchFilter extends FilterBase {
return value;
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFAR(c != ReturnCode.INCLUDE);
return c;
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);
changeFAR(code != ReturnCode.INCLUDE);
return code;
}
@Override
@ -133,7 +139,7 @@ public class WhileMatchFilter extends FilterBase {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -683,16 +683,16 @@ public class Import extends Configured implements Tool {
/**
* Attempt to filter out the keyvalue
* @param kv {@link KeyValue} on which to apply the filter
* @param c {@link Cell} on which to apply the filter
* @return <tt>null</tt> if the key should not be written, otherwise returns the original
* {@link KeyValue}
* {@link Cell}
*/
public static Cell filterKv(Filter filter, Cell kv) throws IOException {
public static Cell filterKv(Filter filter, Cell c) throws IOException {
// apply the filter and skip this kv if the filter doesn't apply
if (filter != null) {
Filter.ReturnCode code = filter.filterKeyValue(kv);
Filter.ReturnCode code = filter.filterCell(c);
if (LOG.isTraceEnabled()) {
LOG.trace("Filter returned:" + code + " for the key value:" + kv);
LOG.trace("Filter returned:" + code + " for the cell:" + c);
}
// if its not an accept type, then skip this kv
if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
@ -700,7 +700,7 @@ public class Import extends Configured implements Tool {
return null;
}
}
return kv;
return c;
}
// helper: create a new KeyValue based on CF rename map
@ -884,7 +884,7 @@ public class Import extends Configured implements Tool {
System.err.println("By default Import will load data directly into HBase. To instead generate");
System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
System.err.println("If there is a large result that includes too much KeyValue "
System.err.println("If there is a large result that includes too much Cell "
+ "whitch can occur OOME caused by the memery sort in reducer, pass the option:");
System.err.println(" -D" + HAS_LARGE_RESULT + "=true");
System.err
@ -895,9 +895,9 @@ public class Import extends Configured implements Tool {
+ CF_RENAME_PROP + " property. Futher, filters will only use the"
+ " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
+ " whether the current row needs to be ignored completely for processing and "
+ " Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
+ " Filter#filterCell(Cell) method to determine if the Cell should be added;"
+ " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"
+ " the KeyValue.");
+ " the Cell.");
System.err.println("To import data exported from HBase 0.94, use");
System.err.println(" -Dhbase.import.version=0.94");
System.err.println(" -D " + JOB_NAME_CONF_KEY

View File

@ -112,9 +112,14 @@ final public class FilterWrapper extends Filter {
return this.filter.filterRowKey(cell);
}
@Deprecated
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
return this.filter.filterKeyValue(v);
public ReturnCode filterKeyValue(final Cell c) throws IOException {
return filterCell(c);
}
public ReturnCode filterCell(final Cell c) throws IOException {
return this.filter.filterCell(c);
}
@Override
@ -161,7 +166,7 @@ final public class FilterWrapper extends Filter {
}
/**
* @param other
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.mob.MobUtils;
class MobReferenceOnlyFilter extends FilterBase {
@Override
public ReturnCode filterKeyValue(Cell cell) {
public ReturnCode filterCell(final Cell cell) {
if (null != cell) {
// If a cell with a mob reference tag, it's included.
if (MobUtils.isMobReferenceCell(cell)) {

View File

@ -61,7 +61,7 @@ public interface ColumnTracker extends ShipperListener {
* this method are {@link MatchCode#INCLUDE}, {@link MatchCode#SEEK_NEXT_COL} and
* {@link MatchCode#SEEK_NEXT_ROW}.
* @param cell
* @param type The type of the KeyValue
* @param type The type of the Cell
* @return The match code instance.
* @throws IOException in case there is an internal consistency problem caused by a data
* corruption.
@ -70,7 +70,7 @@ public interface ColumnTracker extends ShipperListener {
/**
* Keeps track of the number of versions for the columns asked for. It assumes that the user has
* already checked if the keyvalue needs to be included by calling the
* already checked if the cell needs to be included by calling the
* {@link #checkColumn(Cell, byte)} method. The enum values returned by this method
* are {@link MatchCode#SKIP}, {@link MatchCode#INCLUDE},
* {@link MatchCode#INCLUDE_AND_SEEK_NEXT_COL} and {@link MatchCode#INCLUDE_AND_SEEK_NEXT_ROW}.

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.regionserver.ShipperListener;
* This class is utilized through three methods:
* <ul>
* <li>{@link #add} when encountering a Delete</li>
* <li>{@link #isDeleted} when checking if a Put KeyValue has been deleted</li>
* <li>{@link #isDeleted} when checking if a Put Cell has been deleted</li>
* <li>{@link #update} when reaching the end of a StoreFile</li>
* </ul>
*/
@ -47,7 +47,7 @@ public interface DeleteTracker extends ShipperListener {
/**
* Check if the specified cell buffer has been deleted by a previously seen delete.
* @param cell - current cell to check if deleted by a previously seen delete
* @return deleteResult The result tells whether the KeyValue is deleted and why
* @return deleteResult The result tells whether the Cell is deleted and why
*/
DeleteResult isDeleted(Cell cell);
@ -76,12 +76,12 @@ public interface DeleteTracker extends ShipperListener {
* and why. Based on the delete result, the ScanQueryMatcher will decide the next operation
*/
enum DeleteResult {
FAMILY_DELETED, // The KeyValue is deleted by a delete family.
FAMILY_VERSION_DELETED, // The KeyValue is deleted by a delete family version.
COLUMN_DELETED, // The KeyValue is deleted by a delete column.
VERSION_DELETED, // The KeyValue is deleted by a version delete.
FAMILY_DELETED, // The Cell is deleted by a delete family.
FAMILY_VERSION_DELETED, // The Cell is deleted by a delete family version.
COLUMN_DELETED, // The Cell is deleted by a delete column.
VERSION_DELETED, // The Cell is deleted by a version delete.
NOT_DELETED,
VERSION_MASKED // The KeyValue is masked by max number of versions which is considered as
VERSION_MASKED // The Cell is masked by max number of versions which is considered as
// deleted in strong semantics of versions(See MvccTracker)
}

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* This class is utilized through three methods:
* <ul>
* <li>{@link #add} when encountering a Delete or DeleteColumn</li>
* <li>{@link #isDeleted} when checking if a Put KeyValue has been deleted</li>
* <li>{@link #isDeleted} when checking if a Put Cell has been deleted</li>
* <li>{@link #update} when reaching the end of a StoreFile or row for scans</li>
* </ul>
* <p>
@ -63,7 +63,7 @@ public class ScanDeleteTracker implements DeleteTracker {
}
/**
* Add the specified KeyValue to the list of deletes to check against for this row operation.
* Add the specified Cell to the list of deletes to check against for this row operation.
* <p>
* This is called when a Delete is encountered.
* @param cell - the delete cell
@ -97,7 +97,7 @@ public class ScanDeleteTracker implements DeleteTracker {
}
/**
* Check if the specified KeyValue buffer has been deleted by a previously seen delete.
* Check if the specified Cell buffer has been deleted by a previously seen delete.
* @param cell - current cell to check if deleted by a previously seen delete
* @return deleteResult
*/

View File

@ -152,12 +152,12 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
}
return filter == null ? matchCode : mergeFilterResponse(cell, matchCode,
filter.filterKeyValue(cell));
filter.filterCell(cell));
}
/*
* Call this when scan has filter. Decide the desired behavior by checkVersions's MatchCode
* and filterKeyValue's ReturnCode. Cell may be skipped by filter, so the column versions
* and filterCell's ReturnCode. Cell may be skipped by filter, so the column versions
* in result may be less than user need. It will check versions again after filter.
*
* ColumnChecker FilterResponse Desired behavior

View File

@ -92,7 +92,7 @@ class AccessControlFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(Cell cell) {
public ReturnCode filterCell(final Cell cell) {
if (isSystemTable) {
return ReturnCode.INCLUDE;
}

View File

@ -1050,7 +1050,7 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
}
@Override
public ReturnCode filterKeyValue(Cell cell) throws IOException {
public ReturnCode filterCell(final Cell cell) throws IOException {
List<Tag> putVisTags = new ArrayList<>();
Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);
if (putVisTags.isEmpty() && deleteCellVisTags.isEmpty()) {

View File

@ -56,7 +56,7 @@ class VisibilityLabelFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(Cell cell) throws IOException {
public ReturnCode filterCell(final Cell cell) throws IOException {
if (curFamily.getBytes() == null
|| !(CellUtil.matchingFamily(cell, curFamily.getBytes(), curFamily.getOffset(),
curFamily.getLength()))) {

View File

@ -112,9 +112,9 @@ public abstract class AbstractTestScanCursor {
}
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell c) throws IOException {
Threads.sleep(TIMEOUT / 2 + 100);
return Bytes.equals(CellUtil.cloneRow(v), ROWS[reversed ? 0 : NUM_ROWS - 1])
return Bytes.equals(CellUtil.cloneRow(c), ROWS[reversed ? 0 : NUM_ROWS - 1])
? ReturnCode.INCLUDE
: ReturnCode.SKIP;
}

View File

@ -37,7 +37,7 @@ public final class ColumnCountOnRowFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell c) {
count++;
return count > limit ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE;
}

View File

@ -972,7 +972,7 @@ public class TestHCM {
return false;
}
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}

View File

@ -90,7 +90,7 @@ public class TestRegionObserverScannerOpenHook {
public static class NoDataFilter extends FilterBase {
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterCell(final Cell ignored) {
return ReturnCode.SKIP;
}

View File

@ -28,7 +28,7 @@ public class FilterAllFilter extends FilterBase {
}
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell c) throws IOException {
return ReturnCode.SKIP;
}

View File

@ -76,8 +76,8 @@ public class TestColumnPaginationFilter
*/
private void basicFilterTests(ColumnPaginationFilter filter) throws Exception
{
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE_AND_NEXT_COL);
KeyValue c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter1", filter.filterCell(c) == Filter.ReturnCode.INCLUDE_AND_NEXT_COL);
}
/**

View File

@ -234,11 +234,11 @@ public class TestDependentColumnFilter {
Filter filter = new DependentColumnFilter(FAMILIES[0], QUALIFIER);
List<Cell> accepted = new ArrayList<>();
for(Cell val : testVals) {
if(filter.filterKeyValue(val) == ReturnCode.INCLUDE) {
if(filter.filterCell(val) == ReturnCode.INCLUDE) {
accepted.add(val);
}
}
assertEquals("check all values accepted from filterKeyValue", 5, accepted.size());
assertEquals("check all values accepted from filterCell", 5, accepted.size());
filter.filterRowCells(accepted);
assertEquals("check filterRow(List<KeyValue>) dropped cell without corresponding column entry", 4, accepted.size());
@ -247,7 +247,7 @@ public class TestDependentColumnFilter {
filter = new DependentColumnFilter(FAMILIES[1], QUALIFIER, true);
accepted.clear();
for(KeyValue val : testVals) {
if(filter.filterKeyValue(val) == ReturnCode.INCLUDE) {
if(filter.filterCell(val) == ReturnCode.INCLUDE) {
accepted.add(val);
}
}

View File

@ -610,7 +610,7 @@ public class TestFilter {
}
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}
}
@ -666,14 +666,14 @@ public class TestFilter {
/**
* Tests the the {@link WhileMatchFilter} works in combination with a
* {@link Filter} that uses the {@link Filter#filterKeyValue(Cell)} method.
* {@link Filter} that uses the {@link Filter#filterCell(Cell)} method.
*
* See HBASE-2258.
*
* @throws Exception
*/
@Test
public void testWhileMatchFilterWithFilterKeyValue() throws Exception {
public void testWhileMatchFilterWithFilterCell() throws Exception {
Scan s = new Scan();
WhileMatchFilter filter = new WhileMatchFilter(
new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, Bytes.toBytes("foo"))
@ -2037,7 +2037,7 @@ public class TestFilter {
public byte [] toByteArray() {return null;}
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}

View File

@ -146,7 +146,7 @@ public class TestFilterFromRegionSide {
}
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterCell(final Cell v) {
if (count++ < NUM_COLS) {
return ReturnCode.INCLUDE;
}

View File

@ -137,7 +137,7 @@ public class TestFilterList {
* <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
* <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
* if false, we will also call</li>
* <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
* <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell</li>
* <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
* filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
* </li>
@ -152,7 +152,7 @@ public class TestFilterList {
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));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
assertFalse(filterMPONE.filterRow());
}
@ -161,7 +161,7 @@ public class TestFilterList {
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));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
assertFalse(filterMPONE.filterRow());
/* reach MAX_PAGES already, should filter any rows */
@ -169,7 +169,7 @@ public class TestFilterList {
assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
Bytes.toBytes(0));
assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
assertFalse(filterMPONE.filterRow());
/* We should filter any row */
@ -203,7 +203,7 @@ public class TestFilterList {
* <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
* <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
* if false, we will also call</li>
* <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
* <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell</li>
* <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
* filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
* </li>
@ -216,14 +216,14 @@ public class TestFilterList {
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));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterCell(kv));
}
filterMPALL.reset();
rowkey = Bytes.toBytes("z");
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));
assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterCell(kv));
}
/**
@ -251,7 +251,7 @@ public class TestFilterList {
* <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
* <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
* if false, we will also call</li>
* <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
* <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
* <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
* filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
* </li>
@ -266,7 +266,7 @@ public class TestFilterList {
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));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
assertFalse(filterMPONE.filterRow());
}
@ -276,7 +276,7 @@ public class TestFilterList {
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));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
assertFalse(filterMPONE.filterRow());
}
@ -286,7 +286,7 @@ public class TestFilterList {
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));
assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
assertFalse(filterMPONE.filterRow());
}
}
@ -305,23 +305,23 @@ public class TestFilterList {
FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
flist.addFilter(new PrefixFilter(r1));
flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
flist.reset();
flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
assertEquals(ReturnCode.SKIP, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
assertEquals(ReturnCode.SKIP, flist.filterCell(new KeyValue(r2, r2, r2)));
flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
flist.addFilter(new AlwaysNextColFilter());
flist.addFilter(new PrefixFilter(r1));
flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
flist.reset();
flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2)));
}
/**
@ -339,12 +339,12 @@ public class TestFilterList {
flist.addFilter(new AlwaysNextColFilter());
flist.addFilter(new InclusiveStopFilter(r1));
flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
flist.reset();
flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2)));
}
public static class AlwaysNextColFilter extends FilterBase {
@ -353,7 +353,7 @@ public class TestFilterList {
}
@Override
public ReturnCode filterKeyValue(Cell v) {
public ReturnCode filterCell(final Cell v) {
return ReturnCode.NEXT_COL;
}
@ -388,14 +388,14 @@ public class TestFilterList {
}
/**
* Test filterKeyValue logic.
* Test filterCell logic.
* @throws Exception
*/
@Test
public void testFilterKeyValue() throws Exception {
public void testFilterCell() throws Exception {
Filter includeFilter = new FilterBase() {
@Override
public Filter.ReturnCode filterKeyValue(Cell v) {
public Filter.ReturnCode filterCell(final Cell v) {
return Filter.ReturnCode.INCLUDE;
}
};
@ -404,7 +404,7 @@ public class TestFilterList {
boolean returnInclude = true;
@Override
public Filter.ReturnCode filterKeyValue(Cell v) {
public Filter.ReturnCode filterCell(final Cell v) {
Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
Filter.ReturnCode.SKIP;
returnInclude = !returnInclude;
@ -416,7 +416,7 @@ public class TestFilterList {
boolean returnIncludeOnly = false;
@Override
public Filter.ReturnCode filterKeyValue(Cell v) {
public Filter.ReturnCode filterCell(final Cell v) {
Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
returnIncludeOnly = !returnIncludeOnly;
@ -428,17 +428,17 @@ public class TestFilterList {
FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
// INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterCell(null));
// INCLUDE, SKIP, INCLUDE.
assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterCell(null));
// Check must pass all filter.
FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
// INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterCell(null));
// INCLUDE, SKIP, INCLUDE.
assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterCell(null));
}
/**
@ -458,14 +458,14 @@ public class TestFilterList {
}
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}
};
Filter filterMinHint = new FilterBase() {
@Override
public ReturnCode filterKeyValue(Cell ignored) {
public ReturnCode filterCell(final Cell ignored) {
return ReturnCode.SEEK_NEXT_USING_HINT;
}
@ -480,7 +480,7 @@ public class TestFilterList {
Filter filterMaxHint = new FilterBase() {
@Override
public ReturnCode filterKeyValue(Cell ignored) {
public ReturnCode filterCell(final Cell ignored) {
return ReturnCode.SEEK_NEXT_USING_HINT;
}
@ -521,30 +521,30 @@ public class TestFilterList {
// Should take the first hint
filterList = new FilterList(Operator.MUST_PASS_ALL,
Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
filterList.filterKeyValue(null);
filterList.filterCell(null);
assertEquals(0,
CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
filterList = new FilterList(Operator.MUST_PASS_ALL,
Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
filterList.filterKeyValue(null);
filterList.filterCell(null);
assertEquals(0,
CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
// Should have first hint even if a filter has no hint
filterList = new FilterList(Operator.MUST_PASS_ALL,
Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint }));
filterList.filterKeyValue(null);
filterList.filterCell(null);
assertEquals(0,
CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
filterList = new FilterList(Operator.MUST_PASS_ALL,
Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
filterList.filterKeyValue(null);
filterList.filterCell(null);
assertEquals(0,
CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
filterList = new FilterList(Operator.MUST_PASS_ALL,
Arrays.asList(new Filter[] { filterNoHint, filterMinHint }));
filterList.filterKeyValue(null);
filterList.filterCell(null);
assertEquals(0,
CellComparatorImpl.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
}
@ -552,7 +552,7 @@ public class TestFilterList {
/**
* Tests the behavior of transform() in a hierarchical filter.
*
* transform() only applies after a filterKeyValue() whose return-code includes the KeyValue.
* transform() only applies after a filterCell() whose return-code includes the KeyValue.
* Lazy evaluation of AND
*/
@Test
@ -577,18 +577,18 @@ public class TestFilterList {
Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value"));
// Value for fam:qual1 should be stripped:
assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1));
assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual1));
final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual1));
assertEquals(0, transformedQual1.getValueLength());
// Value for fam:qual2 should not be stripped:
assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2));
assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual2));
final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual2));
assertEquals("value", Bytes.toString(transformedQual2.getValueArray(),
transformedQual2.getValueOffset(), transformedQual2.getValueLength()));
// Other keys should be skipped:
assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
assertEquals(Filter.ReturnCode.SKIP, flist.filterCell(kvQual3));
}
@Test
@ -603,16 +603,16 @@ public class TestFilterList {
KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
3, Bytes.toBytes("value"));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterKeyValue(kv1));
assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv2));
assertEquals(ReturnCode.SKIP, filterList01.filterKeyValue(kv3));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterCell(kv1));
assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv2));
assertEquals(ReturnCode.SKIP, filterList01.filterCell(kv3));
FilterList filterList11 =
new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv1));
assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv2));
assertEquals(ReturnCode.SKIP, filterList11.filterKeyValue(kv3));
assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv1));
assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv2));
assertEquals(ReturnCode.SKIP, filterList11.filterCell(kv3));
}
@Test
@ -630,10 +630,10 @@ public class TestFilterList {
KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
Bytes.toBytes("value"));
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv2));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv3));
assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv4));
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
assertEquals(ReturnCode.SKIP, filterList.filterCell(kv2));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3));
assertEquals(ReturnCode.SKIP, filterList.filterCell(kv4));
}
private static class MockFilter extends FilterBase {
@ -645,7 +645,7 @@ public class TestFilterList {
}
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell v) throws IOException {
this.didCellPassToTheFilter = true;
return targetRetCode;
}
@ -665,65 +665,65 @@ public class TestFilterList {
MockFilter mockFilter = new MockFilter(ReturnCode.NEXT_COL);
FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
filter.filterKeyValue(kv1);
filter.filterCell(kv1);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv2);
filter.filterCell(kv2);
assertFalse(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv3);
filter.filterCell(kv3);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
filter.filterKeyValue(kv1);
filter.filterCell(kv1);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv2);
filter.filterCell(kv2);
assertFalse(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv3);
filter.filterCell(kv3);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter = new MockFilter(ReturnCode.NEXT_ROW);
filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
filter.filterKeyValue(kv1);
filter.filterCell(kv1);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv2);
filter.filterCell(kv2);
assertFalse(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv3);
filter.filterCell(kv3);
assertFalse(mockFilter.didCellPassToTheFilter);
filter.reset();
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv4);
filter.filterCell(kv4);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
filter.filterKeyValue(kv1);
filter.filterCell(kv1);
assertTrue(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv2);
filter.filterCell(kv2);
assertFalse(mockFilter.didCellPassToTheFilter);
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv3);
filter.filterCell(kv3);
assertFalse(mockFilter.didCellPassToTheFilter);
filter.reset();
mockFilter.didCellPassToTheFilter = false;
filter.filterKeyValue(kv4);
filter.filterCell(kv4);
assertTrue(mockFilter.didCellPassToTheFilter);
}
@ -740,29 +740,29 @@ public class TestFilterList {
MockFilter filter7 = new MockFilter(ReturnCode.NEXT_ROW);
FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, filter3);
assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6);
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6);
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1);
assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter2, filter1, filter5);
assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter2,
new FilterList(Operator.MUST_PASS_ALL, filter3, filter4));
assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter7);
assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
}
@Test
@ -776,33 +776,33 @@ public class TestFilterList {
MockFilter filter5 = new MockFilter(ReturnCode.SKIP);
MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
assertEquals(filterList.filterKeyValue(kv1), ReturnCode.INCLUDE);
assertEquals(filterList.filterCell(kv1), ReturnCode.INCLUDE);
filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, filter3);
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter5, filter6);
assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter6);
assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter1);
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter2, filter1, filter5);
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6);
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
}
static class MockSeekHintFilter extends FilterBase {
@ -813,7 +813,7 @@ public class TestFilterList {
}
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell v) throws IOException {
return ReturnCode.SEEK_NEXT_USING_HINT;
}
@ -844,7 +844,7 @@ public class TestFilterList {
filterList.addFilter(filter2);
filterList.addFilter(filter3);
Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
Assert.assertEquals(kv3, filterList.getNextCellHint(kv1));
filterList = new FilterList(Operator.MUST_PASS_ALL);
@ -853,7 +853,7 @@ public class TestFilterList {
filterList.addFilter(filter2);
filterList.addFilter(filter3);
Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
Assert.assertEquals(kv1, filterList.getNextCellHint(kv1));
}
@ -869,12 +869,12 @@ public class TestFilterList {
prefixFilter.setReversed(true);
filterList.addFilter(prefixFilter);
filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22)));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2)));
filterList.reset();
filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.SKIP, filterList.filterCell(new KeyValue(r1, r1, r1)));
filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
filterList.setReversed(true);
@ -885,12 +885,12 @@ public class TestFilterList {
filterList.addFilter(alwaysNextColFilter);
filterList.addFilter(prefixFilter);
filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22)));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2)));
filterList.reset();
filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(new KeyValue(r1, r1, r1)));
assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(new KeyValue(r1, r1, r1)));
}
@Test
@ -910,20 +910,20 @@ public class TestFilterList {
FilterList keyOnlyFilterFirst =
new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv1));
c = keyOnlyFilterFirst.transformCell(kv1);
assertEquals(0, c.getValueLength());
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(kv2));
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv2));
c = keyOnlyFilterFirst.transformCell(kv2);
assertEquals(0, c.getValueLength());
internalFilterList.reset();
FilterList keyOnlyFilterLast =
new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv1));
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv1));
c = keyOnlyFilterLast.transformCell(kv1);
assertEquals(0, c.getValueLength());
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(kv2));
assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv2));
c = keyOnlyFilterLast.transformCell(kv2);
assertEquals(0, c.getValueLength());
}
@ -933,18 +933,18 @@ public class TestFilterList {
KeyValue kv = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
1, Bytes.toBytes("value"));
FilterList filterList = new FilterList(Operator.MUST_PASS_ALL);
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
assertEquals(kv, filterList.transformCell(kv));
filterList = new FilterList(Operator.MUST_PASS_ONE);
assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
assertEquals(kv, filterList.transformCell(kv));
}
private static class MockNextRowFilter extends FilterBase {
private int hitCount = 0;
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell v) throws IOException {
hitCount++;
return ReturnCode.NEXT_ROW;
}
@ -962,8 +962,8 @@ public class TestFilterList {
Bytes.toBytes("value"));
MockNextRowFilter mockNextRowFilter = new MockNextRowFilter();
FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockNextRowFilter);
filter.filterKeyValue(kv1);
filter.filterKeyValue(kv2);
filter.filterCell(kv1);
filter.filterCell(kv2);
assertEquals(2, mockNextRowFilter.getHitCount());
}
}

View File

@ -40,7 +40,7 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
/**
* Test the functionality of
* {@link FirstKeyValueMatchingQualifiersFilter#filterKeyValue(org.apache.hadoop.hbase.Cell)}
* {@link FirstKeyValueMatchingQualifiersFilter#filterCell(org.apache.hadoop.hbase.Cell)}
*
* @throws Exception
*/
@ -51,26 +51,26 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
Filter filter = new FirstKeyValueMatchingQualifiersFilter(quals);
// Match in first attempt
KeyValue kv;
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
KeyValue cell;
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
assertTrue("includeAndSetFlag",
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("flagIsSetSkipToNextRow",
filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
// A mismatch in first attempt and match in second attempt.
filter.reset();
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1);
System.out.println(filter.filterKeyValue(kv));
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1);
System.out.println(filter.filterCell(cell));
assertTrue("includeFlagIsUnset",
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("includeAndSetFlag",
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
assertTrue("flagIsSetSkipToNextRow",
filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
}
}

View File

@ -166,7 +166,7 @@ public class TestInvocationRecordFilter {
visitedKeyValues.clear();
}
public ReturnCode filterKeyValue(Cell ignored) {
public ReturnCode filterCell(final Cell ignored) {
visitedKeyValues.add(ignored);
return ReturnCode.INCLUDE;
}

View File

@ -97,7 +97,7 @@ public class TestMultiRowRangeFilter {
* Expected :SEEK_NEXT_USING_HINT
* Actual :INCLUDE
* */
assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null));
assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterCell(null));
}
@Test
@ -107,15 +107,15 @@ public class TestMultiRowRangeFilter {
new MultiRowRangeFilter.RowRange(Bytes.toBytes("d"), true, Bytes.toBytes("e"), true)
));
filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("a")));
assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null));
assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterCell(null));
filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("b")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("c")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("d")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("e")));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
assertEquals(Filter.ReturnCode.INCLUDE, filter.filterCell(null));
}
@Test

View File

@ -37,7 +37,7 @@ import java.util.ArrayList;
/**
* Tests for {@link SingleColumnValueExcludeFilter}. Because this filter
* extends {@link SingleColumnValueFilter}, only the added functionality is
* tested. That is, method filterKeyValue(KeyValue).
* tested. That is, method filterCell(Cell).
*
*/
@Category({FilterTests.class, SmallTests.class})
@ -50,17 +50,17 @@ public class TestSingleColumnValueExcludeFilter {
private static final byte[] VAL_2 = Bytes.toBytes("ab");
/**
* Test the overridden functionality of filterKeyValue(KeyValue)
* Test the overridden functionality of filterCell(Cell)
* @throws Exception
*/
@Test
public void testFilterKeyValue() throws Exception {
public void testFilterCell() throws Exception {
Filter filter = new SingleColumnValueExcludeFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
CompareOperator.EQUAL, VAL_1);
// A 'match' situation
List<Cell> kvs = new ArrayList<>();
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
KeyValue c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
kvs.add (new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1));
kvs.add (new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1));
@ -69,21 +69,21 @@ public class TestSingleColumnValueExcludeFilter {
filter.filterRowCells(kvs);
assertEquals("resultSize", kvs.size(), 2);
assertTrue("leftKV1", CellComparatorImpl.COMPARATOR.compare(kvs.get(0), kv) == 0);
assertTrue("leftKV2", CellComparatorImpl.COMPARATOR.compare(kvs.get(1), kv) == 0);
assertTrue("leftKV1", CellComparatorImpl.COMPARATOR.compare(kvs.get(0), c) == 0);
assertTrue("leftKV2", CellComparatorImpl.COMPARATOR.compare(kvs.get(1), c) == 0);
assertFalse("allRemainingWhenMatch", filter.filterAllRemaining());
// A 'mismatch' situation
filter.reset();
// INCLUDE expected because test column has not yet passed
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
// Test column will pass (wont match), expect NEXT_ROW
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("testedMismatch", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("testedMismatch", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
// After a mismatch (at least with LatestVersionOnly), subsequent columns are EXCLUDE
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
c = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
}

View File

@ -104,143 +104,143 @@ public class TestSingleColumnValueFilter {
public void testLongComparator() throws IOException {
Filter filter = new SingleColumnValueFilter(COLUMN_FAMILY,
COLUMN_QUALIFIER, CompareOperator.GREATER, new LongComparator(100L));
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
Bytes.toBytes(1L));
assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
assertTrue("less than", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
filter.reset();
byte[] buffer = kv.getBuffer();
byte[] buffer = cell.getBuffer();
Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
assertTrue("less than", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
filter.reset();
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
Bytes.toBytes(100L));
assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
assertTrue("Equals 100", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
filter.reset();
buffer = kv.getBuffer();
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
assertTrue("Equals 100", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
filter.reset();
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
Bytes.toBytes(120L));
assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
assertTrue("include 120", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
filter.reset();
buffer = kv.getBuffer();
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("include 120", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
}
private void basicFilterTests(SingleColumnValueFilter filter)
throws Exception {
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
byte[] buffer = kv.getBuffer();
KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("basicFilter1", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
byte[] buffer = cell.getBuffer();
Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
assertTrue("basicFilter1", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
assertTrue("basicFilter2", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
assertTrue("basicFilter2", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
assertTrue("basicFilter3", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("basicFilter3", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertFalse("basicFilterNotNull", filter.filterRow());
filter.reset();
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
buffer = kv.getBuffer();
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter4", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
buffer = kv.getBuffer();
assertTrue("basicFilter4", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("basicFilter4", filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
assertTrue("basicFilter4", filter.filterCell(c) == Filter.ReturnCode.NEXT_ROW);
assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
assertTrue("basicFilterNotNull", filter.filterRow());
filter.reset();
filter.setLatestVersionOnly(false);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter5", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
assertTrue("basicFilter5", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("basicFilter5", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("basicFilter5", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertFalse("basicFilterNotNull", filter.filterRow());
}
private void nullFilterTests(Filter filter) throws Exception {
((SingleColumnValueFilter) filter).setFilterIfMissing(true);
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
byte[] buffer = kv.getBuffer();
KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
assertTrue("null1", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
byte[] buffer = cell.getBuffer();
Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("null1", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertFalse("null1FilterRow", filter.filterRow());
filter.reset();
kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
cell = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
assertTrue("null2", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("null2", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertTrue("null2FilterRow", filter.filterRow());
}
private void substrFilterTests(Filter filter)
throws Exception {
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
FULLSTRING_1);
assertTrue("substrTrue",
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
byte[] buffer = kv.getBuffer();
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
byte[] buffer = cell.getBuffer();
Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
assertTrue("substrTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
FULLSTRING_2);
assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
assertTrue("substrFalse", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("substrFalse", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
assertFalse("substrFilterNotNull", filter.filterRow());
}
private void regexFilterTests(Filter filter)
throws Exception {
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
FULLSTRING_1);
assertTrue("regexTrue",
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
byte[] buffer = kv.getBuffer();
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
byte[] buffer = cell.getBuffer();
Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
assertTrue("regexTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
FULLSTRING_2);
assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
buffer = kv.getBuffer();
assertTrue("regexFalse", filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
buffer = cell.getBuffer();
c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("regexFalse", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
assertFalse("regexFilterNotNull", filter.filterRow());
}
private void regexPatternFilterTests(Filter filter)
throws Exception {
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
KeyValue cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
FULLSTRING_1);
assertTrue("regexTrue",
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
byte[] buffer = kv.getBuffer();
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
byte[] buffer = cell.getBuffer();
Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
assertTrue("regexTrue", filter.filterCell(c) == Filter.ReturnCode.INCLUDE);
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
assertFalse("regexFilterNotNull", filter.filterRow());
}

View File

@ -3451,7 +3451,7 @@ public class TestHRegion {
scan.setLoadColumnFamiliesOnDemand(true);
Filter bogusFilter = new FilterBase() {
@Override
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}
@Override

View File

@ -1140,7 +1140,7 @@ public class TestHStore {
}
}, new FilterBase() {
@Override
public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
public Filter.ReturnCode filterCell(final Cell c) throws IOException {
return ReturnCode.INCLUDE;
}
}, expectedSize);
@ -1164,7 +1164,7 @@ public class TestHStore {
}
}, new FilterBase() {
@Override
public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
public Filter.ReturnCode filterCell(final Cell c) throws IOException {
if (timeToGoNextRow.get()) {
timeToGoNextRow.set(false);
return ReturnCode.NEXT_ROW;
@ -1193,7 +1193,7 @@ public class TestHStore {
}
}, new FilterBase() {
@Override
public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
public Filter.ReturnCode filterCell(final Cell c) throws IOException {
if (timeToGetHint.get()) {
timeToGetHint.set(false);
return Filter.ReturnCode.SEEK_NEXT_USING_HINT;

View File

@ -273,7 +273,7 @@ public class TestScannerHeartbeatMessages {
public static class SparseFilter extends FilterBase {
@Override
public ReturnCode filterKeyValue(Cell v) throws IOException {
public ReturnCode filterCell(final Cell v) throws IOException {
try {
Thread.sleep(CLIENT_TIMEOUT / 2 + 100);
} catch (InterruptedException e) {

View File

@ -107,7 +107,7 @@ public class SparkSQLPushDownFilter extends FilterBase{
}
@Override
public ReturnCode filterKeyValue(Cell c) throws IOException {
public ReturnCode filterCell(final Cell c) throws IOException {
//If the map RowValueMap is empty then we need to populate
// the row key