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 2014/10/01 10:36:25 UTC

git commit: HBASE-12112 Avoid KeyValueUtil#ensureKeyValue some more simple cases.

Repository: hbase
Updated Branches:
  refs/heads/master 231bc9876 -> 4fac4c1ba


HBASE-12112 Avoid KeyValueUtil#ensureKeyValue some more simple cases.


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

Branch: refs/heads/master
Commit: 4fac4c1ba6fa1a0b30a798d3d1f2a8f803a5c531
Parents: 231bc98
Author: anoopsjohn <an...@gmail.com>
Authored: Wed Oct 1 14:05:57 2014 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Wed Oct 1 14:05:57 2014 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClientScanner.java      |  8 +--
 .../client/ClientSmallReversedScanner.java      |  8 +--
 .../hadoop/hbase/client/ClientSmallScanner.java |  7 +-
 .../apache/hadoop/hbase/client/Increment.java   |  6 +-
 .../apache/hadoop/hbase/client/Mutation.java    | 30 ++++++---
 .../org/apache/hadoop/hbase/client/Result.java  |  2 +-
 .../hadoop/hbase/filter/KeyOnlyFilter.java      | 26 +++++---
 .../java/org/apache/hadoop/hbase/CellKey.java   | 69 +++++++++++++++++++
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 11 +++
 .../io/encoding/BufferedDataBlockEncoder.java   | 13 +++-
 .../hbase/io/hfile/HFilePrettyPrinter.java      | 70 +++++++++++---------
 .../hadoop/hbase/io/hfile/HFileReaderV2.java    |  5 +-
 .../hadoop/hbase/io/hfile/HFileReaderV3.java    |  5 +-
 .../regionserver/DefaultStoreFileManager.java   |  3 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  2 +-
 .../hadoop/hbase/regionserver/HStore.java       | 42 ++++++------
 .../hbase/regionserver/RSRpcServices.java       | 16 ++---
 .../apache/hadoop/hbase/regionserver/Store.java |  4 +-
 .../hbase/regionserver/StoreFileManager.java    |  4 +-
 .../regionserver/StripeStoreFileManager.java    |  9 +--
 20 files changed, 224 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index 09af2b7..9d59242 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -27,11 +27,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownScannerException;
@@ -452,9 +452,9 @@ public class ClientScanner extends AbstractClientScanner {
           if (values != null && values.length > 0) {
             for (Result rs : values) {
               cache.add(rs);
-              for (Cell kv : rs.rawCells()) {
-                // TODO make method in Cell or CellUtil
-                remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
+              // We don't make Iterator here
+              for (Cell cell : rs.rawCells()) {
+                remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
               }
               countdown--;
               this.lastResult = rs;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
index a1df4a4..2cab830 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
@@ -23,11 +23,10 @@ package org.apache.hadoop.hbase.client;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -160,8 +159,9 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
               continue;
             }
             cache.add(rs);
-            for (Cell kv : rs.rawCells()) {
-              remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
+            // We don't make Iterator here
+            for (Cell cell : rs.rawCells()) {
+              remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
             }
             countdown--;
             this.lastResult = rs;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
index 3e4ce69..478ba76 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
@@ -27,8 +27,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@@ -225,8 +225,9 @@ public class ClientSmallScanner extends ClientScanner {
               continue;
             }
             cache.add(rs);
-            for (Cell kv : rs.rawCells()) {
-              remainingResultSize -= KeyValueUtil.ensureKeyValue(kv).heapSize();
+            // We don't make Iterator here
+            for (Cell cell : rs.rawCells()) {
+              remainingResultSize -= CellUtil.estimatedHeapSizeOf(cell);
             }
             countdown--;
             this.lastResult = rs;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index 21af382..6fc3961 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.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.io.TimeRange;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
@@ -240,9 +239,8 @@ public class Increment extends Mutation implements Comparable<Row> {
           } else {
             moreThanOneB = true;
           }
-          KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-          sb.append(Bytes.toStringBinary(kv.getKey()) + "+=" +
-              Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
+          sb.append(CellUtil.getCellKey(cell) + "+=" +
+              Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
         }
         sb.append("}");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index c7ca4c7..a75d3b0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -191,13 +190,8 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
         if (--maxCols <= 0 ) {
           continue;
         }
-        // KeyValue v1 expectation.  Cast for now until we go all Cell all the time.
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        Map<String, Object> kvMap = kv.toStringMap();
-        // row and family information are already available in the bigger map
-        kvMap.remove("row");
-        kvMap.remove("family");
-        qualifierDetails.add(kvMap);
+        Map<String, Object> cellMap = cellToStringMap(cell);
+        qualifierDetails.add(cellMap);
       }
     }
     map.put("totalColumns", colCount);
@@ -208,6 +202,23 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
     return map;
   }
 
+  private static Map<String, Object> cellToStringMap(Cell c) {
+    Map<String, Object> stringMap = new HashMap<String, Object>();
+    stringMap.put("qualifier", Bytes.toStringBinary(c.getQualifierArray(), c.getQualifierOffset(),
+                c.getQualifierLength()));
+    stringMap.put("timestamp", c.getTimestamp());
+    stringMap.put("vlen", c.getValueLength());
+    List<Tag> tags = Tag.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    if (tags != null) {
+      List<String> tagsString = new ArrayList<String>();
+      for (Tag t : tags) {
+        tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(t.getValue()));
+      }
+      stringMap.put("tag", tagsString);
+    }
+    return stringMap;
+  }
+
   /**
    * Set the durability for this mutation
    * @param d
@@ -367,8 +378,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
           size * ClassSize.REFERENCE);
 
       for(Cell cell : entry.getValue()) {
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        heapsize += kv.heapSize();
+        heapsize += CellUtil.estimatedHeapSizeOf(cell);
       }
     }
     heapsize += getAttributeSize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index a041b91..09c58cf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -743,7 +743,7 @@ public class Result implements CellScannable, CellScanner {
   public static long getTotalSizeOfCells(Result result) {
     long size = 0;
     for (Cell c : result.rawCells()) {
-      size += KeyValueUtil.ensureKeyValue(c).heapSize();
+      size += CellUtil.estimatedHeapSizeOf(c);
     }
     return size;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/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 e945d48..cebb26a 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
@@ -29,6 +29,7 @@ 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;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -49,15 +50,24 @@ public class KeyOnlyFilter extends FilterBase {
   public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
 
   @Override
-  public Cell transformCell(Cell kv) {
-    // 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);
+  public Cell transformCell(Cell cell) {
+    return createKeyOnlyCell(cell);
+  }
 
-    return v.createKeyOnly(this.lenAsVal);
+  private Cell createKeyOnlyCell(Cell c) {
+    // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
+    // Rebuild as: <keylen:4><0:4><key:keylen>
+    int dataLen = lenAsVal ? Bytes.SIZEOF_INT : 0;
+    int keyOffset = (2 * Bytes.SIZEOF_INT);
+    int keyLen = KeyValueUtil.keyLength(c);
+    byte[] newBuffer = new byte[keyLen + keyOffset + dataLen];
+    Bytes.putInt(newBuffer, 0, keyLen);
+    Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
+    KeyValueUtil.appendKeyTo(c, newBuffer, keyOffset);
+    if (lenAsVal) {
+      Bytes.putInt(newBuffer, newBuffer.length - dataLen, c.getValueLength());
+    }
+    return new KeyValue(newBuffer);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellKey.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellKey.java
new file mode 100644
index 0000000..f4c0722
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellKey.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This wraps the key portion of a Cell. Key includes rowkey, family, qualifier, timestamp and type
+ */
+@InterfaceAudience.Private
+public class CellKey {
+
+  private byte[] rowArray;
+  private int rowOffset;
+  private int rowLength;
+  private byte[] familyArray;
+  private int familyOffset;
+  private int familyLength;
+  private byte[] qualifierArray;
+  private int qualifierOffset;
+  private int qualifierLength;
+  private long ts;
+  private byte type;
+
+  public CellKey(byte[] rowArray, int rowOffset, int rowLength, byte[] familyArray,
+      int familyOffset, int familyLength, byte[] qualifierArray, int qualifierOffset,
+      int qualifierLength, long ts, byte type) {
+    this.rowArray = rowArray;
+    this.rowOffset = rowOffset;
+    this.rowLength = rowLength;
+    this.familyArray = familyArray;
+    this.familyOffset = familyOffset;
+    this.familyLength = familyLength;
+    this.qualifierArray = qualifierArray;
+    this.qualifierOffset = qualifierOffset;
+    this.qualifierLength = qualifierLength;
+    this.ts = ts;
+    this.type = type;
+  }
+
+  @Override
+  public String toString() {
+    String row = Bytes.toStringBinary(rowArray, rowOffset, rowLength);
+    String family = (familyLength == 0) ? "" : Bytes.toStringBinary(familyArray, familyOffset,
+        familyLength);
+    String qualifier = (qualifierLength == 0) ? "" : Bytes.toStringBinary(qualifierArray,
+        qualifierOffset, qualifierLength);
+    return row + "/" + family +
+        (family != null && family.length() > 0 ? ":" : "") + qualifier
+        + "/" + KeyValue.humanReadableTimestamp(ts) + "/" + Type.codeToType(type);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index f6b2475..ce0f546 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -668,6 +668,17 @@ public final class CellUtil {
   }
 
   /**
+   * @param cell
+   * @return Key portion of the Cell including rk, cf, qualifier, ts and type.
+   */
+  public static CellKey getCellKey(Cell cell){
+    return new CellKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+        cell.getTimestamp(), cell.getTypeByte());
+  }
+
+  /**
    * Write rowkey excluding the common part.
    * @param cell
    * @param rLen

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index 58ebcad..a4b3857 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -32,12 +32,14 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.WritableUtils;
 
 /**
@@ -326,7 +328,10 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
   // there. So this has to be an instance of SettableSequenceId. SeekerState need not be
   // SettableSequenceId as we never return that to top layers. When we have to, we make
   // ClonedSeekerState from it.
-  protected static class ClonedSeekerState implements Cell, SettableSequenceId {
+  protected static class ClonedSeekerState implements Cell, HeapSize, SettableSequenceId {
+    private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
+        + (4 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT)
+        + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE));
     private byte[] keyOnlyBuffer;
     private ByteBuffer currentBuffer;
     private short rowLength;
@@ -507,6 +512,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     public void setSequenceId(long seqId) {
       this.seqId = seqId;
     }
+
+    @Override
+    public long heapSize() {
+      return FIXED_OVERHEAD + rowLength + familyLength + qualifierLength + valueLength + tagsLength
+          + KeyValue.TIMESTAMP_TYPE_SIZE;
+    }
   }
 
   protected abstract static class

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 148b9e3..bde6282 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -45,13 +45,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+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.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
@@ -307,11 +309,12 @@ public class HFilePrettyPrinter extends Configured implements Tool {
 
   private void scanKeysValues(Path file, KeyValueStatsCollector fileStats,
       HFileScanner scanner,  byte[] row) throws IOException {
-    KeyValue pkv = null;
+    Cell pCell = null;
     do {
-      KeyValue kv = KeyValueUtil.ensureKeyValue(scanner.getKeyValue());
+      Cell cell = scanner.getKeyValue();
       if (row != null && row.length != 0) {
-        int result = Bytes.compareTo(kv.getRow(), row);
+        int result = CellComparator.compareRows(cell.getRowArray(), cell.getRowOffset(),
+            cell.getRowLength(), row, 0, row.length);
         if (result > 0) {
           break;
         } else if (result < 0) {
@@ -320,48 +323,51 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       }
       // collect stats
       if (printStats) {
-        fileStats.collect(kv);
+        fileStats.collect(cell);
       }
       // dump key value
       if (printKey) {
-        System.out.print("K: " + kv);
+        System.out.print("K: " + cell);
         if (printValue) {
-          System.out.print(" V: " + Bytes.toStringBinary(kv.getValue()));
+          System.out.print(" V: "
+              + Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
+                  cell.getValueLength()));
           int i = 0;
-          List<Tag> tags = kv.getTags();
+          List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
+              cell.getTagsLength());
           for (Tag tag : tags) {
-            System.out
-                .print(String.format(" T[%d]: %s", i++, Bytes.toStringBinary(tag.getValue())));
+            System.out.print(String.format(" T[%d]: %s", i++,
+                Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength())));
           }
         }
         System.out.println();
       }
       // check if rows are in order
-      if (checkRow && pkv != null) {
-        if (Bytes.compareTo(pkv.getRow(), kv.getRow()) > 0) {
+      if (checkRow && pCell != null) {
+        if (CellComparator.compareRows(pCell, cell) > 0) {
           System.err.println("WARNING, previous row is greater then"
               + " current row\n\tfilename -> " + file + "\n\tprevious -> "
-              + Bytes.toStringBinary(pkv.getKey()) + "\n\tcurrent  -> "
-              + Bytes.toStringBinary(kv.getKey()));
+              + CellUtil.getCellKey(pCell) + "\n\tcurrent  -> "
+              + CellUtil.getCellKey(cell));
         }
       }
       // check if families are consistent
       if (checkFamily) {
-        String fam = Bytes.toString(kv.getFamily());
+        String fam = Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(),
+            cell.getFamilyLength());
         if (!file.toString().contains(fam)) {
           System.err.println("WARNING, filename does not match kv family,"
               + "\n\tfilename -> " + file + "\n\tkeyvalue -> "
-              + Bytes.toStringBinary(kv.getKey()));
+              + CellUtil.getCellKey(cell));
         }
-        if (pkv != null
-            && !Bytes.equals(pkv.getFamily(), kv.getFamily())) {
+        if (pCell != null && CellComparator.compareFamilies(pCell, cell) != 0) {
           System.err.println("WARNING, previous kv has different family"
               + " compared to current key\n\tfilename -> " + file
-              + "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey())
-              + "\n\tcurrent  -> " + Bytes.toStringBinary(kv.getKey()));
+              + "\n\tprevious -> " + CellUtil.getCellKey(pCell)
+              + "\n\tcurrent  -> " + CellUtil.getCellKey(cell));
         }
       }
-      pkv = kv;
+      pCell = cell;
       ++count;
     } while (scanner.next());
   }
@@ -451,21 +457,21 @@ public class HFilePrettyPrinter extends Configured implements Tool {
 
     byte[] biggestRow = null;
 
-    private KeyValue prevKV = null;
+    private Cell prevCell = null;
     private long maxRowBytes = 0;
     private long curRowKeyLength;
 
-    public void collect(KeyValue kv) {
-      valLen.update(kv.getValueLength());
-      if (prevKV != null &&
-          KeyValue.COMPARATOR.compareRows(prevKV, kv) != 0) {
+    public void collect(Cell cell) {
+      valLen.update(cell.getValueLength());
+      if (prevCell != null &&
+          KeyValue.COMPARATOR.compareRows(prevCell, cell) != 0) {
         // new row
         collectRow();
       }
-      curRowBytes += kv.getLength();
-      curRowKeyLength = kv.getKeyLength();
+      curRowBytes += KeyValueUtil.length(cell);
+      curRowKeyLength = KeyValueUtil.keyLength(cell);
       curRowCols++;
-      prevKV = kv;
+      prevCell = cell;
     }
 
     private void collectRow() {
@@ -473,8 +479,8 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       rowSizeCols.update(curRowCols);
       keyLen.update(curRowKeyLength);
 
-      if (curRowBytes > maxRowBytes && prevKV != null) {
-        biggestRow = prevKV.getRow();
+      if (curRowBytes > maxRowBytes && prevCell != null) {
+        biggestRow = prevCell.getRow();
         maxRowBytes = curRowBytes;
       }
 
@@ -490,7 +496,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
 
     @Override
     public String toString() {
-      if (prevKV == null)
+      if (prevCell == null)
         return "no data available for statistics";
 
       // Dump the metrics to the output stream

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
index c243b1d..f2249ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
@@ -29,10 +29,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+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.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
@@ -1021,10 +1021,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
         if (comp == 0) {
           if (seekBefore) {
             if (lastKeyValueSize < 0) {
-              KeyValue kv = KeyValueUtil.ensureKeyValue(key);
               throw new IllegalStateException("blockSeek with seekBefore "
                   + "at the first key of the block: key="
-                  + Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
+                  + CellUtil.getCellKey(key)
                   + ", blockOffset=" + block.getOffset() + ", onDiskSize="
                   + block.getOnDiskSizeWithHeader());
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java
index 0d5ced9..e3c92cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.crypto.Cipher;
@@ -295,10 +295,9 @@ public class HFileReaderV3 extends HFileReaderV2 {
         if (comp == 0) {
           if (seekBefore) {
             if (lastKeyValueSize < 0) {
-              KeyValue kv = KeyValueUtil.ensureKeyValue(key);
               throw new IllegalStateException("blockSeek with seekBefore "
                   + "at the first key of the block: key="
-                  + Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
+                  + CellUtil.getCellKey(key)
                   + ", blockOffset=" + block.getOffset() + ", onDiskSize="
                   + block.getOnDiskSizeWithHeader());
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
index eb5522d..8f33dc4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 
@@ -104,7 +105,7 @@ class DefaultStoreFileManager implements StoreFileManager {
 
   @Override
   public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
-      Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final KeyValue candidate) {
+      Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
     // Default store has nothing useful to do here.
     // TODO: move this comment when implementing Level:
     // Level store can trim the list by range, removing all the files which cannot have

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/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 2fd03de..e362a17 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
@@ -2042,7 +2042,7 @@ public class HRegion implements HeapSize { // , Writable{
     try {
       Store store = getStore(family);
       // get the closest key. (HStore.getRowKeyAtOrBefore can return null)
-      KeyValue key = store.getRowKeyAtOrBefore(row);
+      Cell key = store.getRowKeyAtOrBefore(row);
       Result result = null;
       if (key != null) {
         Get get = new Get(CellUtil.cloneRow(key));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 4813e10..7a331b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -49,13 +49,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -709,30 +709,30 @@ public class HStore implements Store {
       if (verifyBulkLoads) {
         long verificationStartTime = EnvironmentEdgeManager.currentTime();
         LOG.info("Full verification started for bulk load hfile: " + srcPath.toString());
-        Cell prevKV = null;
+        Cell prevCell = null;
         HFileScanner scanner = reader.getScanner(false, false, false);
         scanner.seekTo();
         do {
-          Cell kv = scanner.getKeyValue();
-          if (prevKV != null) {
-            if (Bytes.compareTo(prevKV.getRowArray(), prevKV.getRowOffset(),
-                prevKV.getRowLength(), kv.getRowArray(), kv.getRowOffset(),
-                kv.getRowLength()) > 0) {
+          Cell cell = scanner.getKeyValue();
+          if (prevCell != null) {
+            if (CellComparator.compareRows(prevCell, cell) > 0) {
               throw new InvalidHFileException("Previous row is greater than"
                   + " current row: path=" + srcPath + " previous="
-                  + Bytes.toStringBinary(KeyValueUtil.ensureKeyValue(prevKV).getKey()) + " current="
-                  + Bytes.toStringBinary(KeyValueUtil.ensureKeyValue(kv).getKey()));
+                  + CellUtil.getCellKey(prevCell) + " current="
+                  + CellUtil.getCellKey(cell));
             }
-            if (Bytes.compareTo(prevKV.getFamilyArray(), prevKV.getFamilyOffset(),
-                prevKV.getFamilyLength(), kv.getFamilyArray(), kv.getFamilyOffset(),
-                kv.getFamilyLength()) != 0) {
+            if (CellComparator.compareFamilies(prevCell, cell) != 0) {
               throw new InvalidHFileException("Previous key had different"
                   + " family compared to current key: path=" + srcPath
-                  + " previous=" + Bytes.toStringBinary(prevKV.getFamily())
-                  + " current=" + Bytes.toStringBinary(kv.getFamily()));
+                  + " previous="
+                  + Bytes.toStringBinary(prevCell.getFamilyArray(), prevCell.getFamilyOffset(),
+                      prevCell.getFamilyLength())
+                  + " current="
+                  + Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
+                      cell.getFamilyLength()));
             }
           }
-          prevKV = kv;
+          prevCell = cell;
         } while (scanner.next());
       LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString()
          + " took " + (EnvironmentEdgeManager.currentTime() - verificationStartTime)
@@ -1673,7 +1673,7 @@ public class HStore implements Store {
   }
 
   @Override
-  public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
+  public Cell getRowKeyAtOrBefore(final byte[] row) throws IOException {
     // If minVersions is set, we will not ignore expired KVs.
     // As we're only looking for the latest matches, that should be OK.
     // With minVersions > 0 we guarantee that any KV that has any version
@@ -1698,17 +1698,17 @@ public class HStore implements Store {
         StoreFile sf = sfIterator.next();
         sfIterator.remove(); // Remove sf from iterator.
         boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
-        KeyValue keyv = KeyValueUtil.ensureKeyValue(state.getCandidate());
+        Cell candidate = state.getCandidate();
         // we have an optimization here which stops the search if we find exact match.
-        if (keyv != null && CellUtil.matchingRow(keyv, row)) {
-          return KeyValueUtil.ensureKeyValue(state.getCandidate());
+        if (candidate != null && CellUtil.matchingRow(candidate, row)) {
+          return candidate;
         }
         if (haveNewCandidate) {
           sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
-              sfIterator, state.getTargetKey(), KeyValueUtil.ensureKeyValue(state.getCandidate()));
+              sfIterator, state.getTargetKey(), candidate);
         }
       }
-      return KeyValueUtil.ensureKeyValue(state.getCandidate());
+      return state.getCandidate();
     } finally {
       this.lock.readLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 9bbd691..1254ba5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -48,8 +48,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -2046,7 +2044,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           lease = regionServer.leases.removeLease(scannerName);
           List<Result> results = new ArrayList<Result>(rows);
           long currentScanResultSize = 0;
-          long totalKvSize = 0;
+          long totalCellSize = 0;
 
           boolean done = false;
           // Call coprocessor. Get region info from scanner.
@@ -2056,9 +2054,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             if (!results.isEmpty()) {
               for (Result r : results) {
                 for (Cell cell : r.rawCells()) {
-                  KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-                  currentScanResultSize += kv.heapSize();
-                  totalKvSize += kv.getLength();
+                  currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
+                  totalCellSize += CellUtil.estimatedLengthOf(cell);
                 }
               }
             }
@@ -2088,9 +2085,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                   boolean moreRows = scanner.nextRaw(values);
                   if (!values.isEmpty()) {
                     for (Cell cell : values) {
-                      KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-                      currentScanResultSize += kv.heapSize();
-                      totalKvSize += kv.getLength();
+                      currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
+                      totalCellSize += CellUtil.estimatedLengthOf(cell);
                     }
                     results.add(Result.create(values, null, stale));
                     i++;
@@ -2102,7 +2098,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 }
               }
               region.readRequestsCount.add(i);
-              region.getMetrics().updateScanNext(totalKvSize);
+              region.getMetrics().updateScanNext(totalCellSize);
             } finally {
               region.closeRegionOperation();
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index 60b2411..9078c44 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -148,10 +148,10 @@ public interface Store extends HeapSize, StoreConfigInformation {
    * see deletes before we come across cells we are to delete. Presumption is that the
    * memstore#kvset is processed before memstore#snapshot and so on.
    * @param row The row key of the targeted row.
-   * @return Found keyvalue or null if none found.
+   * @return Found Cell or null if none found.
    * @throws IOException
    */
-  KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException;
+  Cell getRowKeyAtOrBefore(final byte[] row) throws IOException;
 
   FileSystem getFileSystem();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
index f703420..a855c48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
@@ -24,8 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.KVComparator;
 
 import com.google.common.collect.ImmutableCollection;
 
@@ -112,7 +112,7 @@ public interface StoreFileManager {
    * @return The list to replace candidateFiles.
    */
   Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
-    Iterator<StoreFile> candidateFiles, KeyValue targetKey, KeyValue candidate
+    Iterator<StoreFile> candidateFiles, KeyValue targetKey, Cell candidate
   );
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4fac4c1b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
index d4e8800..4bb937b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
@@ -25,16 +25,15 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
@@ -45,8 +44,6 @@ import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.collect.ImmutableCollection;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
 
 /**
  * Stripe implementation of StoreFileManager.
@@ -179,11 +176,11 @@ public class StripeStoreFileManager
   }
 
   /** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)} and
-   * {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, KeyValue)}
+   * {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, Cell)}
    * for details on this methods. */
   @Override
   public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
-      Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final KeyValue candidate) {
+      Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
     KeyBeforeConcatenatedLists.Iterator original =
         (KeyBeforeConcatenatedLists.Iterator)candidateFiles;
     assert original != null;