You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2015/05/21 13:21:07 UTC

hbase git commit: HBASE-13641 Deperecate Filter#filterRowKey(byte[] buffer, int offset, int length) in favor of filterRowKey(Cell firstRowCell).

Repository: hbase
Updated Branches:
  refs/heads/master d69807663 -> 6c3fd3447


HBASE-13641 Deperecate Filter#filterRowKey(byte[] buffer, int offset, int length) in favor of filterRowKey(Cell firstRowCell).


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6c3fd344
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6c3fd344
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6c3fd344

Branch: refs/heads/master
Commit: 6c3fd34475fa956deb570c92d55a3cc08c71c5f0
Parents: d698076
Author: anoopsjohn <an...@gmail.com>
Authored: Thu May 21 16:50:42 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu May 21 16:50:42 2015 +0530

----------------------------------------------------------------------
 .../hbase/filter/ColumnCountGetFilter.java      |  7 +++++
 .../hbase/filter/ColumnPaginationFilter.java    | 11 +++++--
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |  7 +++++
 .../hadoop/hbase/filter/ColumnRangeFilter.java  |  7 +++++
 .../hadoop/hbase/filter/CompareFilter.java      |  8 ++++++
 .../org/apache/hadoop/hbase/filter/Filter.java  | 18 +++++++++++-
 .../apache/hadoop/hbase/filter/FilterBase.java  | 10 +++++++
 .../apache/hadoop/hbase/filter/FilterList.java  | 19 +++++++++++++
 .../hadoop/hbase/filter/FilterWrapper.java      |  6 ++++
 .../hadoop/hbase/filter/FirstKeyOnlyFilter.java |  7 +++++
 .../hadoop/hbase/filter/FuzzyRowFilter.java     |  7 +++++
 .../hbase/filter/InclusiveStopFilter.java       | 11 ++-----
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |  6 ++++
 .../hbase/filter/MultiRowRangeFilter.java       | 11 ++++---
 .../filter/MultipleColumnPrefixFilter.java      |  7 +++++
 .../apache/hadoop/hbase/filter/PageFilter.java  |  6 ++++
 .../hadoop/hbase/filter/PrefixFilter.java       | 12 ++++----
 .../hadoop/hbase/filter/RandomRowFilter.java    |  2 +-
 .../apache/hadoop/hbase/filter/RowFilter.java   |  5 ++--
 .../hbase/filter/SingleColumnValueFilter.java   |  9 ++++--
 .../apache/hadoop/hbase/filter/SkipFilter.java  |  6 ++++
 .../hadoop/hbase/filter/TimestampsFilter.java   |  7 +++++
 .../hadoop/hbase/filter/WhileMatchFilter.java   |  7 +++++
 .../apache/hadoop/hbase/mapreduce/Import.java   |  9 ++++--
 .../hadoop/hbase/regionserver/HRegion.java      |  7 ++---
 .../security/access/AccessControlFilter.java    |  6 ++++
 .../visibility/VisibilityController.java        |  6 ++++
 .../visibility/VisibilityLabelFilter.java       |  6 ++++
 .../apache/hadoop/hbase/filter/TestFilter.java  |  2 +-
 .../hadoop/hbase/filter/TestFilterList.java     | 30 ++++++++++----------
 .../hbase/filter/TestInclusiveStopFilter.java   |  9 +++---
 .../hbase/filter/TestMultiRowRangeFilter.java   | 11 +++----
 .../hadoop/hbase/filter/TestPrefixFilter.java   |  5 ++--
 .../hbase/filter/TestRandomRowFilter.java       |  4 +--
 34 files changed, 226 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
index 572de9f..fd65130 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
@@ -52,6 +53,12 @@ public class ColumnCountGetFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public boolean filterAllRemaining() {
     return this.count > this.limit;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
index 673ca6e..2b5f128 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
@@ -40,8 +41,8 @@ import com.google.protobuf.InvalidProtocolBufferException;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class ColumnPaginationFilter extends FilterBase
-{
+public class ColumnPaginationFilter extends FilterBase {
+
   private int limit = 0;
   private int offset = -1;
   private byte[] columnOffset = null;
@@ -105,6 +106,12 @@ public class ColumnPaginationFilter extends FilterBase
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell v)
   {
     if (columnOffset != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
index d2f058a..a89a1f3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
@@ -52,6 +53,12 @@ public class ColumnPrefixFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell kv) {
     if (this.prefix == null || kv.getQualifierArray() == null) {
       return ReturnCode.INCLUDE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
index d8ea094..c386ad6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter;
 
 import static org.apache.hadoop.hbase.util.Bytes.len;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
@@ -116,6 +117,12 @@ public class ColumnRangeFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell kv) {
     // TODO have a column compare method in Cell
     byte[] buffer = kv.getQualifierArray();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
index 319e123..f641524 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
@@ -19,8 +19,10 @@
 
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -97,6 +99,12 @@ public abstract class CompareFilter extends FilterBase {
     return comparator;
   }
 
+  @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
   protected boolean doCompare(final CompareOp compareOp,
       final ByteArrayComparable comparator, final byte [] data,
       final int offset, final int length) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
index 88bf842..a2f9015 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
  * <ul>
  *   <li> {@link #reset()} : reset the filter state before filtering a new row. </li>
  *   <li> {@link #filterAllRemaining()}: true means row scan is over; false means keep going. </li>
- *   <li> {@link #filterRowKey(byte[],int,int)}: true means drop this row; false means include.</li>
+ *   <li> {@link #filterRowKey(Cell)}: true means drop this row; false means include.</li>
  *   <li> {@link #filterKeyValue(Cell)}: decides whether to include or exclude this Cell.
  *        See {@link ReturnCode}. </li>
  *   <li> {@link #transformCell(Cell)}: if the Cell is included, let the filter transform the
@@ -78,10 +78,26 @@ public abstract class Filter {
    * @param length length of the row key
    * @return true, remove entire row, false, include the row (maybe).
    * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Instead use {@link #filterRowKey(Cell)}
    */
+  @Deprecated
   abstract public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException;
 
   /**
+   * Filters a row based on the row key. If this returns true, the entire row will be excluded. If
+   * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below.
+   *
+   * Concrete implementers can signal a failure condition in their code by throwing an
+   * {@link IOException}.
+   *
+   * @param firstRowCell The first cell coming in the new row
+   * @return true, remove entire row, false, include the row (maybe).
+   * @throws IOException in case an I/O or an filter specific failure needs to be signaled.
+   */
+  abstract public boolean filterRowKey(Cell firstRowCell) throws IOException;
+
+  /**
    * If this returns true, the scan will terminate.
    * 
    * Concrete implementers can signal a failure condition in their code by throwing an

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
index a04dd89..1bcd00a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
@@ -52,12 +52,22 @@ public abstract class FilterBase extends Filter {
    * never filters anything. (ie: returns false).
    *
    * @inheritDoc
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Instead use {@link #filterRowKey(Cell)}
    */
   @Override
+  @Deprecated
   public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
     return false;
   }
 
+  @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // TODO when cell is backed by DirectByteBuffer, we would need to copy row bytes to temp byte[]
+    // and call old method for BC.
+    return filterRowKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
+  }
+
   /**
    * Filters that never filter all remaining can inherit this implementation that
    * never stops the filter early.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index a7856cd..2f89251 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -198,6 +198,25 @@ final public class FilterList extends Filter {
   }
 
   @Override
+  public boolean filterRowKey(Cell firstRowCell) throws IOException {
+    boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
+    int listize = filters.size();
+    for (int i = 0; i < listize; i++) {
+      Filter filter = filters.get(i);
+      if (this.operator == Operator.MUST_PASS_ALL) {
+        if (filter.filterAllRemaining() || filter.filterRowKey(firstRowCell)) {
+          flag = true;
+        }
+      } else if (this.operator == Operator.MUST_PASS_ONE) {
+        if (!filter.filterAllRemaining() && !filter.filterRowKey(firstRowCell)) {
+          flag = false;
+        }
+      }
+    }
+    return flag;
+  }
+
+  @Override
   public boolean filterAllRemaining() throws IOException {
     int listize = filters.size();
     for (int i = 0; i < listize; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
index 5176115..ff0e43b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
@@ -102,10 +102,16 @@ final public class FilterWrapper extends Filter {
 
   @Override
   public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
+    // No call to this.
     return this.filter.filterRowKey(buffer, offset, length);
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    return this.filter.filterRowKey(cell);
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell v) throws IOException {
     return this.filter.filterKeyValue(v);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
index 77ed7d9..80a1deb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.hbase.Cell;
@@ -47,6 +48,12 @@ public class FirstKeyOnlyFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell v) {
     if(foundKV) return ReturnCode.NEXT_ROW;
     foundKV = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index bb5edf4..9b234de 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -83,6 +84,12 @@ public class FuzzyRowFilter extends FilterBase {
     this.fuzzyKeysData = fuzzyKeysData;
   }
 
+  @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
   // TODO: possible improvement: save which fuzzy row key to use when providing a hint
   @Override
   public ReturnCode filterKeyValue(Cell c) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
index cf2d153..b487e98 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
@@ -58,17 +58,10 @@ public class InclusiveStopFilter extends FilterBase {
     return ReturnCode.INCLUDE;
   }
 
-  public boolean filterRowKey(byte[] buffer, int offset, int length) {
-    if (buffer == null) {
-      //noinspection RedundantIfStatement
-      if (this.stopRowKey == null) {
-        return true; //filter...
-      }
-      return false;
-    }
+  public boolean filterRowKey(Cell firstRowCell) {
     // if stopRowKey is <= buffer, then true, filter row.
     int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
-      buffer, offset, length);
+        firstRowCell.getRowArray(), firstRowCell.getRowOffset(), firstRowCell.getRowLength());
 
     if(cmp < 0) {
       done = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index 2a2b525..2fd5aba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -50,6 +50,12 @@ public class KeyOnlyFilter extends FilterBase {
   public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public Cell transformCell(Cell cell) {
     return createKeyOnlyCell(cell);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
index 24c8135..e2f159b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
@@ -83,14 +83,17 @@ public class MultiRowRangeFilter extends FilterBase {
   }
 
   @Override
-  public boolean filterRowKey(byte[] buffer, int offset, int length) {
+  public boolean filterRowKey(Cell firstRowCell) {
     // If it is the first time of running, calculate the current range index for
     // the row key. If index is out of bound which happens when the start row
     // user sets is after the largest stop row of the ranges, stop the scan.
     // If row key is after the current range, find the next range and update index.
-    if (!initialized || !range.contains(buffer, offset, length)) {
+    int length = firstRowCell.getRowLength();
+    int offset = firstRowCell.getRowOffset();
+    if (!initialized
+        || !range.contains(firstRowCell.getRowArray(), offset, length)) {
       byte[] rowkey = new byte[length];
-      System.arraycopy(buffer, offset, rowkey, 0, length);
+      System.arraycopy(firstRowCell.getRowArray(), firstRowCell.getRowOffset(), rowkey, 0, length);
       index = getNextRangeIndex(rowkey);
       if (index >= rangeList.size()) {
         done = true;
@@ -115,7 +118,7 @@ public class MultiRowRangeFilter extends FilterBase {
         }
         initialized = true;
       } else {
-        if (range.contains(buffer, offset, length)) {
+        if (range.contains(firstRowCell.getRowArray(), offset, length)) {
           currentReturnCode = ReturnCode.INCLUDE;
         } else currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
index b7ec11a..79191a3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -64,6 +65,12 @@ public class MultipleColumnPrefixFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell kv) {
     if (sortedPrefixes.size() == 0 || kv.getQualifierArray() == null) {
       return ReturnCode.INCLUDE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
index 0dbd97b..7c68dd2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
@@ -61,6 +61,12 @@ public class PageFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell ignored) throws IOException {
     return ReturnCode.INCLUDE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
index 5b56748..7a031df 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
@@ -50,16 +50,16 @@ public class PrefixFilter extends FilterBase {
     return prefix;
   }
 
-  public boolean filterRowKey(byte[] buffer, int offset, int length) {
-    if (buffer == null || this.prefix == null)
-      return true;
-    if (length < prefix.length)
+  public boolean filterRowKey(Cell firstRowCell) {
+    if (firstRowCell == null || this.prefix == null)
       return true;
+    int length = firstRowCell.getRowLength();
+    if (length < prefix.length) return true;
     // if they are equal, return false => pass row
     // else return true, filter row
     // if we are passed the prefix, set flag
-    int cmp = Bytes.compareTo(buffer, offset, this.prefix.length, this.prefix, 0,
-        this.prefix.length);
+    int cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),
+        this.prefix.length, this.prefix, 0, this.prefix.length);
     if ((!isReversed() && cmp > 0) || (isReversed() && cmp < 0)) {
       passedPrefix = true;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
index 2a25b32..decdc78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
@@ -89,7 +89,7 @@ public class RandomRowFilter extends FilterBase {
   }
 
   @Override
-  public boolean filterRowKey(byte[] buffer, int offset, int length) {
+  public boolean filterRowKey(Cell firstRowCell) {
     if (chance < 0) {
       // with a zero chance, the rows is always excluded
       filterOutRow = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
index cb4337e..e508b8b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
@@ -74,8 +74,9 @@ public class RowFilter extends CompareFilter {
   }
 
   @Override
-  public boolean filterRowKey(byte[] data, int offset, int length) {
-    if(doCompare(this.compareOp, this.comparator, data, offset, length)) {
+  public boolean filterRowKey(Cell firstRowCell) {
+    if (doCompare(this.compareOp, this.comparator, firstRowCell.getRowArray(),
+        firstRowCell.getRowOffset(), firstRowCell.getRowLength())) {
       this.filterOutRow = true;
     }
     return this.filterOutRow;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
index 2fcbf4f..09fc90e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
@@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.filter;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -71,7 +69,6 @@ import com.google.protobuf.InvalidProtocolBufferException;
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class SingleColumnValueFilter extends FilterBase {
-  private static final Log LOG = LogFactory.getLog(SingleColumnValueFilter.class);
 
   protected byte [] columnFamily;
   protected byte [] columnQualifier;
@@ -169,6 +166,12 @@ public class SingleColumnValueFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell c) {
     // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue()));
     if (this.matchedColumn) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
index ce8e511..71ea3c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
@@ -74,6 +74,12 @@ public class SkipFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell v) throws IOException {
     ReturnCode c = filter.filterKeyValue(v);
     changeFR(c != ReturnCode.INCLUDE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
index 32a3d73..0ce0447 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.TreeSet;
@@ -89,6 +90,12 @@ public class TimestampsFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell v) {
     if (this.timestamps.contains(v.getTimestamp())) {
       return ReturnCode.INCLUDE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
index 31d4f77..e75ca49 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
@@ -73,6 +73,13 @@ public class WhileMatchFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    boolean value = filter.filterRowKey(cell);
+    changeFAR(value);
+    return value;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell v) throws IOException {
     ReturnCode c = filter.filterKeyValue(v);
     changeFAR(c != ReturnCode.INCLUDE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index 9c5b5af..e2f4ce0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -108,7 +107,9 @@ public class Import extends Configured implements Tool {
           LOG.trace("Considering the row."
               + Bytes.toString(row.get(), row.getOffset(), row.getLength()));
         }
-        if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
+        if (filter == null
+            || !filter.filterRowKey(KeyValueUtil.createFirstOnRow(row.get(), row.getOffset(),
+                (short) row.getLength()))) {
           for (Cell kv : value.rawCells()) {
             kv = filterKv(filter, kv);
             // skip if we filtered it out
@@ -163,7 +164,9 @@ public class Import extends Configured implements Tool {
         LOG.trace("Considering the row."
             + Bytes.toString(key.get(), key.getOffset(), key.getLength()));
       }
-      if (filter == null || !filter.filterRowKey(key.get(), key.getOffset(), key.getLength())) {
+      if (filter == null
+          || !filter.filterRowKey(KeyValueUtil.createFirstOnRow(key.get(), key.getOffset(),
+              (short) key.getLength()))) {
         processKV(key, result, context, put, delete);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 3bfb1a6..576a23e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -5555,7 +5555,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
           // Check if rowkey filter wants to exclude this row. If so, loop to next.
           // Technically, if we hit limits before on this row, we don't need this call.
-          if (filterRowKey(currentRow, offset, length)) {
+          if (filterRowKey(current)) {
             boolean moreRows = nextRow(current);
             if (!moreRows) {
               return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
@@ -5707,9 +5707,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           && filter.filterRow();
     }
 
-    private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
-      return filter != null
-          && filter.filterRowKey(row, offset, length);
+    private boolean filterRowKey(Cell current) throws IOException {
+      return filter != null && filter.filterRowKey(current);
     }
 
     protected boolean nextRow(Cell curRowCell) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
index 48a982e..293e60e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
@@ -87,6 +87,12 @@ class AccessControlFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell cell) {
     if (isSystemTable) {
       return ReturnCode.INCLUDE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index e30344c..871678f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -1072,6 +1072,12 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     }
 
     @Override
+    public boolean filterRowKey(Cell cell) throws IOException {
+      // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+      return false;
+    }
+
+    @Override
     public ReturnCode filterKeyValue(Cell cell) throws IOException {
       List<Tag> putVisTags = new ArrayList<Tag>();
       Byte putCellVisTagsFormat = VisibilityUtils.extractVisibilityTags(cell, putVisTags);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java
index eb8abbe..18bd9f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java
@@ -50,6 +50,12 @@ class VisibilityLabelFilter extends FilterBase {
   }
 
   @Override
+  public boolean filterRowKey(Cell cell) throws IOException {
+    // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    return false;
+  }
+
+  @Override
   public ReturnCode filterKeyValue(Cell cell) throws IOException {
     if (curFamily.getBytes() == null
         || (Bytes.compareTo(curFamily.getBytes(), curFamily.getOffset(), curFamily.getLength(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index 02c4c9d..3601b01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -628,7 +628,7 @@ public class TestFilter {
   /**
    * Tests the the {@link WhileMatchFilter} works in combination with a
    * {@link Filter} that uses the
-   * {@link Filter#filterRowKey(byte[], int, int)} method.
+   * {@link Filter#filterRowKey(Cell)} method.
    *
    * See HBASE-2258.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index 0e01484..8854efe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -115,7 +115,7 @@ public class TestFilterList {
     /* Will pass both */
     byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
     for (int i = 0; i < MAX_PAGES - 1; i++) {
-      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
         Bytes.toBytes(i));
       assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -124,7 +124,7 @@ public class TestFilterList {
 
     /* Only pass PageFilter */
     rowkey = Bytes.toBytes("z");
-    assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+    assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
         Bytes.toBytes(0));
     assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -132,7 +132,7 @@ public class TestFilterList {
 
     /* reach MAX_PAGES already, should filter any rows */
     rowkey = Bytes.toBytes("yyy");
-    assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+    assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
         Bytes.toBytes(0));
     assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -140,7 +140,7 @@ public class TestFilterList {
 
     /* We should filter any row */
     rowkey = Bytes.toBytes("z");
-    assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+    assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     assertTrue(filterMPONE.filterAllRemaining());
   }
 
@@ -179,14 +179,14 @@ public class TestFilterList {
     assertFalse(filterMPALL.filterAllRemaining());
     byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
     for (int i = 0; i < MAX_PAGES - 1; i++) {
-      assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+      assertFalse(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
         Bytes.toBytes(i));
       assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
     }
     filterMPALL.reset();
     rowkey = Bytes.toBytes("z");
-    assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+    assertTrue(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
     // Should fail here; row should be filtered out.
     KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
     assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
@@ -229,7 +229,7 @@ public class TestFilterList {
     /* We should be able to fill MAX_PAGES without incrementing page counter */
     byte [] rowkey = Bytes.toBytes("yyyyyyyy");
     for (int i = 0; i < MAX_PAGES; i++) {
-      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
           Bytes.toBytes(i));
         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -239,7 +239,7 @@ public class TestFilterList {
     /* Now let's fill the page filter */
     rowkey = Bytes.toBytes("xxxxxxx");
     for (int i = 0; i < MAX_PAGES; i++) {
-      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
           Bytes.toBytes(i));
         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -249,7 +249,7 @@ public class TestFilterList {
     /* We should still be able to include even though page filter is at max */
     rowkey = Bytes.toBytes("yyy");
     for (int i = 0; i < MAX_PAGES; i++) {
-      assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
           Bytes.toBytes(i));
         assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
@@ -269,23 +269,23 @@ public class TestFilterList {
   
     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
     flist.addFilter(new PrefixFilter(r1));
-    flist.filterRowKey(r1, 0, r1.length);
+    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
     assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
     assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
 
     flist.reset();
-    flist.filterRowKey(r2, 0, r2.length);
+    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
     assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
   
     flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
     flist.addFilter(new AlwaysNextColFilter());
     flist.addFilter(new PrefixFilter(r1));
-    flist.filterRowKey(r1, 0, r1.length);
+    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
     assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
     assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
 
     flist.reset();
-    flist.filterRowKey(r2, 0, r2.length);
+    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
     assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
   }
 
@@ -302,12 +302,12 @@ public class TestFilterList {
     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
     flist.addFilter(new AlwaysNextColFilter());
     flist.addFilter(new InclusiveStopFilter(r1));
-    flist.filterRowKey(r1, 0, r1.length);
+    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
     assertEquals(flist.filterKeyValue(new KeyValue(r1,r1,r1)), ReturnCode.INCLUDE);
     assertEquals(flist.filterKeyValue(new KeyValue(r11,r11,r11)), ReturnCode.INCLUDE);
 
     flist.reset();
-    flist.filterRowKey(r2, 0, r2.length);
+    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
     assertEquals(flist.filterKeyValue(new KeyValue(r2,r2,r2)), ReturnCode.SKIP);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
index e527ca8..fe4e5d1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -71,16 +72,14 @@ public class TestInclusiveStopFilter {
 
   private void stopRowTests(Filter filter) throws Exception {
     assertFalse("Filtering on " + Bytes.toString(GOOD_ROW),
-      filter.filterRowKey(GOOD_ROW, 0, GOOD_ROW.length));
+      filter.filterRowKey(KeyValueUtil.createFirstOnRow(GOOD_ROW)));
     assertFalse("Filtering on " + Bytes.toString(STOP_ROW),
-      filter.filterRowKey(STOP_ROW, 0, STOP_ROW.length));
+      filter.filterRowKey(KeyValueUtil.createFirstOnRow(STOP_ROW)));
     assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
-      filter.filterRowKey(PAST_STOP_ROW, 0, PAST_STOP_ROW.length));
+      filter.filterRowKey(KeyValueUtil.createFirstOnRow(PAST_STOP_ROW)));
 
     assertTrue("FilterAllRemaining", filter.filterAllRemaining());
     assertFalse("FilterNotNull", filter.filterRow());
-
-    assertFalse("Filter a null", filter.filterRowKey(null, 0, 0));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
index 0c596aa..9f62335 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -75,15 +76,15 @@ public class TestMultiRowRangeFilter {
             new MultiRowRangeFilter.RowRange(Bytes.toBytes("b"), true, Bytes.toBytes("c"), true),
             new MultiRowRangeFilter.RowRange(Bytes.toBytes("d"), true, Bytes.toBytes("e"), true)
     ));
-    filter.filterRowKey(Bytes.toBytes("a"), 0, 1);
+    filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("a")));
     assertEquals(Filter.ReturnCode.SEEK_NEXT_USING_HINT, filter.filterKeyValue(null));
-    filter.filterRowKey(Bytes.toBytes("b"), 0, 1);
+    filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("b")));
     assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
-    filter.filterRowKey(Bytes.toBytes("c"), 0, 1);
+    filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("c")));
     assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
-    filter.filterRowKey(Bytes.toBytes("d"), 0, 1);
+    filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("d")));
     assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
-    filter.filterRowKey(Bytes.toBytes("e"), 0, 1);
+    filter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("e")));
     assertEquals(Filter.ReturnCode.INCLUDE, filter.filterKeyValue(null));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
index 02a55ba..754bdb6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.filter;
 
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -72,13 +73,13 @@ public class TestPrefixFilter {
     for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
       byte [] t = createRow(c);
       assertFalse("Failed with character " + c,
-        filter.filterRowKey(t, 0, t.length));
+        filter.filterRowKey(KeyValueUtil.createFirstOnRow(t)));
       assertFalse(filter.filterAllRemaining());
     }
     String yahooSite = "com.yahoo.www";
     byte [] yahooSiteBytes = Bytes.toBytes(yahooSite);
     assertTrue("Failed with character " +
-      yahooSite, filter.filterRowKey(yahooSiteBytes, 0, yahooSiteBytes.length));
+      yahooSite, filter.filterRowKey(KeyValueUtil.createFirstOnRow(yahooSiteBytes)));
     assertEquals(filter.filterAllRemaining(), lastFilterAllRemaining);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6c3fd344/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
index 8effca5..ca4fa06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.filter;
 
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -47,8 +48,7 @@ public class TestRandomRowFilter {
     int included = 0;
     int max = 1000000;
     for (int i = 0; i < max; i++) {
-      if (!quarterChanceFilter.filterRowKey(Bytes.toBytes("row"), 0, Bytes
-          .toBytes("row").length)) {
+      if (!quarterChanceFilter.filterRowKey(KeyValueUtil.createFirstOnRow(Bytes.toBytes("row")))) {
         included++;
       }
     }