HBASE-10609 Remove filterKeyValue(Cell ignored) from FilterBase
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1573774 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e196b203ac
commit
f3233d72ba
|
@ -98,6 +98,10 @@ public abstract class Filter {
|
|||
* code is described below. This allows filters to filter only certain number of columns, then
|
||||
* terminate without matching ever column.
|
||||
*
|
||||
* If filterRowKey returns true, filterKeyValue needs to be consistent with it.
|
||||
*
|
||||
* filterKeyValue can assume that filterRowKey has already been called for the row.
|
||||
*
|
||||
* If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
|
||||
* <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called just in case the caller calls
|
||||
* for the next row.
|
||||
|
|
|
@ -72,17 +72,6 @@ public abstract class FilterBase extends Filter {
|
|||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Filters that dont filter by key value can inherit this implementation that
|
||||
* includes all Cells.
|
||||
*
|
||||
* @inheritDoc
|
||||
*/
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
/**
|
||||
* By default no transformation takes place
|
||||
*
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -27,6 +28,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -57,6 +59,11 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
return v.createKeyOnly(this.lenAsVal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument((filterArguments.size() == 0 || filterArguments.size() == 1),
|
||||
"Expected: 0 or 1 but got: %s", filterArguments.size());
|
||||
|
|
|
@ -22,9 +22,12 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
/**
|
||||
* Implementation of Filter interface that limits results to a specific page
|
||||
|
@ -57,6 +60,11 @@ public class PageFilter extends FilterBase {
|
|||
return pageSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public boolean filterAllRemaining() {
|
||||
return this.rowsAccepted >= this.pageSize;
|
||||
}
|
||||
|
|
|
@ -516,6 +516,10 @@ public class TestHCM {
|
|||
syncBlockingFilter.set(true);
|
||||
return false;
|
||||
}
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException{
|
||||
return new BlockingFilter();
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.Durability;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
|
||||
import org.apache.hadoop.hbase.filter.FilterList.Operator;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
|
@ -598,6 +599,11 @@ public class TestFilter {
|
|||
// always filter out rows
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1984,6 +1990,11 @@ public class TestFilter {
|
|||
@Override
|
||||
public byte [] toByteArray() {return null;}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean filterRow() throws IOException {
|
||||
ipcHandlerThread = Thread.currentThread();
|
||||
|
|
|
@ -419,7 +419,14 @@ public class TestFilterList {
|
|||
|
||||
Filter filterNoHint = new FilterBase() {
|
||||
@Override
|
||||
public byte [] toByteArray() {return null;}
|
||||
public byte [] toByteArray() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
};
|
||||
|
||||
Filter filterMinHint = new FilterBase() {
|
||||
|
|
|
@ -96,6 +96,7 @@ import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
|||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
|
@ -2638,6 +2639,10 @@ public class TestHRegion {
|
|||
Scan scan = new Scan();
|
||||
scan.setLoadColumnFamiliesOnDemand(true);
|
||||
Filter bogusFilter = new FilterBase() {
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(Cell ignored) throws IOException {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
@Override
|
||||
public boolean isFamilyEssential(byte[] name) {
|
||||
return Bytes.equals(name, cf_first);
|
||||
|
|
Loading…
Reference in New Issue