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/07/09 12:05:54 UTC

[2/2] hbase git commit: HBASE-13387 Add ByteBufferedCell an extension to Cell.

HBASE-13387 Add ByteBufferedCell an extension to Cell.


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

Branch: refs/heads/master
Commit: d07ff5ec5a790282015df7329e797f0d7fd7c3e2
Parents: 41c8ec7
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Jul 9 15:35:24 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Jul 9 15:35:24 2015 +0530

----------------------------------------------------------------------
 dev-support/test-patch.properties               |   2 +-
 .../BigDecimalColumnInterpreter.java            |   3 +-
 .../coprocessor/DoubleColumnInterpreter.java    |   3 +-
 .../coprocessor/LongColumnInterpreter.java      |   3 +-
 .../hbase/security/access/TablePermission.java  |  12 +-
 .../apache/hadoop/hbase/ByteBufferedCell.java   |  85 +++
 .../org/apache/hadoop/hbase/CellComparator.java | 269 ++++-----
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 539 ++++++++++++++++---
 .../org/apache/hadoop/hbase/KeyValueUtil.java   |  83 ---
 .../apache/hadoop/hbase/codec/CellCodec.java    |   1 +
 .../hadoop/hbase/util/ByteBufferUtils.java      |  91 +++-
 .../apache/hadoop/hbase/TestCellComparator.java |  31 ++
 .../org/apache/hadoop/hbase/TestCellUtil.java   | 282 ++++++++++
 .../prefixtree/row/TestPrefixTreeSearcher.java  |   2 +-
 .../hbase/coprocessor/BaseRegionObserver.java   |   2 -
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   6 +-
 .../hbase/regionserver/DefaultMemStore.java     |   9 +-
 .../regionserver/ExplicitColumnTracker.java     |  22 +-
 .../GetClosestRowBeforeTracker.java             |   4 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  22 +-
 .../hadoop/hbase/regionserver/HStore.java       |   1 -
 .../hbase/regionserver/ScanDeleteTracker.java   |  20 +-
 .../hbase/regionserver/ScanQueryMatcher.java    |  18 +-
 .../hbase/regionserver/StoreFileScanner.java    |   7 +-
 .../hadoop/hbase/regionserver/StoreScanner.java |   4 +-
 .../security/access/AccessControlFilter.java    |  10 +-
 .../DefaultVisibilityLabelServiceImpl.java      |  13 +-
 .../visibility/VisibilityLabelFilter.java       |  11 +-
 .../visibility/VisibilityScanDeleteTracker.java |  22 +-
 .../io/encoding/TestDataBlockEncoders.java      |   4 +-
 .../encoding/TestSeekToBlockWithEncoders.java   |  13 +-
 .../hadoop/hbase/util/TestByteBufferUtils.java  |  28 +
 32 files changed, 1155 insertions(+), 467 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/dev-support/test-patch.properties
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.properties b/dev-support/test-patch.properties
index 0815b0e..c652e3f 100644
--- a/dev-support/test-patch.properties
+++ b/dev-support/test-patch.properties
@@ -21,7 +21,7 @@ MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
 OK_RELEASEAUDIT_WARNINGS=0
 # Allow four warnings.  Javadoc complains about sun.misc.Unsafe use.
 # See HBASE-7457, HBASE-13761
-OK_JAVADOC_WARNINGS=6
+OK_JAVADOC_WARNINGS=7
 
 MAX_LINE_LENGTH=100
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
index 97724bd..7d0715b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
@@ -46,8 +46,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
     if (kv == null || CellUtil.cloneValue(kv) == null) {
       return null;
     }
-    return Bytes.toBigDecimal(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()).
-        setScale(2, RoundingMode.HALF_EVEN);
+    return CellUtil.getValueAsBigDecimal(kv).setScale(2, RoundingMode.HALF_EVEN);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
index 8b0c690..5b8571f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client.coprocessor;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
@@ -46,7 +47,7 @@ public class DoubleColumnInterpreter extends ColumnInterpreter<Double, Double,
       throws IOException {
     if (c == null || c.getValueLength() != Bytes.SIZEOF_DOUBLE)
       return null;
-    return Bytes.toDouble(c.getValueArray(), c.getValueOffset());
+    return CellUtil.getValueAsDouble(c);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
index e8e5e3a..586575a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.client.coprocessor;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
@@ -43,7 +44,7 @@ public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
       throws IOException {
     if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG)
       return null;
-    return Bytes.toLong(kv.getValueArray(), kv.getValueOffset());
+    return CellUtil.getValueAsLong(kv);
   }
 
    @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
index 95d962a..e0d3424 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
@@ -22,8 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 @InterfaceAudience.Private
 public class TablePermission extends Permission {
-  private static final Log LOG = LogFactory.getLog(TablePermission.class);
 
   private TableName table;
   private byte[] family;
@@ -248,15 +246,11 @@ public class TablePermission extends Permission {
       return false;
     }
 
-    if (family != null &&
-        (Bytes.compareTo(family, 0, family.length,
-            kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength()) != 0)) {
+    if (family != null && !(CellUtil.matchingFamily(kv, family))) {
       return false;
     }
 
-    if (qualifier != null &&
-        (Bytes.compareTo(qualifier, 0, qualifier.length,
-            kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()) != 0)) {
+    if (qualifier != null && !(CellUtil.matchingQualifier(kv, qualifier))) {
       return false;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java
new file mode 100644
index 0000000..02519f9
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java
@@ -0,0 +1,85 @@
+/**
+ * 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 java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * This class is a server side extension to the Cell interface. This is used when the actual Cell
+ * implementation is backed by {@link ByteBuffer}. This class contain ByteBuffer backed getters for
+ * row, cf, qualifier, value and tags. Also getters of the position where these field bytes begin. A
+ * cell object can be of this type only in server side. When the object is of this type, use the
+ * getXXXByteBuffer() method along with getXXXPositionInByteBuffer(). If cell is backed by off heap
+ * ByteBuffer the call to getXXXArray() will result is temporary byte array creation and bytes copy
+ * resulting in lot of garbage.
+ */
+@InterfaceAudience.Private
+public abstract class ByteBufferedCell implements Cell {
+
+  /**
+   * @return The {@link ByteBuffer} containing the row bytes.
+   */
+  abstract ByteBuffer getRowByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where row bytes start
+   */
+  abstract int getRowPositionInByteBuffer();
+
+  /**
+   * @return The {@link ByteBuffer} containing the column family bytes.
+   */
+  abstract ByteBuffer getFamilyByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where column family bytes start
+   */
+  abstract int getFamilyPositionInByteBuffer();
+
+  /**
+   * @return The {@link ByteBuffer} containing the column qualifier bytes.
+   */
+  abstract ByteBuffer getQualifierByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where column qualifier bytes start
+   */
+  abstract int getQualifierPositionInByteBuffer();
+
+  /**
+   * @return The {@link ByteBuffer} containing the value bytes.
+   */
+  abstract ByteBuffer getValueByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where value bytes start
+   */
+  abstract int getValuePositionInByteBuffer();
+
+  /**
+   * @return The {@link ByteBuffer} containing the tag bytes.
+   */
+  abstract ByteBuffer getTagsByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where tag bytes start
+   */
+  abstract int getTagsPositionInByteBuffer();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/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 8beb277..f9da914 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
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.primitives.Longs;
@@ -36,8 +37,8 @@ import com.google.common.primitives.Longs;
  * takes account of the special formatting of the row where we have commas to delimit table from
  * regionname, from row.  See KeyValue for how it has a special comparator to do hbase:meta cells
  * and yet another for -ROOT-.
- * While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells format
- * should be taken into consideration, for which the instance of this comparator 
+ * While using this comparator for {{@link #compareRows(Cell, Cell)} et al, the hbase:meta cells
+ * format should be taken into consideration, for which the instance of this comparator
  * should be used.  In all other cases the static APIs in this comparator would be enough
  */
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
@@ -90,8 +91,6 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return an int greater than 0 if left is greater than right
    *                lesser than 0 if left is lesser than right
    *                equal to 0 if left is equal to right
-   * TODO : We will be moving over to 
-   * compare(Cell, Cell) so that the key is also converted to a cell
    */
   public final int compare(Cell left, byte[] key, int offset, int length) {
     // row
@@ -132,178 +131,102 @@ public class CellComparator implements Comparator<Cell>, Serializable {
 
   /**
    * Compares the family and qualifier part of the cell
-   * TODO : Handle BB cases here
    * @param left the left cell
    * @param right the right cell
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public final static int compareColumns(final Cell left, final Cell right) {
-    int lfoffset = left.getFamilyOffset();
-    int rfoffset = right.getFamilyOffset();
-    int lclength = left.getQualifierLength();
-    int rclength = right.getQualifierLength();
-    int lfamilylength = left.getFamilyLength();
-    int rfamilylength = right.getFamilyLength();
-    int diff = compareFamilies(left.getFamilyArray(), lfoffset, lfamilylength,
-        right.getFamilyArray(), rfoffset, rfamilylength);
+    int diff = compareFamilies(left, right);
     if (diff != 0) {
       return diff;
-    } else {
-      return compareQualifiers(left.getQualifierArray(), left.getQualifierOffset(), lclength,
-          right.getQualifierArray(), right.getQualifierOffset(), rclength);
     }
+    return compareQualifiers(left, right);
   }
 
-  /**
-   * Compares the family and qualifier part of the cell
-   * We explicitly pass the offset and length details of the cells to avoid
-   * re-parsing of the offset and length from the cell. Used only internally.
-   * @param left
-   * @param lfamilyOffset
-   * @param lfamilylength
-   * @param lqualOffset
-   * @param lQualLength
-   * @param right
-   * @param rfamilyOffset
-   * @param rfamilylength
-   * @param rqualOffset
-   * @param rqualLength
-   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
-   */
-  private final static int compareColumns(final Cell left, int lfamilyOffset, int lfamilylength,
-      int lqualOffset, int lQualLength, final Cell right, final int rfamilyOffset,
-      final int rfamilylength, final int rqualOffset, int rqualLength) {
-    int diff = compareFamilies(left.getFamilyArray(), lfamilyOffset, lfamilylength,
-        right.getFamilyArray(), rfamilyOffset, rfamilylength);
-    if (diff != 0) {
-      return diff;
-    } else {
-      return compareQualifiers(left.getQualifierArray(), lqualOffset, lQualLength,
-          right.getQualifierArray(), rqualOffset, rqualLength);
-    }
-  }
-  
-  /**
-   * Compares the family and qualifier part of a cell with a serialized Key value byte[]
-   * We explicitly pass the offset and length details of the cells to avoid
-   * re-parsing of the offset and length from the cell. Used only internally.
-   * @param left the cell to be compared
-   * @param lfamilyOffset
-   * @param lfamilylength
-   * @param lqualOffset
-   * @param lQualLength
-   * @param right the serialized key value byte array to be compared
-   * @param rfamilyOffset
-   * @param rfamilylength
-   * @param rqualOffset
-   * @param rqualLength
-   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
-   */
-  private final static int compareColumns(final Cell left, final int lfamilyOffset,
-      final int lfamilylength, final int lqualOffset, final int lQualLength, final byte[] right,
-      final int rfamilyOffset, final int rfamilylength, final int rqualOffset,
-      final int rqualLength) {
-    int diff = compareFamilies(left.getFamilyArray(), lfamilyOffset, lfamilylength, right,
-        rfamilyOffset, rfamilylength);
-    if (diff != 0) {
+  private final static int compareColumns(Cell left, byte[] right, int rfoffset, int rflength,
+      int rqoffset, int rqlength) {
+    int diff = compareFamilies(left, right, rfoffset, rflength);
+    if (diff != 0)
       return diff;
-    } else {
-      return compareQualifiers(left.getQualifierArray(), lqualOffset, lQualLength, right,
-          rqualOffset, rqualLength);
-    }
+    return compareQualifiers(left, right, rqoffset, rqlength);
   }
 
   /**
    * Compare the families of left and right cell
-   * TODO : Handle BB cases here
    * @param left
    * @param right
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public final static int compareFamilies(Cell left, Cell right) {
-    return compareFamilies(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(),
+          ((ByteBufferedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), right.getFamilyLength());
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(),
+          right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
+    }
+    if (right instanceof ByteBufferedCell) {
+      return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), right.getFamilyLength(),
+          left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength()));
+    }
+    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
   }
 
-  /**
-   * We explicitly pass the offset and length details of the cells to avoid
-   * re-parsing of the offset and length from the cell. Used only internally.
-   * @param left
-   * @param lOffset
-   * @param lLength
-   * @param right
-   * @param rOffset
-   * @param rLength
-   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
-   */
-  private final static int compareFamilies(Cell left, int lOffset, int lLength, Cell right,
-      int rOffset, int rLength) {
-    return compareFamilies(left.getFamilyArray(), lOffset, lLength, right.getFamilyArray(),
-        rOffset, rLength);
-  }
-
-  private final static int compareFamilies(Cell left, int lOffset, int lLength, byte[] right,
-      int rOffset, int rLength) {
-    return compareFamilies(left.getFamilyArray(), lOffset, lLength, right, rOffset, rLength);
-  }
-
-  private final static int compareFamilies(byte[] leftFamily, int lFamOffset, int lFamLength,
-      byte[] rightFamily, int rFamOffset, int rFamLen) {
-    return Bytes.compareTo(leftFamily, lFamOffset, lFamLength, rightFamily, rFamOffset, rFamLen);
+  private final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(), right,
+          roffset, rlength);
+    }
+    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+        right, roffset, rlength);
   }
 
   /**
    * Compare the qualifiers part of the left and right cells.
-   * TODO : Handle BB cases here
    * @param left
    * @param right
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public final static int compareQualifiers(Cell left, Cell right) {
-    return compareQualifiers(left.getQualifierArray(), left.getQualifierOffset(),
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils
+          .compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
+              ((ByteBufferedCell) left).getQualifierPositionInByteBuffer(),
+              left.getQualifierLength(), ((ByteBufferedCell) right).getQualifierByteBuffer(),
+              ((ByteBufferedCell) right).getQualifierPositionInByteBuffer(),
+              right.getQualifierLength());
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), left.getQualifierLength(),
+          right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
+    }
+    if (right instanceof ByteBufferedCell) {
+      return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferedCell) right).getQualifierPositionInByteBuffer(),
+          right.getQualifierLength(), left.getQualifierArray(), left.getQualifierOffset(),
+          left.getQualifierLength()));
+    }
+    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
         right.getQualifierLength());
   }
 
- /**
-  * We explicitly pass the offset and length details of the cells to avoid
-  * re-parsing of the offset and length from the cell. Used only internally.
-  * @param left
-  * @param lOffset
-  * @param lLength
-  * @param right
-  * @param rOffset
-  * @param rLength
-  * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
-  */
-  private final static int compareQualifiers(Cell left, int lOffset, int lLength, Cell right,
-      int rOffset, int rLength) {
-    return compareQualifiers(left.getQualifierArray(), lOffset,
-        lLength, right.getQualifierArray(), rOffset,
-        rLength);
-  }
-
-  /**
-   * We explicitly pass the offset and length details of the cells to avoid
-   * re-parsing of the offset and length from the cell. Used only internally.
-   * @param left
-   * @param lOffset
-   * @param lLength
-   * @param right
-   * @param rOffset
-   * @param rLength
-   * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
-   */
-  private final static int compareQualifiers(Cell left, int lOffset, int lLength, byte[] right,
-      int rOffset, int rLength) {
-    return compareQualifiers(left.getQualifierArray(), lOffset,
-        lLength, right, rOffset,
-        rLength);
-  }
-
-  private static int compareQualifiers(byte[] leftCol, int lColOffset, int lColLength,
-      byte[] rightCol, int rColOffset, int rColLength) {
-    return Bytes.compareTo(leftCol, lColOffset, lColLength, rightCol, rColOffset, rColLength);
+  public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), left.getQualifierLength(),
+          right, rOffset, rLength);
+    }
+    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
+        left.getQualifierLength(), right, rOffset, rLength);
   }
 
   /**
@@ -312,7 +235,6 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * the "same-prefix" comparator. Note that we are assuming that row portions
    * of both KVs have already been parsed and found identical, and we don't
    * validate that assumption here.
-   * TODO :  we will have to handle BB cases here
    * @param commonPrefix
    *          the length of the common prefix of the two key-values being
    *          compared, including row length and row
@@ -348,7 +270,6 @@ public class CellComparator implements Comparator<Cell>, Serializable {
       return -1;
     }
 
-    int lfamilyoffset = left.getFamilyOffset();
     int rfamilyoffset = commonLength + roffset;
 
     // Column family length.
@@ -359,16 +280,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
     boolean sameFamilySize = (lfamilylength == rfamilylength);
     if (!sameFamilySize) {
       // comparing column family is enough.
-      return compareFamilies(left, lfamilyoffset, lfamilylength, right,
-          rfamilyoffset, rfamilylength);
+      return compareFamilies(left, right, rfamilyoffset, rfamilylength);
     }
     // Compare family & qualifier together.
     // Families are same. Compare on qualifiers.
-    int lQualOffset = left.getQualifierOffset();
-    int lQualLength = left.getQualifierLength();
-    int comparison = compareColumns(left, lfamilyoffset, lfamilylength, lQualOffset, lQualLength,
-        right, rfamilyoffset, rfamilylength, rfamilyoffset + rfamilylength,
-        (rcolumnlength - rfamilylength));
+    int comparison = compareColumns(left, right, rfamilyoffset, rfamilylength, rfamilyoffset
+        + rfamilylength, (rcolumnlength - rfamilylength));
     if (comparison != 0) {
       return comparison;
     }
@@ -392,12 +309,27 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * Compares the rows of the left and right cell.
    * For the hbase:meta case this method is overridden such that it can handle hbase:meta cells.
    * The caller should ensure using the appropriate comparator for hbase:meta.
-   * TODO : Handle BB cases here
    * @param left
    * @param right
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public int compareRows(final Cell left, final Cell right) {
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
+          ((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(),
+          ((ByteBufferedCell) right).getRowByteBuffer(),
+          ((ByteBufferedCell) right).getRowPositionInByteBuffer(), right.getRowLength());
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
+          ((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(),
+          right.getRowArray(), right.getRowOffset(), right.getRowLength());
+    }
+    if (right instanceof ByteBufferedCell) {
+      return -(ByteBufferUtils.compareTo(((ByteBufferedCell) right).getRowByteBuffer(),
+          ((ByteBufferedCell) right).getRowPositionInByteBuffer(), right.getRowLength(),
+          left.getRowArray(), left.getRowOffset(), left.getRowLength()));
+    }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
         right.getRowArray(), right.getRowOffset(), right.getRowLength());
   }
@@ -418,11 +350,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return 0 if both cell and the byte[] are equal, 1 if the cell is bigger
    *         than byte[], -1 otherwise
    */
-  public int compareRows(Cell left, byte[] right, int roffset,
-      int rlength) {
-    // TODO : for BB based cells all the hasArray based checks would happen
-    // here. But we may have
-    // to end up in multiple APIs accepting byte[] and BBs
+  public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
+          ((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(), right,
+          roffset, rlength);
+    }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
         roffset, rlength);
   }
@@ -447,18 +380,12 @@ public class CellComparator implements Comparator<Cell>, Serializable {
         && right.getTypeByte() == Type.Minimum.getCode()) {
       return -1;
     }
-    boolean sameFamilySize = (lFamLength == rFamLength);
-    int lFamOffset = left.getFamilyOffset();
-    int rFamOffset = right.getFamilyOffset();
-    if (!sameFamilySize) {
+    if (lFamLength != rFamLength) {
       // comparing column family is enough.
-      return compareFamilies(left, lFamOffset, lFamLength, right, rFamOffset, rFamLength);
+      return compareFamilies(left, right);
     }
-    // Families are same. Compare on qualifiers.
-    int lQualOffset = left.getQualifierOffset();
-    int rQualOffset = right.getQualifierOffset();
-    int diff = compareColumns(left, lFamOffset, lFamLength, lQualOffset, lQualLength, right,
-        rFamOffset, rFamLength, rQualOffset, rQualLength);
+    // Compare cf:qualifier
+    int diff = compareColumns(left, right);
     if (diff != 0) return diff;
 
     diff = compareTimestamps(left, right);
@@ -521,24 +448,18 @@ public class CellComparator implements Comparator<Cell>, Serializable {
       // left is "bigger", i.e. it appears later in the sorted order
       return 1;
     }
-    int qualLen = currentCell.getQualifierLength();
     if (flen + clen == 0 && type == Type.Minimum.getCode()) {
       return -1;
     }
 
-    compare = compareFamilies(nextIndexedCell, nextIndexedCell.getFamilyOffset(),
-        nextIndexedCell.getFamilyLength(), currentCell, currentCell.getFamilyOffset(),
-        flen);
+    compare = compareFamilies(nextIndexedCell, currentCell);
     if (compare != 0) {
       return compare;
     }
     if (colHint == null) {
-      compare = compareQualifiers(nextIndexedCell, nextIndexedCell.getQualifierOffset(),
-          nextIndexedCell.getQualifierLength(), currentCell, currentCell.getQualifierOffset(),
-          qualLen);
+      compare = compareQualifiers(nextIndexedCell, currentCell);
     } else {
-      compare = compareQualifiers(nextIndexedCell, nextIndexedCell.getQualifierOffset(),
-          nextIndexedCell.getQualifierLength(), colHint, coff, clen);
+      compare = compareQualifiers(nextIndexedCell, colHint, coff, clen);
     }
     if (compare != 0) {
       return compare;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/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 5c15681..72d146b 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
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HConstants.EMPTY_BYTE_ARRAY;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.List;
@@ -114,27 +115,55 @@ public final class CellUtil {
   /******************** copyTo **********************************/
 
   public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
-      cell.getRowLength());
-    return destinationOffset + cell.getRowLength();
+    short rowLen = cell.getRowLength();
+    if (cell instanceof ByteBufferedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferedCell) cell).getRowByteBuffer(),
+          ((ByteBufferedCell) cell).getRowPositionInByteBuffer(), destinationOffset, rowLen);
+    } else {
+      System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
+          rowLen);
+    }
+    return destinationOffset + rowLen;
   }
 
   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination, destinationOffset,
-      cell.getFamilyLength());
-    return destinationOffset + cell.getFamilyLength();
+    byte fLen = cell.getFamilyLength();
+    if (cell instanceof ByteBufferedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferedCell) cell).getFamilyPositionInByteBuffer(), destinationOffset, fLen);
+    } else {
+      System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
+          destinationOffset, fLen);
+    }
+    return destinationOffset + fLen;
   }
 
   public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
-      destinationOffset, cell.getQualifierLength());
-    return destinationOffset + cell.getQualifierLength();
+    int qlen = cell.getQualifierLength();
+    if (cell instanceof ByteBufferedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferedCell) cell).getQualifierPositionInByteBuffer(), destinationOffset, qlen);
+    } else {
+      System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
+          destinationOffset, qlen);
+    }
+    return destinationOffset + qlen;
   }
 
   public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
-        cell.getValueLength());
-    return destinationOffset + cell.getValueLength();
+    int vlen = cell.getValueLength();
+    if (cell instanceof ByteBufferedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferedCell) cell).getValueByteBuffer(),
+          ((ByteBufferedCell) cell).getValuePositionInByteBuffer(), destinationOffset, vlen);
+    } else {
+      System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
+          vlen);
+    }
+    return destinationOffset + vlen;
   }
 
   /**
@@ -145,14 +174,25 @@ public final class CellUtil {
    * @return position after tags
    */
   public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
-    System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
-        cell.getTagsLength());
-    return destinationOffset + cell.getTagsLength();
+    int tlen = cell.getTagsLength();
+    if (cell instanceof ByteBufferedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferedCell) cell).getTagsByteBuffer(),
+          ((ByteBufferedCell) cell).getTagsPositionInByteBuffer(), destinationOffset, tlen);
+    } else {
+      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
+          tlen);
+    }
+    return destinationOffset + tlen;
   }
 
   /********************* misc *************************************/
 
   public static byte getRowByte(Cell cell, int index) {
+    if (cell instanceof ByteBufferedCell) {
+      return ((ByteBufferedCell) cell).getRowByteBuffer().get(
+          ((ByteBufferedCell) cell).getRowPositionInByteBuffer() + index);
+    }
     return cell.getRowArray()[cell.getRowOffset() + index];
   }
 
@@ -162,7 +202,14 @@ public final class CellUtil {
     return buffer;
   }
 
+  /**
+   * @param cell
+   * @return cell's qualifier wrapped into a ByteBuffer.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   */
+  @Deprecated
   public static ByteBuffer getQualifierBufferShallowCopy(Cell cell) {
+    // No usage of this in code.
     ByteBuffer buffer = ByteBuffer.wrap(cell.getQualifierArray(), cell.getQualifierOffset(),
         cell.getQualifierLength());
     return buffer;
@@ -374,10 +421,12 @@ public final class CellUtil {
    * @param left
    * @param right
    * @return True if the rows in <code>left</code> and <code>right</code> Cells match
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Instead use {@link #matchingRows(Cell, Cell)}
    */
+  @Deprecated
   public static boolean matchingRow(final Cell left, final Cell right) {
-    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+    return matchingRows(left, right);
   }
 
   public static boolean matchingRow(final Cell left, final byte[] buf) {
@@ -389,11 +438,35 @@ public final class CellUtil {
 
   public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
       final int length) {
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
+          ((ByteBufferedCell) left).getRowPositionInByteBuffer(), left.getRowLength(), buf, offset,
+          length) == 0;
+    }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
         length);
   }
 
   public static boolean matchingFamily(final Cell left, final Cell right) {
+    byte lfamlength = left.getFamilyLength();
+    byte rfamlength = right.getFamilyLength();
+    if (lfamlength != rfamlength) return false;
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), lfamlength,
+          ((ByteBufferedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), rfamlength) == 0;
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), lfamlength,
+          right.getFamilyArray(), right.getFamilyOffset(), rfamlength) == 0;
+    }
+    if (right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferedCell) right).getFamilyPositionInByteBuffer(), rfamlength,
+          left.getFamilyArray(), left.getFamilyOffset(), lfamlength) == 0;
+    }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
   }
@@ -407,11 +480,35 @@ public final class CellUtil {
 
   public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
       final int length) {
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferedCell) left).getFamilyPositionInByteBuffer(), left.getFamilyLength(), buf,
+          offset, length) == 0;
+    }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
         offset, length);
   }
 
   public static boolean matchingQualifier(final Cell left, final Cell right) {
+    int lqlength = left.getQualifierLength();
+    int rqlength = right.getQualifierLength();
+    if (lqlength != rqlength) return false;
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), lqlength,
+          ((ByteBufferedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferedCell) right).getQualifierPositionInByteBuffer(), rqlength) == 0;
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), lqlength,
+          right.getQualifierArray(), right.getQualifierOffset(), rqlength) == 0;
+    }
+    if (right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferedCell) right).getQualifierPositionInByteBuffer(), rqlength,
+          left.getQualifierArray(), left.getQualifierOffset(), lqlength) == 0;
+    }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
         right.getQualifierLength());
@@ -445,6 +542,11 @@ public final class CellUtil {
     if (buf == null) {
       return left.getQualifierLength() == 0;
     }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferedCell) left).getQualifierPositionInByteBuffer(), left.getQualifierLength(),
+          buf, offset, length) == 0;
+    }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), buf, offset, length);
   }
@@ -469,11 +571,35 @@ public final class CellUtil {
   }
 
   public static boolean matchingValue(final Cell left, final Cell right) {
-    return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
-        right.getValueArray(), right.getValueOffset(), right.getValueLength());
+    int lvlength = left.getValueLength();
+    int rvlength = right.getValueLength();
+    if (lvlength != rvlength) return false;
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
+          ((ByteBufferedCell) left).getValuePositionInByteBuffer(), lvlength,
+          ((ByteBufferedCell) right).getValueByteBuffer(),
+          ((ByteBufferedCell) right).getValuePositionInByteBuffer(), rvlength) == 0;
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
+          ((ByteBufferedCell) left).getValuePositionInByteBuffer(), lvlength,
+          right.getValueArray(), right.getValueOffset(), rvlength) == 0;
+    }
+    if (right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getValueByteBuffer(),
+          ((ByteBufferedCell) right).getValuePositionInByteBuffer(), rvlength,
+          left.getValueArray(), left.getValueOffset(), lvlength) == 0;
+    }
+    return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
+        right.getValueArray(), right.getValueOffset(), rvlength);
   }
 
   public static boolean matchingValue(final Cell left, final byte[] buf) {
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
+          ((ByteBufferedCell) left).getValuePositionInByteBuffer(), left.getValueLength(), buf, 0,
+          buf.length) == 0;
+    }
     return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
         buf.length);
   }
@@ -967,66 +1093,100 @@ public final class CellUtil {
   public static boolean matchingRows(final Cell left, final Cell right) {
     short lrowlength = left.getRowLength();
     short rrowlength = right.getRowLength();
-    return matchingRows(left, lrowlength, right, rrowlength);
+    if (lrowlength != rrowlength) return false;
+    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
+          ((ByteBufferedCell) left).getRowPositionInByteBuffer(), lrowlength,
+          ((ByteBufferedCell) right).getRowByteBuffer(),
+          ((ByteBufferedCell) right).getRowPositionInByteBuffer(), rrowlength) == 0;
+    }
+    if (left instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
+          ((ByteBufferedCell) left).getRowPositionInByteBuffer(), lrowlength, right.getRowArray(),
+          right.getRowOffset(), rrowlength) == 0;
+    }
+    if (right instanceof ByteBufferedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferedCell) right).getRowByteBuffer(),
+          ((ByteBufferedCell) right).getRowPositionInByteBuffer(), rrowlength, left.getRowArray(),
+          left.getRowOffset(), lrowlength) == 0;
+    }
+    return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
+        right.getRowArray(), right.getRowOffset(), right.getRowLength());
   }
 
   /**
+   * Compares the row and column of two keyvalues for equality
+   *
    * @param left
-   * @param lrowlength
    * @param right
-   * @param rrowlength
-   * @return True if rows match.
+   * @return True if same row and column.
    */
-  private static boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
-      final short rrowlength) {
-    return lrowlength == rrowlength
-        && matchingRows(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
-            right.getRowOffset(), rrowlength);
+  public static boolean matchingRowColumn(final Cell left, final Cell right) {
+    if ((left.getRowLength() + left.getFamilyLength() + left.getQualifierLength()) != (right
+        .getRowLength() + right.getFamilyLength() + right.getQualifierLength())) {
+      return false;
+    }
+
+    if (!matchingRows(left, right)) {
+      return false;
+    }
+    return matchingColumn(left, right);
   }
 
   /**
-   * Compare rows. Just calls Bytes.equals, but it's good to have this
-   * encapsulated.
-   * 
-   * @param left
-   *          Left row array.
-   * @param loffset
-   *          Left row offset.
-   * @param llength
-   *          Left row length.
-   * @param right
-   *          Right row array.
-   * @param roffset
-   *          Right row offset.
-   * @param rlength
-   *          Right row length.
-   * @return Whether rows are the same row.
+   * Converts the rowkey bytes of the given cell into an int value
+   *
+   * @param cell
+   * @return rowkey as int
    */
-  private static boolean matchingRows(final byte[] left, final int loffset, final int llength,
-      final byte[] right, final int roffset, final int rlength) {
-    return Bytes.equals(left, loffset, llength, right, roffset, rlength);
+  public static int getRowAsInt(Cell cell) {
+    if (cell instanceof ByteBufferedCell) {
+      return ByteBufferUtils.toInt(((ByteBufferedCell) cell).getRowByteBuffer(),
+          ((ByteBufferedCell) cell).getRowPositionInByteBuffer());
+    }
+    return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
   }
 
   /**
-   * Compares the row and column of two keyvalues for equality
-   * 
-   * @param left
-   * @param right
-   * @return True if same row and column.
+   * Converts the value bytes of the given cell into a long value
+   *
+   * @param cell
+   * @return value as long
    */
-  public static boolean matchingRowColumn(final Cell left, final Cell right) {
-    short lrowlength = left.getRowLength();
-    short rrowlength = right.getRowLength();
+  public static long getValueAsLong(Cell cell) {
+    if (cell instanceof ByteBufferedCell) {
+      return ByteBufferUtils.toLong(((ByteBufferedCell) cell).getValueByteBuffer(),
+          ((ByteBufferedCell) cell).getValuePositionInByteBuffer());
+    }
+    return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
+  }
 
-    if ((lrowlength + left.getFamilyLength() + left.getQualifierLength()) != (rrowlength
-        + right.getFamilyLength() + right.getQualifierLength())) {
-      return false;
+  /**
+   * Converts the value bytes of the given cell into a double value
+   *
+   * @param cell
+   * @return value as double
+   */
+  public static double getValueAsDouble(Cell cell) {
+    if (cell instanceof ByteBufferedCell) {
+      return ByteBufferUtils.toDouble(((ByteBufferedCell) cell).getValueByteBuffer(),
+          ((ByteBufferedCell) cell).getValuePositionInByteBuffer());
     }
+    return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
+  }
 
-    if (!matchingRows(left, lrowlength, right, rrowlength)) {
-      return false;
+  /**
+   * Converts the value bytes of the given cell into a BigDecimal
+   *
+   * @param cell
+   * @return value as BigDecimal
+   */
+  public static BigDecimal getValueAsBigDecimal(Cell cell) {
+    if (cell instanceof ByteBufferedCell) {
+      return ByteBufferUtils.toBigDecimal(((ByteBufferedCell) cell).getValueByteBuffer(),
+          ((ByteBufferedCell) cell).getValuePositionInByteBuffer(), cell.getValueLength());
     }
-    return matchingColumn(left, right);
+    return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }
 
   /**
@@ -1039,6 +1199,71 @@ public final class CellUtil {
     return new FirstOnRowFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
 
+  /**
+   * Create a Cell that is smaller than all other possible Cells for the given Cell row's next row.
+   * Makes the next row's rowkey by appending single byte 0x00 to the end of current row key.
+   */
+  public static Cell createFirstOnNextRow(final Cell cell) {
+    byte[] nextRow = new byte[cell.getRowLength() + 1];
+    copyRowTo(cell, nextRow, 0);
+    nextRow[nextRow.length - 1] = 0;// maybe not necessary
+    return new FirstOnRowFakeCell(nextRow, 0, (short) nextRow.length);
+  }
+
+  /**
+   * Create a Cell that is smaller than all other possible Cells for the given Cell's rk:cf and
+   * passed qualifier.
+   *
+   * @param cell
+   * @param qArray
+   * @param qoffest
+   * @param qlength
+   * @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
+   */
+  public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
+    return new FirstOnRowColumnFakeCell(cell.getRowArray(), cell.getRowOffset(),
+        cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+        qArray, qoffest, qlength);
+  }
+
+  /**
+   * Creates the first cell with the row/family/qualifier of this cell and the given timestamp.
+   * Uses the "maximum" type that guarantees that the new cell is the lowest possible for this
+   * combination of row, family, qualifier, and timestamp. This cell's own timestamp is ignored.
+   *
+   * @param cell - cell
+   * @param ts
+   */
+  public static Cell createFirstOnRowColTS(Cell cell, long ts) {
+    return new FirstOnRowColumnTSFakeCell(cell.getRowArray(), cell.getRowOffset(),
+        cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), ts);
+  }
+
+  /**
+   * Create a Cell that is larger than all other possible Cells for the given Cell's row.
+   *
+   * @param cell
+   * @return Last possible Cell on passed Cell's row.
+   */
+  public static Cell createLastOnRow(final Cell cell) {
+    return new LastOnRowFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
+  }
+
+  /**
+   * Create a Cell that is larger than all other possible Cells for the given Cell's rk:cf:q. Used
+   * in creating "fake keys" for the multi-column Bloom filter optimization to skip the row/column
+   * we already know is not in the file.
+   *
+   * @param cell
+   * @return Last possible Cell on passed Cell's rk:cf:q.
+   */
+  public static Cell createLastOnRowCol(final Cell cell) {
+    return new LastOnRowColumnFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+        cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
+  }
+
   @InterfaceAudience.Private
   private static abstract class FakeCell implements Cell {
 
@@ -1151,12 +1376,12 @@ public final class CellUtil {
   @InterfaceAudience.Private
   private static class FirstOnRowFakeCell extends FakeCell {
     private final byte[] rowArray;
-    private final int roffest;
+    private final int roffset;
     private final short rlength;
 
     public FirstOnRowFakeCell(final byte[] row, int roffset, short rlength) {
       this.rowArray = row;
-      this.roffest = roffset;
+      this.roffset = roffset;
       this.rlength = rlength;
     }
 
@@ -1167,7 +1392,7 @@ public final class CellUtil {
 
     @Override
     public int getRowOffset() {
-      return this.roffest;
+      return this.roffset;
     }
 
     @Override
@@ -1187,7 +1412,189 @@ public final class CellUtil {
 
     @Override
     public byte[] getRow() {
-      return Bytes.copy(this.rowArray, this.roffest, this.rlength);
+      return Bytes.copy(this.rowArray, this.roffset, this.rlength);
+    }
+  }
+
+  @InterfaceAudience.Private
+  private static class FirstOnRowColumnFakeCell extends FirstOnRowFakeCell {
+    private final byte[] fArray;
+    private final int foffset;
+    private final byte flength;
+    private final byte[] qArray;
+    private final int qoffset;
+    private final int qlength;
+
+    public FirstOnRowColumnFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
+        int foffset, byte flength, byte[] qArray, int qoffset, int qlength) {
+      super(rArray, roffset, rlength);
+      this.fArray = fArray;
+      this.foffset = foffset;
+      this.flength = flength;
+      this.qArray = qArray;
+      this.qoffset = qoffset;
+      this.qlength = qlength;
+    }
+
+    @Override
+    public byte[] getFamilyArray() {
+      return this.fArray;
+    }
+
+    @Override
+    public int getFamilyOffset() {
+      return this.foffset;
+    }
+
+    @Override
+    public byte getFamilyLength() {
+      return this.flength;
+    }
+
+    @Override
+    public byte[] getQualifierArray() {
+      return this.qArray;
+    }
+
+    @Override
+    public int getQualifierOffset() {
+      return this.qoffset;
+    }
+
+    @Override
+    public int getQualifierLength() {
+      return this.qlength;
+    }
+
+    @Override
+    public byte[] getFamily() {
+      return Bytes.copy(this.fArray, this.foffset, this.flength);
+    }
+
+    @Override
+    public byte[] getQualifier() {
+      return Bytes.copy(this.qArray, this.qoffset, this.qlength);
+    }
+  }
+
+  @InterfaceAudience.Private
+  private static class FirstOnRowColumnTSFakeCell extends FirstOnRowColumnFakeCell {
+
+    private long ts;
+
+    public FirstOnRowColumnTSFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
+        int foffset, byte flength, byte[] qArray, int qoffset, int qlength, long ts) {
+      super(rArray, roffset, rlength, fArray, foffset, flength, qArray, qoffset, qlength);
+      this.ts = ts;
+    }
+
+    @Override
+    public long getTimestamp() {
+      return this.ts;
+    }
+  }
+
+  @InterfaceAudience.Private
+  private static class LastOnRowFakeCell extends FakeCell {
+    private final byte[] rowArray;
+    private final int roffset;
+    private final short rlength;
+
+    public LastOnRowFakeCell(byte[] row, int roffset, short rlength) {
+      this.rowArray = row;
+      this.roffset = roffset;
+      this.rlength = rlength;
+    }
+
+    @Override
+    public byte[] getRowArray() {
+      return this.rowArray;
+    }
+
+    @Override
+    public int getRowOffset() {
+      return this.roffset;
+    }
+
+    @Override
+    public short getRowLength() {
+      return this.rlength;
+    }
+
+    @Override
+    public long getTimestamp() {
+      return HConstants.OLDEST_TIMESTAMP;
+    }
+
+    @Override
+    public byte getTypeByte() {
+      return Type.Minimum.getCode();
+    }
+
+    @Override
+    public byte[] getRow() {
+      return Bytes.copy(this.rowArray, this.roffset, this.rlength);
+    }
+  }
+
+  @InterfaceAudience.Private
+  private static class LastOnRowColumnFakeCell extends LastOnRowFakeCell {
+    private final byte[] fArray;
+    private final int foffset;
+    private final byte flength;
+    private final byte[] qArray;
+    private final int qoffset;
+    private final int qlength;
+
+    public LastOnRowColumnFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
+        int foffset, byte flength, byte[] qArray, int qoffset, int qlength) {
+      super(rArray, roffset, rlength);
+      this.fArray = fArray;
+      this.foffset = foffset;
+      this.flength = flength;
+      this.qArray = qArray;
+      this.qoffset = qoffset;
+      this.qlength = qlength;
+    }
+
+    @Override
+    public byte[] getFamilyArray() {
+      return this.fArray;
+    }
+
+    @Override
+    public int getFamilyOffset() {
+      return this.foffset;
+    }
+
+    @Override
+    public byte getFamilyLength() {
+      return this.flength;
+    }
+
+    @Override
+    public byte[] getQualifierArray() {
+      return this.qArray;
+    }
+
+    @Override
+    public int getQualifierOffset() {
+      return this.qoffset;
+    }
+
+    @Override
+    public int getQualifierLength() {
+      return this.qlength;
+    }
+
+    @Override
+    public byte[] getFamily() {
+      return Bytes.copy(this.fArray, this.foffset, this.flength);
+    }
+
+    @Override
+    public byte[] getQualifier() {
+      return Bytes.copy(this.qArray, this.qoffset, this.qlength);
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index 9a935bf..74a26a2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IterableUtils;
-import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
 
@@ -222,26 +221,6 @@ public class KeyValueUtil {
   /*************** next/previous **********************************/
 
   /**
-   * Append single byte 0x00 to the end of the input row key
-   */
-  public static KeyValue createFirstKeyInNextRow(final Cell in){
-    byte[] nextRow = new byte[in.getRowLength() + 1];
-    System.arraycopy(in.getRowArray(), in.getRowOffset(), nextRow, 0, in.getRowLength());
-    nextRow[nextRow.length - 1] = 0;//maybe not necessary
-    return createFirstOnRow(nextRow);
-  }
-
-  /**
-   * Increment the row bytes and clear the other fields
-   */
-  public static KeyValue createFirstKeyInIncrementedRow(final Cell in){
-    byte[] thisRow = new SimpleMutableByteRange(in.getRowArray(), in.getRowOffset(),
-        in.getRowLength()).deepCopyToNewArray();
-    byte[] nextRow = Bytes.unsignedCopyAndIncrement(thisRow);
-    return createFirstOnRow(nextRow);
-  }
-
-  /**
    * Decrement the timestamp.  For tests (currently wasteful)
    *
    * Remember timestamps are sorted reverse chronologically.
@@ -285,54 +264,8 @@ public class KeyValueUtil {
     return new KeyValue(row, roffset, rlength, family, foffset, flength, qualifier, qoffset,
         qlength, HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
   }
-  
-  /**
-   * Creates a keyValue for the specified keyvalue larger than or equal to all other possible
-   * KeyValues that have the same row, family, qualifer.  Used for reseeking
-   * @param kv
-   * @return KeyValue
-   */
-  public static KeyValue createLastOnRow(Cell kv) {
-    return createLastOnRow(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), null, 0, 0,
-        null, 0, 0);
-  }
-
-  /**
-   * Similar to
-   * {@link #createLastOnRow(byte[], int, int, byte[], int, int, byte[], int, int)}
-   * but creates the last key on the row/column of this KV (the value part of
-   * the returned KV is always empty). Used in creating "fake keys" for the
-   * multi-column Bloom filter optimization to skip the row/column we already
-   * know is not in the file.
-   * 
-   * @param kv - cell
-   * @return the last key on the row/column of the given key-value pair
-   */
-  public static KeyValue createLastOnRowCol(Cell kv) {
-    return new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
-        kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(),
-        kv.getQualifierOffset(), kv.getQualifierLength(), HConstants.OLDEST_TIMESTAMP,
-        Type.Minimum, null, 0, 0);
-  }
 
   /**
-   * Creates the first KV with the row/family/qualifier of this KV and the given
-   * timestamp. Uses the "maximum" KV type that guarantees that the new KV is
-   * the lowest possible for this combination of row, family, qualifier, and
-   * timestamp. This KV's own timestamp is ignored. While this function copies
-   * the value from this KV, it is normally used on key-only KVs.
-   * 
-   * @param kv - cell
-   * @param ts
-   */
-  public static KeyValue createFirstOnRowColTS(Cell kv, long ts) {
-    return new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
-        kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), kv.getQualifierArray(),
-        kv.getQualifierOffset(), kv.getQualifierLength(), ts, Type.Maximum, kv.getValueArray(),
-        kv.getValueOffset(), kv.getValueLength());
-  }
-  
-  /**
    * Create a KeyValue that is smaller than all other possible KeyValues
    * for the given row. That is any (valid) KeyValue on 'row' would sort
    * _after_ the result.
@@ -344,7 +277,6 @@ public class KeyValueUtil {
     return new KeyValue(row, roffset, rlength,
         null, 0, 0, null, 0, 0, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
   }
-  
 
   /**
    * Creates a KeyValue that is last on the specified row id. That is,
@@ -516,21 +448,6 @@ public class KeyValueUtil {
     return new KeyValue(buffer, boffset, len);
   }
 
-  /**
-   * Creates the first KV with the row/family/qualifier of this KV and the
-   * given timestamp. Uses the "maximum" KV type that guarantees that the new
-   * KV is the lowest possible for this combination of row, family, qualifier,
-   * and timestamp. This KV's own timestamp is ignored. While this function
-   * copies the value from this KV, it is normally used on key-only KVs.
-   */
-  public static KeyValue createFirstOnRowColTS(KeyValue kv, long ts) {
-    return new KeyValue(
-        kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
-        kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
-        kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
-        ts, Type.Maximum, kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
-  }
-
   /*************** misc **********************************/
   /**
    * @param cell

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
index 9d03d89..2ed89f1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
@@ -68,6 +68,7 @@ public class CellCodec implements Codec {
      */
     private void write(final byte [] bytes, final int offset, final int length)
     throws IOException {
+      // TODO add BB backed os check and do for write. Pass Cell
       this.out.write(Bytes.toBytes(length));
       this.out.write(bytes, offset, length);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 3dbb9be..7bb72a0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -21,6 +21,8 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -28,6 +30,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
 
+import sun.nio.ch.DirectBuffer;
+
 /**
  * Utility functions for working with byte buffers, such as reading/writing
  * variable-length long numbers.
@@ -511,7 +515,21 @@ public final class ByteBufferUtils {
 
   public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
     if (UnsafeAccess.isAvailable()) {
-      return compareToUnsafe(buf1, o1, l1, buf2, o2, l2);
+      long offset1Adj, offset2Adj;
+      Object refObj1 = null, refObj2 = null;
+      if (buf1.hasArray()) {
+        offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
+        refObj1 = buf1.array();
+      } else {
+        offset1Adj = o1 + ((DirectBuffer) buf1).address();
+      }
+      if (buf2.hasArray()) {
+        offset2Adj = o2 + buf2.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
+        refObj2 = buf2.array();
+      } else {
+        offset2Adj = o2 + ((DirectBuffer) buf2).address();
+      }
+      return compareToUnsafe(refObj1, offset1Adj, l1, refObj2, offset2Adj, l2);
     }
     int end1 = o1 + l1;
     int end2 = o2 + l2;
@@ -525,7 +543,32 @@ public final class ByteBufferUtils {
     return l1 - l2;
   }
 
-  static int compareToUnsafe(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
+  public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
+    if (UnsafeAccess.isAvailable()) {
+      long offset1Adj;
+      Object refObj1 = null;
+      if (buf1.hasArray()) {
+        offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
+        refObj1 = buf1.array();
+      } else {
+        offset1Adj = o1 + ((DirectBuffer) buf1).address();
+      }
+      return compareToUnsafe(refObj1, offset1Adj, l1, buf2, o2
+          + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l2);
+    }
+    int end1 = o1 + l1;
+    int end2 = o2 + l2;
+    for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
+      int a = buf1.get(i) & 0xFF;
+      int b = buf2[j] & 0xFF;
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return l1 - l2;
+  }
+
+  static int compareToUnsafe(Object obj1, long o1, int l1, Object obj2, long o2, int l2) {
     final int minLength = Math.min(l1, l2);
     final int minWords = minLength / Bytes.SIZEOF_LONG;
 
@@ -536,8 +579,8 @@ public final class ByteBufferUtils {
      */
     int j = minWords << 3; // Same as minWords * SIZEOF_LONG
     for (int i = 0; i < j; i += Bytes.SIZEOF_LONG) {
-      long lw = UnsafeAccess.getAsLong(buf1, o1 + i);
-      long rw = UnsafeAccess.getAsLong(buf2, o2 + i);
+      long lw = UnsafeAccess.theUnsafe.getLong(obj1, o1 + i);
+      long rw = UnsafeAccess.theUnsafe.getLong(obj2, o2 + i);
       long diff = lw ^ rw;
       if (diff != 0) {
         return lessThanUnsignedLong(lw, rw) ? -1 : 1;
@@ -546,24 +589,24 @@ public final class ByteBufferUtils {
     int offset = j;
 
     if (minLength - offset >= Bytes.SIZEOF_INT) {
-      int il = UnsafeAccess.getAsInt(buf1, o1 + offset);
-      int ir = UnsafeAccess.getAsInt(buf2, o2 + offset);
+      int il = UnsafeAccess.theUnsafe.getInt(obj1, o1 + offset);
+      int ir = UnsafeAccess.theUnsafe.getInt(obj2, o2 + offset);
       if (il != ir) {
         return lessThanUnsignedInt(il, ir) ? -1 : 1;
       }
       offset += Bytes.SIZEOF_INT;
     }
     if (minLength - offset >= Bytes.SIZEOF_SHORT) {
-      short sl = UnsafeAccess.getAsShort(buf1, o1 + offset);
-      short sr = UnsafeAccess.getAsShort(buf2, o2 + offset);
+      short sl = UnsafeAccess.theUnsafe.getShort(obj1, o1 + offset);
+      short sr = UnsafeAccess.theUnsafe.getShort(obj2, o2 + offset);
       if (sl != sr) {
         return lessThanUnsignedShort(sl, sr) ? -1 : 1;
       }
       offset += Bytes.SIZEOF_SHORT;
     }
     if (minLength - offset == 1) {
-      int a = (buf1.get(o1 + offset) & 0xff);
-      int b = (buf2.get(o2 + offset) & 0xff);
+      int a = (UnsafeAccess.theUnsafe.getByte(obj1, o1 + offset) & 0xff);
+      int b = (UnsafeAccess.theUnsafe.getByte(obj2, o2 + offset) & 0xff);
       if (a != b) {
         return a - b;
       }
@@ -665,6 +708,34 @@ public final class ByteBufferUtils {
   }
 
   /**
+   * Reads a double value at the given buffer's offset.
+   * @param buffer
+   * @param offset offset where double is
+   * @return double value at offset
+   */
+  public static double toDouble(ByteBuffer buffer, int offset) {
+    return Double.longBitsToDouble(toLong(buffer, offset));
+  }
+
+  /**
+   * Reads a BigDecimal value at the given buffer's offset.
+   * @param buffer
+   * @param offset
+   * @return BigDecimal value at offset
+   */
+  public static BigDecimal toBigDecimal(ByteBuffer buffer, int offset, int length) {
+    if (buffer == null || length < Bytes.SIZEOF_INT + 1 ||
+      (offset + length > buffer.limit())) {
+      return null;
+    }
+
+    int scale = toInt(buffer, offset);
+    byte[] tcBytes = new byte[length - Bytes.SIZEOF_INT];
+    copyFromBufferToArray(tcBytes, buffer, offset + Bytes.SIZEOF_INT, 0, length - Bytes.SIZEOF_INT);
+    return new BigDecimal(new BigInteger(tcBytes), scale);
+  }
+
+  /**
    * Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes
    * to buffer's current position. This also advances the position in the 'out' buffer by 'length'
    * @param out

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index 1f5e206..80fc33e 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.hbase;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.nio.ByteBuffer;
+
 import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -87,4 +91,31 @@ public class TestCellComparator {
     kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
     assertTrue((comparator.compare(kv1, kv2.getKey(), 0, kv2.getKey().length)) == 0);
   }
+
+  @Test
+  public void testCompareByteBufferedCell() {
+    byte[] r1 = Bytes.toBytes("row1");
+    byte[] r2 = Bytes.toBytes("row2");
+    byte[] f1 = Bytes.toBytes("cf1");
+    byte[] q1 = Bytes.toBytes("qual1");
+    byte[] q2 = Bytes.toBytes("qual2");
+    byte[] v = Bytes.toBytes("val1");
+    KeyValue kv = new KeyValue(r1, f1, q1, v);
+    ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    kv = new KeyValue(r2, f1, q1, v);
+    buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    assertEquals(0, CellComparator.compareColumns(bbCell1, bbCell2));
+    assertEquals(0, CellComparator.compareColumns(bbCell1, kv));
+    kv = new KeyValue(r2, f1, q2, v);
+    buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell3 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    assertEquals(0, CellComparator.compareFamilies(bbCell2, bbCell3));
+    assertTrue(CellComparator.compareQualifiers(bbCell2, bbCell3) < 0);
+    assertTrue(CellComparator.compareColumns(bbCell2, bbCell3) < 0);
+
+    assertEquals(0, CellComparator.COMPARATOR.compareRows(bbCell2, bbCell3));
+    assertTrue(CellComparator.COMPARATOR.compareRows(bbCell1, bbCell2) < 0);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
index fea517f..bfe5b1a 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
@@ -29,6 +31,7 @@ import java.util.TreeMap;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -431,4 +434,283 @@ public class TestCellUtil {
 
     // TODO: test with tags
   }
+
+  @Test
+  public void testCloneCellFieldsFromByteBufferedCell() {
+    byte[] r = Bytes.toBytes("row1");
+    byte[] f = Bytes.toBytes("cf1");
+    byte[] q = Bytes.toBytes("qual1");
+    byte[] v = Bytes.toBytes("val1");
+    byte[] tags = Bytes.toBytes("tag1");
+    KeyValue kv = new KeyValue(r, f, q, 0, q.length, 1234L, Type.Put, v, 0, v.length, tags);
+    ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    byte[] rDest = CellUtil.cloneRow(bbCell);
+    assertTrue(Bytes.equals(r, rDest));
+    byte[] fDest = CellUtil.cloneFamily(bbCell);
+    assertTrue(Bytes.equals(f, fDest));
+    byte[] qDest = CellUtil.cloneQualifier(bbCell);
+    assertTrue(Bytes.equals(q, qDest));
+    byte[] vDest = CellUtil.cloneValue(bbCell);
+    assertTrue(Bytes.equals(v, vDest));
+    byte[] tDest = new byte[tags.length];
+    CellUtil.copyTagTo(bbCell, tDest, 0);
+    assertTrue(Bytes.equals(tags, tDest));
+  }
+
+  @Test
+  public void testMatchingCellFieldsFromByteBufferedCell() {
+    byte[] r = Bytes.toBytes("row1");
+    byte[] f = Bytes.toBytes("cf1");
+    byte[] q1 = Bytes.toBytes("qual1");
+    byte[] q2 = Bytes.toBytes("qual2");
+    byte[] v = Bytes.toBytes("val1");
+    byte[] tags = Bytes.toBytes("tag1");
+    KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
+    ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    kv = new KeyValue(r, f, q2, 0, q2.length, 1234L, Type.Put, v, 0, v.length, tags);
+    buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
+    assertTrue(CellUtil.matchingRows(kv, bbCell2));
+    assertTrue(CellUtil.matchingRow(bbCell1, r));
+    assertTrue(CellUtil.matchingFamily(bbCell1, bbCell2));
+    assertTrue(CellUtil.matchingFamily(kv, bbCell2));
+    assertTrue(CellUtil.matchingFamily(bbCell1, f));
+    assertFalse(CellUtil.matchingQualifier(bbCell1, bbCell2));
+    assertTrue(CellUtil.matchingQualifier(kv, bbCell2));
+    assertTrue(CellUtil.matchingQualifier(bbCell1, q1));
+    assertTrue(CellUtil.matchingQualifier(bbCell2, q2));
+    assertTrue(CellUtil.matchingValue(bbCell1, bbCell2));
+    assertTrue(CellUtil.matchingValue(kv, bbCell2));
+    assertTrue(CellUtil.matchingValue(bbCell1, v));
+    assertFalse(CellUtil.matchingColumn(bbCell1, bbCell2));
+    assertTrue(CellUtil.matchingColumn(kv, bbCell2));
+    assertTrue(CellUtil.matchingColumn(bbCell1, f, q1));
+    assertTrue(CellUtil.matchingColumn(bbCell2, f, q2));
+  }
+
+  @Test
+  public void testCellFieldsAsPrimitiveTypesFromByteBufferedCell() {
+    int ri = 123;
+    byte[] r = Bytes.toBytes(ri);
+    byte[] f = Bytes.toBytes("cf1");
+    byte[] q = Bytes.toBytes("qual1");
+    long vl = 10981L;
+    byte[] v = Bytes.toBytes(vl);
+    KeyValue kv = new KeyValue(r, f, q, v);
+    ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
+    Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    assertEquals(ri, CellUtil.getRowAsInt(bbCell));
+    assertEquals(vl, CellUtil.getValueAsLong(bbCell));
+    double vd = 3005.5;
+    v = Bytes.toBytes(vd);
+    kv = new KeyValue(r, f, q, v);
+    buffer = ByteBuffer.wrap(kv.getBuffer());
+    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    assertEquals(vd, CellUtil.getValueAsDouble(bbCell), 0.0);
+    BigDecimal bd = new BigDecimal(9999);
+    v = Bytes.toBytes(bd);
+    kv = new KeyValue(r, f, q, v);
+    buffer = ByteBuffer.wrap(kv.getBuffer());
+    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
+  }
+
+  // TODO remove this test impl once we have a Cell implementation backed by ByteBuffer
+  public static class ByteBufferedCellImpl extends ByteBufferedCell {
+
+    private final ByteBuffer buffer;
+    private final int offset, length;
+
+    public ByteBufferedCellImpl(ByteBuffer buffer, int offset, int length) {
+      this.buffer = buffer;
+      this.offset = offset;
+      this.length = length;
+    }
+
+    @Override
+    public byte[] getRowArray() {
+      return CellUtil.cloneRow(this);
+    }
+
+    @Override
+    public int getRowOffset() {
+      return 0;
+    }
+
+    @Override
+    public short getRowLength() {
+      return ByteBufferUtils.toShort(this.buffer, this.offset + KeyValue.ROW_OFFSET);
+    }
+
+    @Override
+    public byte[] getFamilyArray() {
+      return CellUtil.cloneFamily(this);
+    }
+
+    @Override
+    public int getFamilyOffset() {
+      return 0;
+    }
+
+    @Override
+    public byte getFamilyLength() {
+      return this.buffer.get(getFamilyPositionInByteBuffer() - 1);
+    }
+
+    @Override
+    public byte[] getQualifierArray() {
+      return CellUtil.cloneQualifier(this);
+    }
+
+    @Override
+    public int getQualifierOffset() {
+      return 0;
+    }
+
+    @Override
+    public int getQualifierLength() {
+      return getKeyLength()
+          - (int) KeyValue.getKeyDataStructureSize(getRowLength(), getFamilyLength(), 0);
+    }
+
+    private int getKeyLength() {
+      return ByteBufferUtils.toInt(this.buffer, this.offset);
+    }
+
+    @Override
+    public long getTimestamp() {
+      int tsOffset = this.offset + KeyValue.ROW_OFFSET + getKeyLength()
+          - KeyValue.TIMESTAMP_TYPE_SIZE;
+      return ByteBufferUtils.toLong(buffer, tsOffset);
+    }
+
+    @Override
+    public byte getTypeByte() {
+      return KeyValue.Type.Put.getCode();
+    }
+
+    @Override
+    public long getMvccVersion() {
+      return 0;
+    }
+
+    @Override
+    public long getSequenceId() {
+      return 0;
+    }
+
+    @Override
+    public byte[] getValueArray() {
+      return CellUtil.cloneValue(this);
+    }
+
+    @Override
+    public int getValueOffset() {
+      return 0;
+    }
+
+    @Override
+    public int getValueLength() {
+      return ByteBufferUtils.toInt(this.buffer, this.offset + KeyValue.KEY_LENGTH_SIZE);
+    }
+
+    @Override
+    public byte[] getTagsArray() {
+      byte[] tDest = new byte[getTagsLength()];
+      CellUtil.copyTagTo(this, tDest, 0);
+      return tDest;
+    }
+
+    @Override
+    public int getTagsOffset() {
+      return 0;
+    }
+
+    @Override
+    public int getTagsLength() {
+      int tagsLen = this.length
+          - (getKeyLength() + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
+      if (tagsLen > 0) {
+        tagsLen -= KeyValue.TAGS_LENGTH_SIZE;
+      }
+      return tagsLen;
+    }
+
+    @Override
+    public byte[] getValue() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public byte[] getFamily() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public byte[] getQualifier() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public byte[] getRow() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    ByteBuffer getRowByteBuffer() {
+      return this.buffer;
+    }
+
+    @Override
+    int getRowPositionInByteBuffer() {
+      return this.offset + KeyValue.ROW_KEY_OFFSET;
+    }
+
+    @Override
+    ByteBuffer getFamilyByteBuffer() {
+      return this.buffer;
+    }
+
+    @Override
+    int getFamilyPositionInByteBuffer() {
+      return this.offset + KeyValue.ROW_KEY_OFFSET + getRowLength() + Bytes.SIZEOF_BYTE;
+    }
+
+    @Override
+    ByteBuffer getQualifierByteBuffer() {
+      return this.buffer;
+    }
+
+    @Override
+    int getQualifierPositionInByteBuffer() {
+      return getFamilyPositionInByteBuffer() + getFamilyLength();
+    }
+
+    @Override
+    ByteBuffer getValueByteBuffer() {
+      return this.buffer;
+    }
+
+    @Override
+    int getValuePositionInByteBuffer() {
+      return this.offset + KeyValue.ROW_OFFSET + getKeyLength();
+    }
+
+    @Override
+    ByteBuffer getTagsByteBuffer() {
+      return this.buffer;
+    }
+
+    @Override
+    int getTagsPositionInByteBuffer() {
+      int tagsLen = getTagsLength();
+      if (tagsLen == 0) {
+        return this.offset + this.length;
+      }
+      return this.offset + this.length - tagsLen;
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java
index c005c17..afe30e2 100644
--- a/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java
+++ b/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java
@@ -144,7 +144,7 @@ public class TestPrefixTreeSearcher {
           KeyValue kv = rows.getInputs().get(i);
 
           //nextRow
-          KeyValue inputNextRow = KeyValueUtil.createFirstKeyInNextRow(kv);
+          Cell inputNextRow = CellUtil.createFirstOnNextRow(kv);
 
           CellScannerPosition position = beforeVsAfterOnMiss
               ? searcher.positionAtOrBefore(inputNextRow)

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
index 33bb699..f47c9f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
@@ -439,8 +439,6 @@ public abstract class BaseRegionObserver implements RegionObserver {
   @Override
   public boolean postScannerFilterRow(final ObserverContext<RegionCoprocessorEnvironment> e,
       final InternalScanner s, final Cell curRowCell, final boolean hasMore) 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 postScannerFilterRow(e, s, curRowCell.getRowArray(), curRowCell.getRowOffset(),
         curRowCell.getRowLength(), hasMore);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d07ff5ec/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index fcb264d..6a5a9d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -664,12 +664,8 @@ public class HFileWriterImpl implements HFile.Writer {
    */
   @Override
   public void append(final Cell cell) throws IOException {
-    byte[] value = cell.getValueArray();
-    int voffset = cell.getValueOffset();
-    int vlength = cell.getValueLength();
     // checkKey uses comparator to check we are writing in order.
     boolean dupKey = checkKey(cell);
-    checkValue(value, voffset, vlength);
     if (!dupKey) {
       checkBlockBoundary();
     }
@@ -681,7 +677,7 @@ public class HFileWriterImpl implements HFile.Writer {
     fsBlockWriter.write(cell);
 
     totalKeyLength += CellUtil.estimatedSerializedSizeOfKey(cell);
-    totalValueLength += vlength;
+    totalValueLength += cell.getValueLength();
 
     // Are we the first key in this block?
     if (firstCellInBlock == null) {