HBASE-7481 Allow IOExceptions to be thrown from Filter methods

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1458554 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-03-19 22:47:55 +00:00
parent 7c903f685e
commit 1d6a98257a
14 changed files with 212 additions and 140 deletions

View File

@ -19,13 +19,14 @@
package org.apache.hadoop.hbase.filter;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import java.util.List;
/**
* Interface for row and column filters directly applied within the regionserver.
* A filter can expect the following call sequence:
@ -54,58 +55,77 @@ import java.util.List;
public abstract class Filter {
/**
* Reset the state of the filter between rows.
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public void reset();
abstract public void reset() throws IOException;
/**
* Filters a row based on the row key. If this returns true, the entire
* row will be excluded. If false, each KeyValue in the row will be
* passed to {@link #filterKeyValue(KeyValue)} below.
*
* 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(KeyValue)} below.
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param buffer buffer containing row key
* @param offset offset into buffer where row key starts
* @param length length of the row key
* @return true, remove entire row, false, include the row (maybe).
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public boolean filterRowKey(byte [] buffer, int offset, int length);
abstract public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException;
/**
* If this returns true, the scan will terminate.
*
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @return true to end scan, false to continue.
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public boolean filterAllRemaining();
abstract public boolean filterAllRemaining() throws IOException;
/**
* 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.
*
* 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 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.
*
* <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 v the KeyValue 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 KeyValue v);
abstract public ReturnCode filterKeyValue(final KeyValue v) throws IOException;
/**
* Give the filter a chance to transform the passed KeyValue.
* If the KeyValue is changed a new KeyValue object must be returned.
* Give the filter a chance to transform the passed KeyValue. If the KeyValue is changed a new
* KeyValue object must be returned.
*
* @see org.apache.hadoop.hbase.KeyValue#shallowCopy()
*
* The transformed KeyValue is what is eventually returned to the
* client. Most filters will return the passed KeyValue unchanged.
* @see org.apache.hadoop.hbase.filter.KeyOnlyFilter#transform(KeyValue)
* for an example of a transformation.
*
* The transformed KeyValue is what is eventually returned to the client. Most filters will
* return the passed KeyValue unchanged.
* @see org.apache.hadoop.hbase.filter.KeyOnlyFilter#transform(KeyValue) for an example of a
* transformation.
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param v the KeyValue in question
* @return the changed KeyValue
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public KeyValue transform(final KeyValue v);
abstract public KeyValue transform(final KeyValue v) throws IOException;
/**
* Return codes for filterValue().
@ -139,55 +159,84 @@ public abstract class Filter {
}
/**
* Chance to alter the list of keyvalues to be submitted.
* Modifications to the list will carry on
* Chance to alter the list of keyvalues to be submitted. Modifications to the list will carry on
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param kvs the list of keyvalues to be filtered
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public void filterRow(List<KeyValue> kvs);
abstract public void filterRow(List<KeyValue> kvs) throws IOException;
/**
* Primarily used to check for conflicts with scans(such as scans that do not read a full row at a
* time).
*
* @return True if this filter actively uses filterRow(List) or filterRow().
* Primarily used to check for conflicts with scans(such as scans
* that do not read a full row at a time)
*/
abstract public boolean hasFilterRow();
/**
* Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)}
* 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).
* Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)} 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).
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @return true to exclude row, false to include row.
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public boolean filterRow();
abstract public boolean filterRow() throws IOException;
/**
* If the filter returns the match code SEEK_NEXT_USING_HINT, then
* it should also tell which is the next key it must seek to.
* After receiving the match code SEEK_NEXT_USING_HINT, the QueryMatcher would
* call this function to find out which key it must next seek to.
* @return KeyValue which must be next seeked. return null if the filter is
* not sure which key to seek to next.
* If the filter returns the match code SEEK_NEXT_USING_HINT, then it should also tell which is
* the next key it must seek to. After receiving the match code SEEK_NEXT_USING_HINT, the
* QueryMatcher would call this function to find out which key it must next seek to.
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @return KeyValue which must be next seeked. return null if the filter is not sure which key to
* seek to next.
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public KeyValue getNextKeyHint(final KeyValue currentKV);
abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException;
/**
* Check that given column family is essential for filter to check row. Most
* filters always return true here. But some could have more sophisticated
* logic which could significantly reduce scanning process by not even
* touching columns until we are 100% sure that it's data is needed in result.
* Check that given column family is essential for filter to check row. Most filters always return
* true here. But some could have more sophisticated logic which could significantly reduce
* scanning process by not even touching columns until we are 100% sure that it's data is needed
* in result.
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public boolean isFamilyEssential(byte[] name);
abstract public boolean isFamilyEssential(byte[] name) throws IOException;
/**
* TODO: JAVADOC
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @return The filter serialized using pb
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract public byte [] toByteArray();
abstract public byte[] toByteArray() throws IOException;
/**
*
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param pbBytes A pb serialized {@link Filter} instance
* @return An instance of {@link Filter} made from <code>bytes</code>
* @throws DeserializationException
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
* @see #toByteArray
*/
public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException {
@ -196,9 +245,13 @@ public abstract class Filter {
}
/**
* Concrete implementers can signal a failure condition in their code by throwing an
* {@link IOException}.
*
* @param other
* @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.
* @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.
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
*/
abstract boolean areSerializedFieldsEqual(Filter other);
}
}

View File

@ -18,13 +18,14 @@
package org.apache.hadoop.hbase.filter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
import java.util.ArrayList;
import java.util.List;
/**
* Abstract base class to help you implement new Filters. Common "ignore" or NOOP type
* methods can go here, helping to reduce boiler plate in an ever-expanding filter
@ -44,7 +45,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public void reset() {
public void reset() throws IOException {
}
/**
@ -54,7 +55,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public boolean filterRowKey(byte [] buffer, int offset, int length) {
public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
return false;
}
@ -65,7 +66,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public boolean filterAllRemaining() {
public boolean filterAllRemaining() throws IOException {
return false;
}
@ -76,7 +77,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public ReturnCode filterKeyValue(KeyValue ignored) {
public ReturnCode filterKeyValue(KeyValue ignored) throws IOException {
return ReturnCode.INCLUDE;
}
@ -86,7 +87,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public KeyValue transform(KeyValue v) {
public KeyValue transform(KeyValue v) throws IOException {
return v;
}
@ -97,7 +98,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public void filterRow(List<KeyValue> ignored) {
public void filterRow(List<KeyValue> ignored) throws IOException {
}
/**
@ -119,7 +120,7 @@ public abstract class FilterBase extends Filter {
* @inheritDoc
*/
@Override
public boolean filterRow() {
public boolean filterRow() throws IOException {
return false;
}
@ -129,7 +130,7 @@ public abstract class FilterBase extends Filter {
*
* @inheritDoc
*/
public KeyValue getNextKeyHint(KeyValue currentKV) {
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
return null;
}
@ -139,7 +140,7 @@ public abstract class FilterBase extends Filter {
*
* @inheritDoc
*/
public boolean isFamilyEssential(byte[] name) {
public boolean isFamilyEssential(byte[] name) throws IOException {
return true;
}
@ -163,7 +164,7 @@ public abstract class FilterBase extends Filter {
/**
* Return length 0 byte array for Filters that don't require special serialization
*/
public byte [] toByteArray() {
public byte[] toByteArray() throws IOException {
return new byte[0];
}

View File

@ -18,7 +18,11 @@
*/
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -29,10 +33,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Implementation of {@link Filter} that represents an ordered List of Filters
@ -141,14 +142,14 @@ public class FilterList extends Filter {
}
@Override
public void reset() {
public void reset() throws IOException {
for (Filter filter : filters) {
filter.reset();
}
}
@Override
public boolean filterRowKey(byte[] rowKey, int offset, int length) {
public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
for (Filter filter : filters) {
if (this.operator == Operator.MUST_PASS_ALL) {
if (filter.filterAllRemaining() ||
@ -166,7 +167,7 @@ public class FilterList extends Filter {
}
@Override
public boolean filterAllRemaining() {
public boolean filterAllRemaining() throws IOException {
for (Filter filter : filters) {
if (filter.filterAllRemaining()) {
if (operator == Operator.MUST_PASS_ALL) {
@ -182,7 +183,7 @@ public class FilterList extends Filter {
}
@Override
public KeyValue transform(KeyValue v) {
public KeyValue transform(KeyValue v) throws IOException {
KeyValue current = v;
for (Filter filter : filters) {
current = filter.transform(current);
@ -191,7 +192,7 @@ public class FilterList extends Filter {
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
ReturnCode rc = operator == Operator.MUST_PASS_ONE?
ReturnCode.SKIP: ReturnCode.INCLUDE;
for (Filter filter : filters) {
@ -242,7 +243,7 @@ public class FilterList extends Filter {
}
@Override
public void filterRow(List<KeyValue> kvs) {
public void filterRow(List<KeyValue> kvs) throws IOException {
for (Filter filter : filters) {
filter.filterRow(kvs);
}
@ -259,7 +260,7 @@ public class FilterList extends Filter {
}
@Override
public boolean filterRow() {
public boolean filterRow() throws IOException {
for (Filter filter : filters) {
if (operator == Operator.MUST_PASS_ALL) {
if (filter.filterRow()) {
@ -277,7 +278,7 @@ public class FilterList extends Filter {
/**
* @return The filter serialized using pb
*/
public byte [] toByteArray() {
public byte[] toByteArray() throws IOException {
FilterProtos.FilterList.Builder builder =
FilterProtos.FilterList.newBuilder();
builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
@ -329,7 +330,7 @@ public class FilterList extends Filter {
}
@Override
public KeyValue getNextKeyHint(KeyValue currentKV) {
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
KeyValue keyHint = null;
for (Filter filter : filters) {
KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
@ -359,7 +360,7 @@ public class FilterList extends Filter {
}
@Override
public boolean isFamilyEssential(byte[] name) {
public boolean isFamilyEssential(byte[] name) throws IOException {
for (Filter filter : filters) {
if (filter.isFamilyEssential(name)) {
return true;

View File

@ -19,7 +19,9 @@
*/
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
@ -27,8 +29,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import java.util.List;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* This is a Filter wrapper class which is used in the server side. Some filter
@ -53,7 +54,7 @@ public class FilterWrapper extends Filter {
/**
* @return The filter serialized using pb
*/
public byte [] toByteArray() {
public byte[] toByteArray() throws IOException {
FilterProtos.FilterWrapper.Builder builder =
FilterProtos.FilterWrapper.newBuilder();
builder.setFilter(ProtobufUtil.toFilter(this.filter));
@ -82,37 +83,37 @@ public class FilterWrapper extends Filter {
}
@Override
public void reset() {
public void reset() throws IOException {
this.filter.reset();
}
@Override
public boolean filterAllRemaining() {
public boolean filterAllRemaining() throws IOException {
return this.filter.filterAllRemaining();
}
@Override
public boolean filterRow() {
public boolean filterRow() throws IOException {
return this.filter.filterRow();
}
@Override
public KeyValue getNextKeyHint(KeyValue currentKV) {
public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
return this.filter.getNextKeyHint(currentKV);
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
return this.filter.filterRowKey(buffer, offset, length);
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
return this.filter.filterKeyValue(v);
}
@Override
public KeyValue transform(KeyValue v) {
public KeyValue transform(KeyValue v) throws IOException {
return this.filter.transform(v);
}
@ -122,7 +123,7 @@ public class FilterWrapper extends Filter {
}
@Override
public void filterRow(List<KeyValue> kvs) {
public void filterRow(List<KeyValue> kvs) throws IOException {
//To fix HBASE-6429,
//Filter with filterRow() returning true is incompatible with scan with limit
//1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
@ -135,9 +136,9 @@ public class FilterWrapper extends Filter {
}
@Override
public boolean isFamilyEssential(byte[] name) {
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
};
}
/**
* @param other

View File

@ -19,7 +19,8 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
@ -27,7 +28,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A wrapper filter that filters an entire row if any of the KeyValue checks do
@ -62,7 +63,8 @@ public class SkipFilter extends FilterBase {
return filter;
}
public void reset() {
@Override
public void reset() throws IOException {
filter.reset();
filterRow = false;
}
@ -71,14 +73,15 @@ public class SkipFilter extends FilterBase {
filterRow = filterRow || value;
}
public ReturnCode filterKeyValue(KeyValue v) {
@Override
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFR(c != ReturnCode.INCLUDE);
return c;
}
@Override
public KeyValue transform(KeyValue v) {
public KeyValue transform(KeyValue v) throws IOException {
return filter.transform(v);
}
@ -93,7 +96,7 @@ public class SkipFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
public byte [] toByteArray() {
public byte[] toByteArray() throws IOException {
FilterProtos.SkipFilter.Builder builder =
FilterProtos.SkipFilter.newBuilder();
builder.setFilter(ProtobufUtil.toFilter(this.filter));
@ -134,7 +137,7 @@ public class SkipFilter extends FilterBase {
return getFilter().areSerializedFieldsEqual(other.getFilter());
}
public boolean isFamilyEssential(byte[] name) {
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
}
@ -142,4 +145,4 @@ public class SkipFilter extends FilterBase {
public String toString() {
return this.getClass().getSimpleName() + " " + this.filter.toString();
}
}
}

View File

@ -19,7 +19,8 @@
package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.KeyValue;
@ -27,7 +28,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.io.IOException;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* A wrapper filter that returns true from {@link #filterAllRemaining()} as soon
@ -51,7 +52,7 @@ public class WhileMatchFilter extends FilterBase {
return filter;
}
public void reset() {
public void reset() throws IOException {
this.filter.reset();
}
@ -59,33 +60,38 @@ public class WhileMatchFilter extends FilterBase {
filterAllRemaining = filterAllRemaining || value;
}
public boolean filterAllRemaining() {
@Override
public boolean filterAllRemaining() throws IOException {
return this.filterAllRemaining || this.filter.filterAllRemaining();
}
public boolean filterRowKey(byte[] buffer, int offset, int length) {
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
boolean value = filter.filterRowKey(buffer, offset, length);
changeFAR(value);
return value;
}
public ReturnCode filterKeyValue(KeyValue v) {
@Override
public ReturnCode filterKeyValue(KeyValue v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFAR(c != ReturnCode.INCLUDE);
return c;
}
@Override
public KeyValue transform(KeyValue v) {
public KeyValue transform(KeyValue v) throws IOException {
return filter.transform(v);
}
public boolean filterRow() {
@Override
public boolean filterRow() throws IOException {
boolean filterRow = this.filter.filterRow();
changeFAR(filterRow);
return filterRow;
}
@Override
public boolean hasFilterRow() {
return true;
}
@ -93,7 +99,7 @@ public class WhileMatchFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
public byte [] toByteArray() {
public byte[] toByteArray() throws IOException {
FilterProtos.WhileMatchFilter.Builder builder =
FilterProtos.WhileMatchFilter.newBuilder();
builder.setFilter(ProtobufUtil.toFilter(this.filter));
@ -134,7 +140,7 @@ public class WhileMatchFilter extends FilterBase {
return getFilter().areSerializedFieldsEqual(other.getFilter());
}
public boolean isFamilyEssential(byte[] name) {
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
}

View File

@ -910,7 +910,7 @@ public final class ProtobufUtil {
* @param filter the Filter to convert
* @return the converted protocol buffer Filter
*/
public static HBaseProtos.Filter toFilter(Filter filter) {
public static HBaseProtos.Filter toFilter(Filter filter) throws IOException {
HBaseProtos.Filter.Builder builder = HBaseProtos.Filter.newBuilder();
builder.setName(filter.getClass().getName());
builder.setSerializedFilter(ByteString.copyFrom(filter.toByteArray()));

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
@ -246,7 +246,7 @@ public class Import {
* @return <tt>null</tt> if the key should not be written, otherwise returns the original
* {@link KeyValue}
*/
private static KeyValue filterKv(KeyValue kv) {
private static KeyValue filterKv(KeyValue kv) throws IOException {
// apply the filter and skip this kv if the filter doesn't apply
if (filter != null) {
Filter.ReturnCode code = filter.filterKeyValue(kv);

View File

@ -3385,10 +3385,13 @@ public class HRegion implements HeapSize { // , Writable{
public long getMvccReadPoint() {
return this.readPt;
}
/**
* Reset both the filter and the old filter.
*
* @throws IOException in case a filter raises an I/O exception.
*/
protected void resetFilters() {
protected void resetFilters() throws IOException {
if (filter != null) {
filter.reset();
}
@ -3504,7 +3507,7 @@ public class HRegion implements HeapSize { // , Writable{
/*
* @return True if a filter rules the scanner is over, done.
*/
public synchronized boolean isFilterDone() {
public synchronized boolean isFilterDone() throws IOException {
return this.filter != null && this.filter.filterAllRemaining();
}
@ -3633,7 +3636,7 @@ public class HRegion implements HeapSize { // , Writable{
}
}
private boolean filterRowKey(byte[] row, int offset, short length) {
private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
return filter != null
&& filter.filterRowKey(row, offset, length);
}

View File

@ -37,10 +37,10 @@ public interface RegionScanner extends InternalScanner {
public HRegionInfo getRegionInfo();
/**
* @return True if a filter indicates that this scanner will return no
* further rows.
* @return True if a filter indicates that this scanner will return no further rows.
* @throws IOException in case of I/O failure on a filter.
*/
public boolean isFilterDone();
public boolean isFilterDone() throws IOException;
/**
* Do a reseek to the required row. Should not be used to seek to a key which

View File

@ -472,7 +472,7 @@ public class ScanQueryMatcher {
return this.filter;
}
public KeyValue getNextKeyHint(KeyValue kv) {
public KeyValue getNextKeyHint(KeyValue kv) throws IOException {
if (filter == null) {
return null;
} else {

View File

@ -45,12 +45,12 @@ import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
@ -117,7 +117,7 @@ public class TestCoprocessorInterface extends HBaseTestCase {
}
@Override
public boolean isFilterDone() {
public boolean isFilterDone() throws IOException {
return delegate.isFilterDone();
}

View File

@ -19,8 +19,10 @@
package org.apache.hadoop.hbase.filter;
import java.io.DataInput;
import java.io.DataOutput;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -28,7 +30,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
@ -43,9 +51,6 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.base.Throwables;
@ -1667,7 +1672,7 @@ public class TestFilter {
public byte [] toByteArray() {return null;}
@Override
public boolean filterRow() {
public boolean filterRow() throws IOException {
ipcHandlerThread = Thread.currentThread();
try {
LOG.info("Handler thread " + ipcHandlerThread + " sleeping in filter...");

View File

@ -18,17 +18,16 @@
*/
package org.apache.hadoop.hbase.filter;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.hbase.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
/**
* Tests for the page filter
*/
@ -66,7 +65,7 @@ public class TestPageFilter {
testFiltersBeyondPageSize(f, ROW_LIMIT);
}
private void testFiltersBeyondPageSize(final Filter f, final int pageSize) {
private void testFiltersBeyondPageSize(final Filter f, final int pageSize) throws IOException {
int count = 0;
for (int i = 0; i < (pageSize * 2); i++) {
boolean filterOut = f.filterRow();