Repository: hbase
Updated Branches:
  refs/heads/master c694ec11d -> 995a5a6c6


HBASE-10930 Change Filters and GetClosestRowBeforeTracker to work with
Cells (Ram)


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

Branch: refs/heads/master
Commit: 995a5a6c6838164f5baa5ec96f4395edd1c50d56
Parents: c694ec1
Author: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Authored: Tue Jul 15 22:07:00 2014 +0530
Committer: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Committed: Tue Jul 15 22:07:00 2014 +0530

----------------------------------------------------------------------
 .../hbase/filter/ColumnPaginationFilter.java    |  8 ++---
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |  8 ++---
 .../hadoop/hbase/filter/ColumnRangeFilter.java  | 14 ++++----
 .../hbase/filter/DependentColumnFilter.java     | 12 +++----
 .../org/apache/hadoop/hbase/filter/Filter.java  |  2 +-
 .../apache/hadoop/hbase/filter/FilterBase.java  |  7 ++--
 .../apache/hadoop/hbase/filter/FilterList.java  | 37 ++++++++++---------
 .../hadoop/hbase/filter/FilterWrapper.java      |  6 ++--
 .../hadoop/hbase/filter/FuzzyRowFilter.java     | 38 ++++++++------------
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |  3 +-
 .../filter/MultipleColumnPrefixFilter.java      | 17 ++++-----
 .../filter/SingleColumnValueExcludeFilter.java  |  5 +--
 .../hbase/filter/SingleColumnValueFilter.java   | 11 ++----
 .../org/apache/hadoop/hbase/CellComparator.java | 10 ++++++
 .../GetClosestRowBeforeTracker.java             | 25 +++++++------
 .../hadoop/hbase/regionserver/HRegion.java      |  2 +-
 .../visibility/VisibilityController.java        | 14 ++++----
 .../hadoop/hbase/filter/TestFilterList.java     | 19 ++++------
 18 files changed, 111 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 09a1e07..b3060bf 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
@@ -20,13 +20,13 @@ package org.apache.hadoop.hbase.filter;
 
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
@@ -144,10 +144,10 @@ public class ColumnPaginationFilter extends FilterBase
   }
 
   @Override
-  public Cell getNextCellHint(Cell kv) {
+  public Cell getNextCellHint(Cell cell) {
     return KeyValueUtil.createFirstOnRow(
-        kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), 
kv.getFamilyArray(),
-        kv.getFamilyOffset(), kv.getFamilyLength(), columnOffset, 0, 
columnOffset.length);
+        cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), 
cell.getFamilyArray(),
+        cell.getFamilyOffset(), cell.getFamilyLength(), columnOffset, 0, 
columnOffset.length);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 93d3d02..884115b 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
@@ -21,13 +21,13 @@ package org.apache.hadoop.hbase.filter;
 
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
@@ -130,10 +130,10 @@ public class ColumnPrefixFilter extends FilterBase {
   }
 
   @Override
-  public Cell getNextCellHint(Cell kv) {
+  public Cell getNextCellHint(Cell cell) {
     return KeyValueUtil.createFirstOnRow(
-        kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), 
kv.getFamilyArray(),
-        kv.getFamilyOffset(), kv.getFamilyLength(), prefix, 0, prefix.length);
+        cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), 
cell.getFamilyArray(),
+        cell.getFamilyOffset(), cell.getFamilyLength(), prefix, 0, 
prefix.length);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 7c2c2fe..596db14 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,9 +21,7 @@ package org.apache.hadoop.hbase.filter;
 
 import static org.apache.hadoop.hbase.util.Bytes.len;
 
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.util.ByteStringer;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -31,9 +29,11 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used for selecting only those keys with columns that are
@@ -216,9 +216,9 @@ public class ColumnRangeFilter extends FilterBase {
   }
 
   @Override
-  public Cell getNextCellHint(Cell kv) {
-    return KeyValueUtil.createFirstOnRow(kv.getRowArray(), kv.getRowOffset(), 
kv
-        .getRowLength(), kv.getFamilyArray(), kv.getFamilyOffset(), kv
+  public Cell getNextCellHint(Cell cell) {
+    return KeyValueUtil.createFirstOnRow(cell.getRowArray(), 
cell.getRowOffset(), cell
+        .getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell
         .getFamilyLength(), this.minColumn, 0, len(this.minColumn));
 
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
index d805cc7..c063fe4 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
@@ -30,8 +30,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -137,20 +135,18 @@ public class DependentColumnFilter extends CompareFilter {
 
   @Override
   public ReturnCode filterKeyValue(Cell c) {
-    // TODO make matching Column a cell method or CellUtil method.
-    KeyValue v = KeyValueUtil.ensureKeyValue(c);
     // Check if the column and qualifier match
-       if (!CellUtil.matchingColumn(v, this.columnFamily, 
this.columnQualifier)) {
+       if (!CellUtil.matchingColumn(c, this.columnFamily, 
this.columnQualifier)) {
         // include non-matches for the time being, they'll be discarded 
afterwards
         return ReturnCode.INCLUDE;
        }
     // If it doesn't pass the op, skip it
     if (comparator != null
-        && doCompare(compareOp, comparator, v.getValueArray(), 
v.getValueOffset(),
-            v.getValueLength()))
+        && doCompare(compareOp, comparator, c.getValueArray(), 
c.getValueOffset(),
+            c.getValueLength()))
       return ReturnCode.SKIP;
        
-    stampSet.add(v.getTimestamp());
+    stampSet.add(c.getTimestamp());
     if(dropDependentColumn) {
        return ReturnCode.SKIP;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 5e646cd..0c63c98 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
@@ -223,7 +223,7 @@ public abstract class Filter {
    *         seek to next.
    * @throws IOException in case an I/O or an filter specific failure needs to 
be signaled.
    */
-  abstract public Cell getNextCellHint(final Cell currentKV) throws 
IOException;
+  abstract public Cell getNextCellHint(final Cell currentCell) throws 
IOException;
 
   /**
    * Check that given column family is essential for filter to check row. Most 
filters always return

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 4988dc8..0418020 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
@@ -24,7 +24,6 @@ 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.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -143,10 +142,8 @@ public abstract class FilterBase extends Filter {
    *
    * @inheritDoc
    */
-  public Cell getNextCellHint(Cell currentKV) throws IOException {
-    // Old filters based off of this class will override KeyValue 
getNextKeyHint(KeyValue).
-    // Thus to maintain compatibility we need to call the old version.
-    return getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV));
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
+    return null;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 41c09ce..019ede7 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
@@ -26,6 +26,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -70,7 +71,7 @@ final public class FilterList extends Filter {
   private Filter seekHintFilter = null;
 
   /** Reference Cell used by {@link #transformCell(Cell)} for validation 
purpose. */
-  private Cell referenceKV = null;
+  private Cell referenceCell = null;
 
   /**
    * When filtering a given Cell in {@link #filterKeyValue(Cell)},
@@ -79,7 +80,7 @@ final public class FilterList extends Filter {
    * Individual filters transformation are applied only when the filter 
includes the Cell.
    * Transformations are composed in the order specified by {@link #filters}.
    */
-  private Cell transformedKV = null;
+  private Cell transformedCell = null;
 
   /**
    * Constructor that takes a set of {@link Filter}s. The default operator
@@ -211,8 +212,12 @@ final public class FilterList extends Filter {
   }
 
   @Override
-  public Cell transformCell(Cell v) throws IOException {
-    return transform(KeyValueUtil.ensureKeyValue(v));
+  public Cell transformCell(Cell c) throws IOException {
+    if (!CellComparator.equals(c, referenceCell)) {
+      throw new IllegalStateException("Reference Cell: " + this.referenceCell 
+ " does not match: "
+          + c);
+    }
+    return this.transformedCell;
   }
 
   /**
@@ -226,22 +231,22 @@ final public class FilterList extends Filter {
   @Override
   public KeyValue transform(KeyValue v) throws IOException {
     // transform() is expected to follow an inclusive filterKeyValue() 
immediately:
-    if (!v.equals(this.referenceKV)) {
+    if (!v.equals(this.referenceCell)) {
       throw new IllegalStateException(
-          "Reference Cell: " + this.referenceKV + " does not match: " + v);
+          "Reference Cell: " + this.referenceCell + " does not match: " + v);
      }
-    return KeyValueUtil.ensureKeyValue(this.transformedKV);
+    return KeyValueUtil.ensureKeyValue(this.transformedCell);
   }
 
   
   @Override
   
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
     justification="Intentional")
-  public ReturnCode filterKeyValue(Cell v) throws IOException {
-    this.referenceKV = v;
+  public ReturnCode filterKeyValue(Cell c) throws IOException {
+    this.referenceCell = c;
 
     // Accumulates successive transformation of every filter that includes the 
Cell:
-    Cell transformed = v;
+    Cell transformed = c;
 
     ReturnCode rc = operator == Operator.MUST_PASS_ONE?
         ReturnCode.SKIP: ReturnCode.INCLUDE;
@@ -250,7 +255,7 @@ final public class FilterList extends Filter {
         if (filter.filterAllRemaining()) {
           return ReturnCode.NEXT_ROW;
         }
-        ReturnCode code = filter.filterKeyValue(v);
+        ReturnCode code = filter.filterKeyValue(c);
         switch (code) {
         // Override INCLUDE and continue to evaluate.
         case INCLUDE_AND_NEXT_COL:
@@ -269,7 +274,7 @@ final public class FilterList extends Filter {
           continue;
         }
 
-        switch (filter.filterKeyValue(v)) {
+        switch (filter.filterKeyValue(c)) {
         case INCLUDE:
           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
             rc = ReturnCode.INCLUDE;
@@ -296,7 +301,7 @@ final public class FilterList extends Filter {
     }
 
     // Save the transformed Cell for transform():
-    this.transformedKV = transformed;
+    this.transformedCell = transformed;
 
     return rc;
   }
@@ -401,16 +406,16 @@ final public class FilterList extends Filter {
   }
 
   @Override
-  public Cell getNextCellHint(Cell currentKV) throws IOException {
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
     Cell keyHint = null;
     if (operator == Operator.MUST_PASS_ALL) {
-      keyHint = seekHintFilter.getNextCellHint(currentKV);
+      keyHint = seekHintFilter.getNextCellHint(currentCell);
       return keyHint;
     }
 
     // If any condition can pass, we need to keep the min hint
     for (Filter filter : filters) {
-      Cell curKeyHint = filter.getNextCellHint(currentKV);
+      Cell curKeyHint = filter.getNextCellHint(currentCell);
       if (curKeyHint == null) {
         // If we ever don't have a hint and this is must-pass-one, then no hint
         return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 51b2a66..57e3e41 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
@@ -20,11 +20,9 @@
 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.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -112,10 +110,10 @@ final public class FilterWrapper extends Filter {
    * Old filter wrapper descendants will implement KV getNextKeyHint(KV) so we 
should call it.
    */
   @Override
-  public Cell getNextCellHint(Cell currentKV) throws IOException {
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
     // Old filters based off of this class will override KeyValue 
getNextKeyHint(KeyValue).
     // Thus to maintain compatibility we need to call the old version.
-    return this.getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV));
+    return this.getNextKeyHint(KeyValueUtil.ensureKeyValue(currentCell));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 02e5f71..0cfde8f 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,11 +17,13 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+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.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -30,9 +32,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Filters data based on fuzzy row key. Performs fast-forwards during scanning.
@@ -74,16 +74,12 @@ public class FuzzyRowFilter extends FilterBase {
 
   // TODO: possible improvement: save which fuzzy row key to use when 
providing a hint
   @Override
-  public ReturnCode filterKeyValue(Cell kv) {
-    // TODO add getRow() equivalent to Cell or change satisfies to take 
b[],o,l style args.
-    KeyValue v = KeyValueUtil.ensureKeyValue(kv);
-
-    byte[] rowKey = v.getRow();
+  public ReturnCode filterKeyValue(Cell c) {
     // assigning "worst" result first and looking for better options
     SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
     for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
-      SatisfiesCode satisfiesCode =
-              satisfies(rowKey, fuzzyData.getFirst(), fuzzyData.getSecond());
+      SatisfiesCode satisfiesCode = satisfies(c.getRowArray(), 
c.getRowOffset(),
+          c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond());
       if (satisfiesCode == SatisfiesCode.YES) {
         return ReturnCode.INCLUDE;
       }
@@ -103,16 +99,13 @@ public class FuzzyRowFilter extends FilterBase {
   }
 
   @Override
-  public Cell getNextCellHint(Cell currentKV) {
-    // TODO make matching Column a cell method or CellUtil method.
-    KeyValue v = KeyValueUtil.ensureKeyValue(currentKV);
-
-    byte[] rowKey = v.getRow();
+  public Cell getNextCellHint(Cell currentCell) {
     byte[] nextRowKey = null;
     // Searching for the "smallest" row key that satisfies at least one fuzzy 
row key
     for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
-      byte[] nextRowKeyCandidate = getNextForFuzzyRule(rowKey,
-              fuzzyData.getFirst(), fuzzyData.getSecond());
+      byte[] nextRowKeyCandidate = 
getNextForFuzzyRule(currentCell.getRowArray(),
+          currentCell.getRowOffset(), currentCell.getRowLength(), 
fuzzyData.getFirst(),
+          fuzzyData.getSecond());
       if (nextRowKeyCandidate == null) {
         continue;
       }
@@ -124,10 +117,9 @@ public class FuzzyRowFilter extends FilterBase {
     if (nextRowKey == null) {
       // SHOULD NEVER happen
       // TODO: is there a better way than throw exception? (stop the scanner?)
-      throw new IllegalStateException("No next row key that satisfies fuzzy 
exists when" +
-                                         " getNextKeyHint() is invoked." +
-                                         " Filter: " + this.toString() +
-                                         " currentKV: " + 
currentKV.toString());
+      throw new IllegalStateException("No next row key that satisfies fuzzy 
exists when"
+          + " getNextKeyHint() is invoked." + " Filter: " + this.toString() + 
" currentKV: "
+          + KeyValueUtil.ensureKeyValue(currentCell).toString());
     }
 
     return KeyValueUtil.createFirstOnRow(nextRowKey);

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 bb36004..1c46229 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
 import com.google.common.base.Preconditions;
@@ -54,6 +53,8 @@ public class KeyOnlyFilter extends FilterBase {
     // TODO Move to KeyValueUtil
 
     // TODO make matching Column a cell method or CellUtil method.
+    // Even if we want to make use of KeyValue.KeyOnlyKeyValue we need to 
convert
+    // the cell to KV so that we can make use of kv.getKey() to form the key 
part
     KeyValue v = KeyValueUtil.ensureKeyValue(kv);
 
     return v.createKeyOnly(this.lenAsVal);

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 1e79ff4..65fe6fd 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,7 +17,11 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.TreeSet;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
@@ -27,10 +31,7 @@ import 
org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.TreeSet;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used for selecting only those keys with columns that matches
@@ -154,10 +155,10 @@ public class MultipleColumnPrefixFilter extends 
FilterBase {
   }
 
   @Override
-  public Cell getNextCellHint(Cell kv) {
+  public Cell getNextCellHint(Cell cell) {
     return KeyValueUtil.createFirstOnRow(
-      kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), 
kv.getFamilyArray(),
-      kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
+      cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), 
cell.getFamilyArray(),
+      cell.getFamilyOffset(), cell.getFamilyLength(), hint, 0, hint.length);
   }
 
   public TreeSet<byte []> createTreeSet() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
index 1058756..76673bb 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
@@ -28,8 +28,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -108,10 +106,9 @@ public class SingleColumnValueExcludeFilter extends 
SingleColumnValueFilter {
   public void filterRowCells(List<Cell> kvs) {
     Iterator<? extends Cell> it = kvs.iterator();
     while (it.hasNext()) {
-      KeyValue kv = KeyValueUtil.ensureKeyValue(it.next());
       // If the current column is actually the tested column,
       // we will skip it instead.
-      if (CellUtil.matchingColumn(kv, this.columnFamily, 
this.columnQualifier)) {
+      if (CellUtil.matchingColumn(it.next(), this.columnFamily, 
this.columnQualifier)) {
         it.remove();
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 e927a46..b3ddb35 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
@@ -29,8 +29,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -172,9 +170,6 @@ public class SingleColumnValueFilter extends FilterBase {
 
   @Override
   public ReturnCode filterKeyValue(Cell c) {
-    // TODO get rid of this.
-    KeyValue keyValue = KeyValueUtil.ensureKeyValue(c);
-    
     // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + 
Bytes.toString(keyValue.getValue()));
     if (this.matchedColumn) {
       // We already found and matched the single column, all keys now pass
@@ -183,12 +178,12 @@ public class SingleColumnValueFilter extends FilterBase {
       // We found but did not match the single column, skip to next row
       return ReturnCode.NEXT_ROW;
     }
-    if (!CellUtil.matchingColumn(keyValue, this.columnFamily, 
this.columnQualifier)) {
+    if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) {
       return ReturnCode.INCLUDE;
     }
     foundColumn = true;
-    if (filterColumnValue(keyValue.getValueArray(),
-        keyValue.getValueOffset(), keyValue.getValueLength())) {
+    if (filterColumnValue(c.getValueArray(),
+        c.getValueOffset(), c.getValueLength())) {
       return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
     }
     this.matchedColumn = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index edf1e52..e9c83cc 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -353,4 +353,14 @@ public class CellComparator implements Comparator<Cell>, 
Serializable{
     return 0;
   }
 
+  /**
+   * Counter part for the KeyValue.RowOnlyComparator
+   */
+  public static class RowComparator extends CellComparator {
+    @Override
+    public int compare(Cell a, Cell b) {
+      return compareRows(a, b);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
index 18ffd96..ec676fa 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java
@@ -25,11 +25,11 @@ import java.util.TreeSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -52,7 +52,7 @@ class GetClosestRowBeforeTracker {
   private final int tablenamePlusDelimiterLength;
 
   // Deletes keyed by row.  Comparator compares on row portion of KeyValue 
only.
-  private final NavigableMap<KeyValue, NavigableSet<KeyValue>> deletes;
+  private final NavigableMap<Cell, NavigableSet<Cell>> deletes;
 
   /**
    * @param c
@@ -77,8 +77,7 @@ class GetClosestRowBeforeTracker {
     this.tablenamePlusDelimiterLength = metaregion? l + 1: -1;
     this.oldestts = System.currentTimeMillis() - ttl;
     this.kvcomparator = c;
-    KeyValue.RowOnlyComparator rc = new 
KeyValue.RowOnlyComparator(this.kvcomparator);
-    this.deletes = new TreeMap<KeyValue, NavigableSet<KeyValue>>(rc);
+    this.deletes = new TreeMap<Cell, NavigableSet<Cell>>(new 
CellComparator.RowComparator());
   }
 
   /**
@@ -94,12 +93,12 @@ class GetClosestRowBeforeTracker {
    * @param kv
    */
   private void addDelete(final Cell kv) {
-    NavigableSet<KeyValue> rowdeletes = this.deletes.get(kv);
+    NavigableSet<Cell> rowdeletes = this.deletes.get(kv);
     if (rowdeletes == null) {
-      rowdeletes = new TreeSet<KeyValue>(this.kvcomparator);
-      this.deletes.put(KeyValueUtil.ensureKeyValue(kv), rowdeletes);
+      rowdeletes = new TreeSet<Cell>(this.kvcomparator);
+      this.deletes.put(kv, rowdeletes);
     }
-    rowdeletes.add(KeyValueUtil.ensureKeyValue(kv));
+    rowdeletes.add(kv);
   }
 
   /*
@@ -128,7 +127,7 @@ class GetClosestRowBeforeTracker {
    */
   private boolean isDeleted(final Cell kv) {
     if (this.deletes.isEmpty()) return false;
-    NavigableSet<KeyValue> rowdeletes = this.deletes.get(kv);
+    NavigableSet<Cell> rowdeletes = this.deletes.get(kv);
     if (rowdeletes == null || rowdeletes.isEmpty()) return false;
     return isDeleted(kv, rowdeletes);
   }
@@ -140,9 +139,9 @@ class GetClosestRowBeforeTracker {
    * @param ds
    * @return True is the specified KeyValue is deleted, false if not
    */
-  public boolean isDeleted(final Cell kv, final NavigableSet<KeyValue> ds) {
+  public boolean isDeleted(final Cell kv, final NavigableSet<Cell> ds) {
     if (deletes == null || deletes.isEmpty()) return false;
-    for (KeyValue d: ds) {
+    for (Cell d: ds) {
       long kvts = kv.getTimestamp();
       long dts = d.getTimestamp();
       if (CellUtil.isDeleteFamily(d)) {
@@ -164,7 +163,7 @@ class GetClosestRowBeforeTracker {
       if (kvts > dts) return false;
 
       // Check Type
-      switch (KeyValue.Type.codeToType(d.getType())) {
+      switch (KeyValue.Type.codeToType(d.getTypeByte())) {
         case Delete: return kvts == dts;
         case DeleteColumn: return true;
         default: continue;
@@ -198,7 +197,7 @@ class GetClosestRowBeforeTracker {
    * @return True if we added a candidate
    */
   boolean handle(final Cell kv) {
-    if (KeyValueUtil.ensureKeyValue(kv).isDelete()) {
+    if (CellUtil.isDelete(kv)) {
       handleDeletes(kv);
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 6bce6a8..80ca848 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
@@ -3899,7 +3899,7 @@ public class HRegion implements HeapSize { // , Writable{
       if (region != null && region.metricsRegion != null) {
         long totalSize = 0;
         for(Cell c:outResults) {
-          // TODO clean up
+          // TODO clean up.  Find way to remove this ensureKeyValue
           KeyValue kv = KeyValueUtil.ensureKeyValue(c);
           totalSize += kv.getLength();
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 795ec69..6920f7b 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
@@ -1249,14 +1249,14 @@ public class VisibilityController extends 
BaseRegionObserver implements MasterOb
 
     // We need to create another KV, unfortunately, because the current new KV
     // has no space for tags
-    KeyValue newKv = KeyValueUtil.ensureKeyValue(newCell);
-    KeyValue rewriteKv = new KeyValue(newKv.getRowArray(), 
newKv.getRowOffset(), newKv.getRowLength(),
-       newKv.getFamilyArray(), newKv.getFamilyOffset(), 
newKv.getFamilyLength(),
-       newKv.getQualifierArray(), newKv.getQualifierOffset(), 
newKv.getQualifierLength(),
-       newKv.getTimestamp(), KeyValue.Type.codeToType(newKv.getTypeByte()),
-       newKv.getValueArray(), newKv.getValueOffset(), newKv.getValueLength(), 
tags);
+    KeyValue rewriteKv = new KeyValue(newCell.getRowArray(), 
newCell.getRowOffset(),
+        newCell.getRowLength(), newCell.getFamilyArray(), 
newCell.getFamilyOffset(),
+        newCell.getFamilyLength(), newCell.getQualifierArray(), 
newCell.getQualifierOffset(),
+        newCell.getQualifierLength(), newCell.getTimestamp(), 
KeyValue.Type.codeToType(newCell
+            .getTypeByte()), newCell.getValueArray(), newCell.getValueOffset(),
+        newCell.getValueLength(), tags);
     // Preserve mvcc data
-    rewriteKv.setSequenceId(newKv.getMvccVersion());
+    rewriteKv.setSequenceId(newCell.getMvccVersion());
     return rewriteKv;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/995a5a6c/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 d3ce165..8e06f31 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
@@ -18,23 +18,16 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNull;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -451,7 +444,7 @@ public class TestFilterList {
       }
 
       @Override
-      public Cell getNextCellHint(Cell currentKV) {
+      public Cell getNextCellHint(Cell cell) {
         return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
       }
 

Reply via email to