You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/03/19 23:47:56 UTC

svn commit: r1458554 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ hbase-server/src/main/java/org...

Author: stack
Date: Tue Mar 19 22:47:55 2013
New Revision: 1458554

URL: http://svn.apache.org/r1458554
Log:
HBASE-7481 Allow IOExceptions to be thrown from Filter methods

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Tue Mar 19 22:47:55 2013
@@ -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.
-   */
-  abstract public void reset();
+   * 
+   * 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() 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();
-
-  /**
-   * 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.
-   */
-  abstract public KeyValue getNextKeyHint(final KeyValue currentKV);
-
-  /**
-   * 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.
-   */
-  abstract public boolean isFamilyEssential(byte[] name);
+  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.
+   * 
+   * 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) 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.
+   * 
+   * 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) 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);
-}
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java Tue Mar 19 22:47:55 2013
@@ -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
    * @inheritDoc
    */
   @Override
-  public void reset() {
+  public void reset() throws IOException {
   }
 
   /**
@@ -54,7 +55,7 @@ public abstract class FilterBase extends
    * @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
    * @inheritDoc
    */
   @Override
-  public boolean filterAllRemaining() {
+  public boolean filterAllRemaining() throws IOException {
     return false;
   }
 
@@ -76,7 +77,7 @@ public abstract class FilterBase extends
    * @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
    * @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
    * @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
    * @inheritDoc
    */
   @Override
-  public boolean filterRow() {
+  public boolean filterRow() throws IOException {
     return false;
   }
 
@@ -129,7 +130,7 @@ public abstract class FilterBase extends
    *
    * @inheritDoc
    */
-  public KeyValue getNextKeyHint(KeyValue currentKV) {
+  public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
     return null;
   }
 
@@ -139,7 +140,7 @@ public abstract class FilterBase extends
    *
    * @inheritDoc
    */
-  public boolean isFamilyEssential(byte[] name) {
+  public boolean isFamilyEssential(byte[] name) throws IOException {
     return true;
   }
 
@@ -163,7 +164,7 @@ public abstract class FilterBase extends
   /**
    * 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];
   }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java Tue Mar 19 22:47:55 2013
@@ -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.
 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;

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java Tue Mar 19 22:47:55 2013
@@ -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.exception
 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 Filte
   /**
    * @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 Filte
   }
 
   @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 Filte
   }
 
   @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 Filte
   }
 
   @Override
-  public boolean isFamilyEssential(byte[] name) {
+  public boolean isFamilyEssential(byte[] name) throws IOException {
     return filter.isFamilyEssential(name);
-  };
+  }
 
   /**
    * @param other

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java Tue Mar 19 22:47:55 2013
@@ -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.exception
 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 FilterBa
     return filter;
   }
 
-  public void reset() {
+  @Override
+  public void reset() throws IOException {
     filter.reset();
     filterRow = false;
   }
@@ -71,14 +73,15 @@ public class SkipFilter extends FilterBa
     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 FilterBa
   /**
    * @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 FilterBa
     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 FilterBa
   public String toString() {
     return this.getClass().getSimpleName() + " " + this.filter.toString();
   }
-}
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java Tue Mar 19 22:47:55 2013
@@ -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.exception
 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 Fi
     return filter;
   }
 
-  public void reset() {
+  public void reset() throws IOException {
     this.filter.reset();
   }
 
@@ -59,33 +60,38 @@ public class WhileMatchFilter extends Fi
     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 Fi
   /**
    * @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 Fi
     return getFilter().areSerializedFieldsEqual(other.getFilter());
   }
 
-  public boolean isFamilyEssential(byte[] name) {
+  public boolean isFamilyEssential(byte[] name) throws IOException {
     return filter.isFamilyEssential(name);
   }
 

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Tue Mar 19 22:47:55 2013
@@ -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()));

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java Tue Mar 19 22:47:55 2013
@@ -35,7 +35,6 @@ import org.apache.hadoop.conf.Configurat
 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.HT
 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);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Mar 19 22:47:55 2013
@@ -3385,10 +3385,13 @@ public class HRegion implements HeapSize
     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
     /*
      * @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
       }
     }
 
-    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);
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScanner.java Tue Mar 19 22:47:55 2013
@@ -37,10 +37,10 @@ public interface RegionScanner extends I
   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

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java Tue Mar 19 22:47:55 2013
@@ -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 {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java Tue Mar 19 22:47:55 2013
@@ -45,12 +45,12 @@ import org.apache.hadoop.hbase.SmallTest
 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 ex
     }
 
     @Override
-    public boolean isFilterDone() {
+    public boolean isFilterDone() throws IOException {
       return delegate.isFilterDone();
     }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Tue Mar 19 22:47:55 2013
@@ -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...");

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java?rev=1458554&r1=1458553&r2=1458554&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java Tue Mar 19 22:47:55 2013
@@ -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();