You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2020/12/28 07:33:51 UTC

[hbase] branch branch-2.3 updated: HBASE-24850 CellComparator perf improvement (#2776)

This is an automated email from the ASF dual-hosted git repository.

ramkrishna pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 0448c56  HBASE-24850 CellComparator perf improvement (#2776)
0448c56 is described below

commit 0448c560caba8a2722f2b74a4c10fb70a8acc392
Author: ramkrish86 <ra...@hotmail.com>
AuthorDate: Mon Dec 28 13:03:28 2020 +0530

    HBASE-24850 CellComparator perf improvement (#2776)
    
    * Using ContiguousCellFormat as a marker alone
    
    * Commit the new file
    
    * Fix the comparator logic that was an oversight
    
    * Fix the sequenceId check order
    
    * Adding few more static methods that helps in scan flow like query
    matcher where we have more cols
    
    * Remove ContiguousCellFormat and ensure compare() can be inlined
    
    * applying negation as per review comment
    
    * Fix checkstyle comments
    
    * fix review comments
    
    * Address review comments
    
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: AnoopSamJohn <an...@apache.org>
    Signed-off-by: huaxiangsun <hu...@apache.org>
---
 .../org/apache/hadoop/hbase/BBKVComparator.java    | 173 -------
 .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java    |  12 +-
 .../apache/hadoop/hbase/CellComparatorImpl.java    | 522 +++++++++++++++++++--
 .../java/org/apache/hadoop/hbase/CellUtil.java     |  50 +-
 .../java/org/apache/hadoop/hbase/KeyValue.java     |  56 ++-
 .../hadoop/hbase/TestByteBufferKeyValue.java       |   2 +-
 .../hbase/regionserver/DataBlockEncodingTool.java  |   3 +-
 7 files changed, 587 insertions(+), 231 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/BBKVComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/BBKVComparator.java
deleted file mode 100644
index bc76a9d..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/BBKVComparator.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * 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.util.Comparator;
-
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.primitives.Longs;
-
-/**
- * A comparator for case where {@link ByteBufferKeyValue} is prevalent type (BBKV
- * is base-type in hbase2). Takes a general comparator as fallback in case types are NOT the
- * expected ByteBufferKeyValue.
- *
- * <p>This is a tricked-out Comparator at heart of hbase read and write. It is in
- * the HOT path so we try all sorts of ugly stuff so we can go faster. See below
- * in this javadoc comment for the list.
- *
- * <p>Apply this comparator narrowly so it is fed exclusively ByteBufferKeyValues
- * as much as is possible so JIT can settle (e.g. make one per ConcurrentSkipListMap
- * in HStore).
- *
- * <p>Exploits specially added methods in BBKV to save on deserializations of shorts,
- * longs, etc: i.e. calculating the family length requires row length; pass it in
- * rather than recalculate it, and so on.
- *
- * <p>This comparator does static dispatch to private final methods so hotspot is comfortable
- * deciding inline.
- *
- * <p>Measurement has it that we almost have it so all inlines from memstore
- * ConcurrentSkipListMap on down to the (unsafe) intrinisics that do byte compare
- * and deserialize shorts and ints; needs a bit more work.
- *
- * <p>Does not take a Type to compare: i.e. it is not a Comparator&lt;Cell> or
- * CellComparator&lt;Cell> or Comparator&lt;ByteBufferKeyValue> because that adds
- * another method to the hierarchy -- from compare(Object, Object)
- * to dynamic compare(Cell, Cell) to static private compare -- and inlining doesn't happen if
- * hierarchy is too deep (it is the case here).
- *
- * <p>Be careful making changes. Compare perf before and after and look at what
- * hotspot ends up generating before committing change (jitwatch is helpful here).
- * Changing this one class doubled write throughput (HBASE-20483).
- */
-@InterfaceAudience.Private
-public class BBKVComparator implements Comparator {
-  protected static final Logger LOG = LoggerFactory.getLogger(BBKVComparator.class);
-  private final Comparator fallback;
-
-  public BBKVComparator(Comparator fallback) {
-    this.fallback = fallback;
-  }
-
-  @Override
-  public int compare(Object l, Object r) {
-    if ((l instanceof ByteBufferKeyValue) && (r instanceof ByteBufferKeyValue)) {
-      return compare((ByteBufferKeyValue)l, (ByteBufferKeyValue)r, false);
-    }
-    // Skip calling compare(Object, Object) and go direct to compare(Cell, Cell)
-    return this.fallback.compare((Cell)l, (Cell)r);
-  }
-
-  // TODO: Come back here. We get a few percentage points extra of throughput if this is a
-  // private method.
-  static int compare(ByteBufferKeyValue left, ByteBufferKeyValue right,
-      boolean ignoreSequenceid) {
-    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
-    // sharing gets us a few percent more throughput in compares. If changes here or there, make
-    // sure done in both places.
-
-    // Compare Rows. Cache row length.
-    int leftRowLength = left.getRowLength();
-    int rightRowLength = right.getRowLength();
-    int diff = ByteBufferUtils.compareTo(left.getRowByteBuffer(), left.getRowPosition(),
-        leftRowLength,
-        right.getRowByteBuffer(), right.getRowPosition(), rightRowLength);
-    if (diff != 0) {
-      return diff;
-    }
-
-    // If the column is not specified, the "minimum" key type appears as latest in the sorted
-    // order, regardless of the timestamp. This is used for specifying the last key/value in a
-    // given row, because there is no "lexicographically last column" (it would be infinitely long).
-    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in that
-    // we can't do memcmp w/ special rules like this.
-    // TODO: Is there a test for this behavior?
-    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
-    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
-    int leftKeyLength = left.getKeyLength();
-    int leftQualifierLength = left.getQualifierLength(leftKeyLength, leftRowLength,
-        leftFamilyLength);
-
-    // No need of left row length below here.
-
-    byte leftType = left.getTypeByte(leftKeyLength);
-    if (leftFamilyLength + leftQualifierLength == 0 &&
-        leftType == KeyValue.Type.Minimum.getCode()) {
-      // left is "bigger", i.e. it appears later in the sorted order
-      return 1;
-    }
-
-    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
-    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
-    int rightKeyLength = right.getKeyLength();
-    int rightQualifierLength = right.getQualifierLength(rightKeyLength, rightRowLength,
-        rightFamilyLength);
-
-   // No need of right row length below here.
-
-    byte rightType = right.getTypeByte(rightKeyLength);
-    if (rightFamilyLength + rightQualifierLength == 0 &&
-        rightType == KeyValue.Type.Minimum.getCode()) {
-      return -1;
-    }
-
-    // Compare families.
-    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
-    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
-    diff = ByteBufferUtils.compareTo(left.getFamilyByteBuffer(), leftFamilyPosition,
-        leftFamilyLength,
-        right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
-    if (diff != 0) {
-      return diff;
-    }
-
-    // Compare qualifiers
-    diff = ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
-        left.getQualifierPosition(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
-        right.getQualifierByteBuffer(),
-        right.getQualifierPosition(rightFamilyPosition, rightFamilyLength),
-        rightQualifierLength);
-    if (diff != 0) {
-      return diff;
-    }
-
-    // Timestamps.
-    // Swap order we pass into compare so we get DESCENDING order.
-    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
-    if (diff != 0) {
-      return diff;
-    }
-
-    // Compare types. Let the delete types sort ahead of puts; i.e. types
-    // of higher numbers sort before those of lesser numbers. Maximum (255)
-    // appears ahead of everything, and minimum (0) appears after
-    // everything.
-    diff = (0xff & rightType) - (0xff & leftType);
-    if (diff != 0) {
-      return diff;
-    }
-
-    // Negate following comparisons so later edits show up first mvccVersion: later sorts first
-    return ignoreSequenceid ? diff : Longs.compare(right.getSequenceId(), left.getSequenceId());
-  }
-}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
index cc7e8d7..d557337 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
@@ -161,7 +161,11 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
 
   @Override
   public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+    return getTypeByte(this.length);
+  }
+
+  byte getTypeByte(int keyLen) {
+    return ByteBufferUtils.toByte(this.buf, this.offset + keyLen - 1);
   }
 
   @Override
@@ -236,7 +240,11 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
 
   // The position in BB where the family length is added.
   private int getFamilyLengthPosition() {
-    return this.offset + Bytes.SIZEOF_SHORT + getRowLength();
+    return getFamilyLengthPosition(getRowLength());
+  }
+
+  int getFamilyLengthPosition(int rowLength) {
+    return this.offset + Bytes.SIZEOF_SHORT + rowLength;
   }
 
   @Override
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
index 4af035a..a6be237 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
@@ -57,29 +57,286 @@ public class CellComparatorImpl implements CellComparator {
   }
 
   @Override
-  public int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
-
+  public int compare(final Cell l, final Cell r, boolean ignoreSequenceid) {
     int diff = 0;
     // "Peel off" the most common path.
-    if (a instanceof ByteBufferKeyValue && b instanceof ByteBufferKeyValue) {
-      diff = BBKVComparator.compare((ByteBufferKeyValue)a, (ByteBufferKeyValue)b, ignoreSequenceid);
+    if (l instanceof KeyValue && r instanceof KeyValue) {
+      diff = compareKeyValues((KeyValue) l, (KeyValue) r);
+      if (diff != 0) {
+        return diff;
+      }
+    } else if (l instanceof KeyValue && r instanceof ByteBufferKeyValue) {
+      diff = compareKVVsBBKV((KeyValue) l, (ByteBufferKeyValue) r);
+      if (diff != 0) {
+        return diff;
+      }
+    } else if (l instanceof ByteBufferKeyValue && r instanceof KeyValue) {
+      diff = compareKVVsBBKV((KeyValue) r, (ByteBufferKeyValue) l);
+      if (diff != 0) {
+        // negate- Findbugs will complain?
+        return -diff;
+      }
+    } else if (l instanceof ByteBufferKeyValue && r instanceof ByteBufferKeyValue) {
+      diff = compareBBKV((ByteBufferKeyValue) l, (ByteBufferKeyValue) r);
       if (diff != 0) {
         return diff;
       }
     } else {
-      diff = compareRows(a, b);
+      int leftRowLength = l.getRowLength();
+      int rightRowLength = r.getRowLength();
+      diff = compareRows(l, leftRowLength, r, rightRowLength);
       if (diff != 0) {
         return diff;
       }
 
-      diff = compareWithoutRow(a, b);
+      diff = compareWithoutRow(l, r);
       if (diff != 0) {
         return diff;
       }
     }
-
     // Negate following comparisons so later edits show up first mvccVersion: later sorts first
-    return ignoreSequenceid? diff: Long.compare(b.getSequenceId(), a.getSequenceId());
+    return ignoreSequenceid ? diff : Long.compare(r.getSequenceId(), l.getSequenceId());
+  }
+
+  private static int compareKeyValues(final KeyValue left, final KeyValue right) {
+    int diff;
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+    diff = Bytes.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
+      right.getRowArray(), right.getRowOffset(), rightRowLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // If the column is not specified, the "minimum" key type appears as latest in the sorted
+    // order, regardless of the timestamp. This is used for specifying the last key/value in a
+    // given row, because there is no "lexicographically last column" (it would be infinitely
+    // long).
+    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in
+    // that
+    // we can't do memcmp w/ special rules like this.
+    // TODO: Is there a test for this behavior?
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    byte leftType = left.getTypeByte(leftKeyLength);
+    if (leftType == KeyValue.Type.Minimum.getCode()
+        && leftFamilyLength + leftQualifierLength == 0) {
+      // left is "bigger", i.e. it appears later in the sorted order
+      return 1;
+    }
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // No need of right row length below here.
+
+    byte rightType = right.getTypeByte(rightKeyLength);
+    if (rightType == KeyValue.Type.Minimum.getCode()
+        && rightFamilyLength + rightQualifierLength == 0) {
+      return -1;
+    }
+
+    // Compare families.
+    int leftFamilyPosition = left.getFamilyOffset(leftFamilyLengthPosition);
+    int rightFamilyPosition = right.getFamilyOffset(rightFamilyLengthPosition);
+    diff = Bytes.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
+      right.getFamilyArray(), rightFamilyPosition, rightFamilyLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Compare qualifiers
+    diff = Bytes.compareTo(left.getQualifierArray(),
+      left.getQualifierOffset(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
+      right.getQualifierArray(), right.getQualifierOffset(rightFamilyPosition, rightFamilyLength),
+      rightQualifierLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Timestamps.
+    // Swap order we pass into compare so we get DESCENDING order.
+    // TODO : Ensure we read the bytes and do the compare instead of the value.
+    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Compare types. Let the delete types sort ahead of puts; i.e. types
+    // of higher numbers sort before those of lesser numbers. Maximum (255)
+    // appears ahead of everything, and minimum (0) appears after
+    // everything.
+    return (0xff & rightType) - (0xff & leftType);
+  }
+
+  private static int compareBBKV(final ByteBufferKeyValue left, final ByteBufferKeyValue right) {
+    int diff;
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+    diff = ByteBufferUtils.compareTo(left.getRowByteBuffer(), left.getRowPosition(),
+      leftRowLength, right.getRowByteBuffer(), right.getRowPosition(), rightRowLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // If the column is not specified, the "minimum" key type appears as latest in the sorted
+    // order, regardless of the timestamp. This is used for specifying the last key/value in a
+    // given row, because there is no "lexicographically last column" (it would be infinitely
+    // long).
+    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in
+    // that
+    // we can't do memcmp w/ special rules like this.
+    // TODO: Is there a test for this behavior?
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    byte leftType = left.getTypeByte(leftKeyLength);
+    if (leftType == KeyValue.Type.Minimum.getCode()
+        && leftFamilyLength + leftQualifierLength == 0) {
+      // left is "bigger", i.e. it appears later in the sorted order
+      return 1;
+    }
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // No need of right row length below here.
+
+    byte rightType = right.getTypeByte(rightKeyLength);
+    if (rightType == KeyValue.Type.Minimum.getCode()
+        && rightFamilyLength + rightQualifierLength == 0) {
+      return -1;
+    }
+
+    // Compare families.
+    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
+    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
+    diff = ByteBufferUtils.compareTo(left.getFamilyByteBuffer(), leftFamilyPosition,
+      leftFamilyLength, right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Compare qualifiers
+    diff = ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
+      left.getQualifierPosition(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
+      right.getQualifierByteBuffer(),
+      right.getQualifierPosition(rightFamilyPosition, rightFamilyLength), rightQualifierLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Timestamps.
+    // Swap order we pass into compare so we get DESCENDING order.
+    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Compare types. Let the delete types sort ahead of puts; i.e. types
+    // of higher numbers sort before those of lesser numbers. Maximum (255)
+    // appears ahead of everything, and minimum (0) appears after
+    // everything.
+    return (0xff & rightType) - (0xff & leftType);
+  }
+
+  private static int compareKVVsBBKV(final KeyValue left, final ByteBufferKeyValue right) {
+    int diff;
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+    diff = ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
+      right.getRowByteBuffer(), right.getRowPosition(), rightRowLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // If the column is not specified, the "minimum" key type appears as latest in the sorted
+    // order, regardless of the timestamp. This is used for specifying the last key/value in a
+    // given row, because there is no "lexicographically last column" (it would be infinitely
+    // long).
+    // The "maximum" key type does not need this behavior. Copied from KeyValue. This is bad in
+    // that
+    // we can't do memcmp w/ special rules like this.
+    // TODO: Is there a test for this behavior?
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    int leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    byte leftType = left.getTypeByte(leftKeyLength);
+    if (leftType == KeyValue.Type.Minimum.getCode()
+        && leftFamilyLength + leftQualifierLength == 0) {
+      // left is "bigger", i.e. it appears later in the sorted order
+      return 1;
+    }
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    int rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // No need of right row length below here.
+
+    byte rightType = right.getTypeByte(rightKeyLength);
+    if (rightType == KeyValue.Type.Minimum.getCode()
+        && rightFamilyLength + rightQualifierLength == 0) {
+      return -1;
+    }
+
+    // Compare families.
+    int leftFamilyPosition = left.getFamilyOffset(leftFamilyLengthPosition);
+    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
+    diff = ByteBufferUtils.compareTo(left.getFamilyArray(), leftFamilyPosition, leftFamilyLength,
+      right.getFamilyByteBuffer(), rightFamilyPosition, rightFamilyLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Compare qualifiers
+    diff = ByteBufferUtils.compareTo(left.getQualifierArray(),
+      left.getQualifierOffset(leftFamilyPosition, leftFamilyLength), leftQualifierLength,
+      right.getQualifierByteBuffer(),
+      right.getQualifierPosition(rightFamilyPosition, rightFamilyLength), rightQualifierLength);
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Timestamps.
+    // Swap order we pass into compare so we get DESCENDING order.
+    diff = Long.compare(right.getTimestamp(rightKeyLength), left.getTimestamp(leftKeyLength));
+    if (diff != 0) {
+      return diff;
+    }
+
+    // Compare types. Let the delete types sort ahead of puts; i.e. types
+    // of higher numbers sort before those of lesser numbers. Maximum (255)
+    // appears ahead of everything, and minimum (0) appears after
+    // everything.
+    return (0xff & rightType) - (0xff & leftType);
   }
 
   /**
@@ -94,6 +351,65 @@ public class CellComparatorImpl implements CellComparator {
     return compareQualifiers(left, right);
   }
 
+  private int compareColumns(final Cell left, final int leftFamLen, final int leftQualLen,
+      final Cell right, final int rightFamLen, final int rightQualLen) {
+    int diff = compareFamilies(left, leftFamLen, right, rightFamLen);
+    if (diff != 0) {
+      return diff;
+    }
+    return compareQualifiers(left, leftQualLen, right, rightQualLen);
+  }
+
+  private int compareFamilies(Cell left, int leftFamLen, Cell right, int rightFamLen) {
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), leftFamLen,
+        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) right).getFamilyPosition(), rightFamLen);
+    }
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), leftFamLen, right.getFamilyArray(),
+        right.getFamilyOffset(), rightFamLen);
+    }
+    if (right instanceof ByteBufferExtendedCell) {
+      // Notice how we flip the order of the compare here. We used to negate the return value but
+      // see what FindBugs says
+      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
+      // It suggest flipping the order to get same effect and 'safer'.
+      return ByteBufferUtils.compareTo(left.getFamilyArray(), left.getFamilyOffset(), leftFamLen,
+        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) right).getFamilyPosition(), rightFamLen);
+    }
+    return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), leftFamLen,
+      right.getFamilyArray(), right.getFamilyOffset(), rightFamLen);
+  }
+
+  private final int compareQualifiers(Cell left, int leftQualLen, Cell right, int rightQualLen) {
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) left).getQualifierPosition(), leftQualLen,
+        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) right).getQualifierPosition(), rightQualLen);
+    }
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) left).getQualifierPosition(), leftQualLen,
+        right.getQualifierArray(), right.getQualifierOffset(), rightQualLen);
+    }
+    if (right instanceof ByteBufferExtendedCell) {
+      // Notice how we flip the order of the compare here. We used to negate the return value but
+      // see what FindBugs says
+      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
+      // It suggest flipping the order to get same effect and 'safer'.
+      return ByteBufferUtils.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
+        leftQualLen, ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) right).getQualifierPosition(), rightQualLen);
+    }
+    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(), leftQualLen,
+      right.getQualifierArray(), right.getQualifierOffset(), rightQualLen);
+  }
+
   /**
    * Compare the families of left and right cell
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
@@ -125,38 +441,174 @@ public class CellComparatorImpl implements CellComparator {
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
   }
 
+  static int compareQualifiers(KeyValue left, KeyValue right) {
+    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
+    // sharing gets us a few percent more throughput in compares. If changes here or there, make
+    // sure done in both places.
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // Compare families.
+    int leftFamilyOffset = left.getFamilyOffset(leftFamilyLengthPosition);
+    int rightFamilyOffset = right.getFamilyOffset(rightFamilyLengthPosition);
+
+    // Compare qualifiers
+    return Bytes.compareTo(left.getQualifierArray(), leftFamilyOffset + leftFamilyLength,
+      leftQualifierLength, right.getQualifierArray(), rightFamilyOffset + rightFamilyLength,
+      rightQualifierLength);
+  }
+
+  static int compareQualifiers(KeyValue left, ByteBufferKeyValue right) {
+    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
+    // sharing gets us a few percent more throughput in compares. If changes here or there, make
+    // sure done in both places.
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // Compare families.
+    int leftFamilyOffset = left.getFamilyOffset(leftFamilyLengthPosition);
+    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
+
+    // Compare qualifiers
+    return ByteBufferUtils.compareTo(left.getQualifierArray(),
+      leftFamilyOffset + leftFamilyLength, leftQualifierLength, right.getQualifierByteBuffer(),
+      rightFamilyPosition + rightFamilyLength, rightQualifierLength);
+  }
+
+  static int compareQualifiers(ByteBufferKeyValue left, KeyValue right) {
+    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
+    // sharing gets us a few percent more throughput in compares. If changes here or there, make
+    // sure done in both places.
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // Compare families.
+    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
+    int rightFamilyOffset = right.getFamilyOffset(rightFamilyLengthPosition);
+
+    // Compare qualifiers
+    return ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
+      leftFamilyPosition + leftFamilyLength, leftQualifierLength, right.getQualifierArray(),
+      rightFamilyOffset + rightFamilyLength, rightQualifierLength);
+  }
+
+  static int compareQualifiers(ByteBufferKeyValue left, ByteBufferKeyValue right) {
+    // NOTE: Same method is in CellComparatorImpl, also private, not shared, intentionally. Not
+    // sharing gets us a few percent more throughput in compares. If changes here or there, make
+    // sure done in both places.
+    // Compare Rows. Cache row length.
+    int leftRowLength = left.getRowLength();
+    int rightRowLength = right.getRowLength();
+
+    int leftFamilyLengthPosition = left.getFamilyLengthPosition(leftRowLength);
+    byte leftFamilyLength = left.getFamilyLength(leftFamilyLengthPosition);
+    int leftKeyLength = left.getKeyLength();
+    int leftQualifierLength =
+        left.getQualifierLength(leftKeyLength, leftRowLength, leftFamilyLength);
+
+    // No need of left row length below here.
+
+    int rightFamilyLengthPosition = right.getFamilyLengthPosition(rightRowLength);
+    byte rightFamilyLength = right.getFamilyLength(rightFamilyLengthPosition);
+    int rightKeyLength = right.getKeyLength();
+    int rightQualifierLength =
+        right.getQualifierLength(rightKeyLength, rightRowLength, rightFamilyLength);
+
+    // Compare families.
+    int leftFamilyPosition = left.getFamilyPosition(leftFamilyLengthPosition);
+    int rightFamilyPosition = right.getFamilyPosition(rightFamilyLengthPosition);
+
+    // Compare qualifiers
+    return ByteBufferUtils.compareTo(left.getQualifierByteBuffer(),
+      leftFamilyPosition + leftFamilyLength, leftQualifierLength, right.getQualifierByteBuffer(),
+      rightFamilyPosition + rightFamilyLength, rightQualifierLength);
+  }
+
   /**
    * Compare the qualifiers part of the left and right cells.
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   @Override
   public final int compareQualifiers(Cell left, Cell right) {
-    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
-      return ByteBufferUtils
-          .compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
-              ((ByteBufferExtendedCell) left).getQualifierPosition(),
-              left.getQualifierLength(), ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
-              ((ByteBufferExtendedCell) right).getQualifierPosition(),
-              right.getQualifierLength());
-    }
-    if (left instanceof ByteBufferExtendedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+    if ((left instanceof ByteBufferKeyValue) && (right instanceof ByteBufferKeyValue)) {
+      return compareQualifiers((ByteBufferKeyValue) left, (ByteBufferKeyValue) right);
+    } else if ((left instanceof KeyValue) && (right instanceof KeyValue)) {
+      return compareQualifiers((KeyValue) left, (KeyValue) right);
+    } else if ((left instanceof KeyValue) && (right instanceof ByteBufferKeyValue)) {
+      return compareQualifiers((KeyValue) left, (ByteBufferKeyValue) right);
+    } else if ((left instanceof ByteBufferKeyValue) && (right instanceof KeyValue)) {
+      return compareQualifiers((ByteBufferKeyValue) left, (KeyValue) right);
+    } else {
+      if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+        return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
+          ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
+      }
+      if (left instanceof ByteBufferExtendedCell) {
+        return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
           ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
           right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
-    }
-    if (right instanceof ByteBufferExtendedCell) {
-      // Notice how we flip the order of the compare here. We used to negate the return value but
-      // see what FindBugs says
-      // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
-      // It suggest flipping the order to get same effect and 'safer'.
-      return ByteBufferUtils.compareTo(left.getQualifierArray(),
-          left.getQualifierOffset(), left.getQualifierLength(),
-          ((ByteBufferExtendedCell)right).getQualifierByteBuffer(),
-          ((ByteBufferExtendedCell)right).getQualifierPosition(), right.getQualifierLength());
-    }
-    return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
+      }
+      if (right instanceof ByteBufferExtendedCell) {
+        // Notice how we flip the order of the compare here. We used to negate the return value but
+        // see what FindBugs says
+        // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
+        // It suggest flipping the order to get same effect and 'safer'.
+        return ByteBufferUtils.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
+          left.getQualifierLength(), ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
+      }
+      return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
         right.getQualifierLength());
+    }
+
   }
 
   /**
@@ -195,8 +647,8 @@ public class CellComparatorImpl implements CellComparator {
           ((ByteBufferExtendedCell)right).getRowByteBuffer(),
           ((ByteBufferExtendedCell)right).getRowPosition(), rightRowLength);
     }
-    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), leftRowLength,
+        right.getRowArray(), right.getRowOffset(), rightRowLength);
   }
 
   /**
@@ -249,10 +701,10 @@ public class CellComparatorImpl implements CellComparator {
     }
     if (lFamLength != rFamLength) {
       // comparing column family is enough.
-      return compareFamilies(left, right);
+      return compareFamilies(left, lFamLength, right, rFamLength);
     }
     // Compare cf:qualifier
-    int diff = compareColumns(left, right);
+    int diff = compareColumns(left, lFamLength, lQualLength, right, rFamLength, rQualLength);
     if (diff != 0) {
       return diff;
     }
@@ -282,7 +734,7 @@ public class CellComparatorImpl implements CellComparator {
 
   @Override
   public Comparator getSimpleComparator() {
-    return new BBKVComparator(this);
+    return this;
   }
 
   /**
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 1dbbe43..9674822 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
@@ -789,6 +789,11 @@ public final class CellUtil {
   public static boolean matchingFamily(final Cell left, final Cell right) {
     byte lfamlength = left.getFamilyLength();
     byte rfamlength = right.getFamilyLength();
+    return matchingFamily(left, lfamlength, right, rfamlength);
+  }
+
+  public static boolean matchingFamily(final Cell left, final byte lfamlength, final Cell right,
+      final byte rfamlength) {
     if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
         ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
@@ -835,6 +840,11 @@ public final class CellUtil {
   public static boolean matchingQualifier(final Cell left, final Cell right) {
     int lqlength = left.getQualifierLength();
     int rqlength = right.getQualifierLength();
+    return matchingQualifier(left, lqlength, right, rqlength);
+  }
+
+  private static boolean matchingQualifier(final Cell left, final int lqlength, final Cell right,
+      final int rqlength) {
     if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
         ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
@@ -920,6 +930,14 @@ public final class CellUtil {
     return matchingQualifier(left, right);
   }
 
+  private static boolean matchingColumn(final Cell left, final byte lFamLen, final int lQualLength,
+      final Cell right, final byte rFamLen, final int rQualLength) {
+    if (!matchingFamily(left, lFamLen, right, rFamLen)) {
+      return false;
+    }
+    return matchingQualifier(left, lQualLength, right, rQualLength);
+  }
+
   public static boolean matchingValue(final Cell left, final Cell right) {
     return PrivateCellUtil.matchingValue(left, right, left.getValueLength(),
       right.getValueLength());
@@ -1570,6 +1588,11 @@ 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);
+  }
+
+  public static boolean matchingRows(final Cell left, final short lrowlength, final Cell right,
+      final short rrowlength) {
     if (lrowlength != rrowlength) return false;
     if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
@@ -1598,16 +1621,29 @@ public final class CellUtil {
    * @return True if same row and column.
    */
   public static boolean matchingRowColumn(final Cell left, final Cell right) {
-    if ((left.getRowLength() + left.getFamilyLength()
-        + left.getQualifierLength()) != (right.getRowLength() + right.getFamilyLength()
-            + right.getQualifierLength())) {
+    short lrowlength = left.getRowLength();
+    short rrowlength = right.getRowLength();
+    // match length
+    if (lrowlength != rrowlength) {
+      return false;
+    }
+
+    byte lfamlength = left.getFamilyLength();
+    byte rfamlength = right.getFamilyLength();
+    if (lfamlength != rfamlength) {
       return false;
     }
 
-    if (!matchingRows(left, right)) {
+    int lqlength = left.getQualifierLength();
+    int rqlength = right.getQualifierLength();
+    if (lqlength != rqlength) {
+      return false;
+    }
+
+    if (!matchingRows(left, lrowlength, right, rrowlength)) {
       return false;
     }
-    return matchingColumn(left, right);
+    return matchingColumn(left, lfamlength, lqlength, right, rfamlength, rqlength);
   }
 
   public static boolean matchingRowColumnBytes(final Cell left, final Cell right) {
@@ -1617,9 +1653,9 @@ public final class CellUtil {
     int rfamlength = right.getFamilyLength();
     int lqlength = left.getQualifierLength();
     int rqlength = right.getQualifierLength();
+
     // match length
-    if ((lrowlength + lfamlength + lqlength) !=
-        (rrowlength + rfamlength + rqlength)) {
+    if ((lrowlength != rrowlength) || (lfamlength != rfamlength) || (lqlength != rqlength)) {
       return false;
     }
 
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index b7ddbab..4df904d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -1351,14 +1352,14 @@ public class KeyValue implements ExtendedCell, Cloneable {
    */
   @Override
   public int getFamilyOffset() {
-    return getFamilyOffset(getRowLength());
+    return getFamilyOffset(getFamilyLengthPosition(getRowLength()));
   }
 
   /**
    * @return Family offset
    */
-  private int getFamilyOffset(int rlength) {
-    return this.offset + ROW_KEY_OFFSET + rlength + Bytes.SIZEOF_BYTE;
+  int getFamilyOffset(int familyLenPosition) {
+    return familyLenPosition + Bytes.SIZEOF_BYTE;
   }
 
   /**
@@ -1366,14 +1367,18 @@ public class KeyValue implements ExtendedCell, Cloneable {
    */
   @Override
   public byte getFamilyLength() {
-    return getFamilyLength(getFamilyOffset());
+    return getFamilyLength(getFamilyLengthPosition(getRowLength()));
   }
 
   /**
    * @return Family length
    */
-  public byte getFamilyLength(int foffset) {
-    return this.bytes[foffset-1];
+  public byte getFamilyLength(int famLenPos) {
+    return this.bytes[famLenPos];
+  }
+
+  int getFamilyLengthPosition(int rowLength) {
+    return this.offset + KeyValue.ROW_KEY_OFFSET + rowLength;
   }
 
   /**
@@ -1396,7 +1401,14 @@ public class KeyValue implements ExtendedCell, Cloneable {
    * @return Qualifier offset
    */
   private int getQualifierOffset(int foffset) {
-    return foffset + getFamilyLength(foffset);
+    return getQualifierOffset(foffset, getFamilyLength());
+  }
+
+  /**
+   * @return Qualifier offset
+   */
+  int getQualifierOffset(int foffset, int flength) {
+    return foffset + flength;
   }
 
   /**
@@ -1411,7 +1423,14 @@ public class KeyValue implements ExtendedCell, Cloneable {
    * @return Qualifier length
    */
   private int getQualifierLength(int rlength, int flength) {
-    return getKeyLength() - (int) getKeyDataStructureSize(rlength, flength, 0);
+    return getQualifierLength(getKeyLength(), rlength, flength);
+  }
+
+  /**
+   * @return Qualifier length
+   */
+  int getQualifierLength(int keyLength, int rlength, int flength) {
+    return keyLength - (int) getKeyDataStructureSize(rlength, flength, 0);
   }
 
   /**
@@ -1504,7 +1523,11 @@ public class KeyValue implements ExtendedCell, Cloneable {
    */
   @Override
   public byte getTypeByte() {
-    return this.bytes[this.offset + getKeyLength() - 1 + ROW_OFFSET];
+    return getTypeByte(getKeyLength());
+  }
+
+  byte getTypeByte(int keyLength) {
+    return this.bytes[this.offset + keyLength - 1 + ROW_OFFSET];
   }
 
   /**
@@ -1878,8 +1901,8 @@ public class KeyValue implements ExtendedCell, Cloneable {
      * @param rlength
      * @return 0 if equal, &lt;0 if left smaller, &gt;0 if right smaller
      */
-    public int compareRows(byte [] left, int loffset, int llength,
-        byte [] right, int roffset, int rlength) {
+    public int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
+        int rlength) {
       return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
     }
 
@@ -2452,6 +2475,10 @@ public class KeyValue implements ExtendedCell, Cloneable {
       return this.bytes[getFamilyOffset() - 1];
     }
 
+    int getFamilyLengthPosition(int rowLength) {
+      return this.offset + Bytes.SIZEOF_SHORT + rowLength;
+    }
+
     @Override
     public int getFamilyOffset() {
       return this.offset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE;
@@ -2484,9 +2511,14 @@ public class KeyValue implements ExtendedCell, Cloneable {
 
     @Override
     public byte getTypeByte() {
-      return this.bytes[this.offset + getKeyLength() - 1];
+      return getTypeByte(getKeyLength());
     }
 
+    byte getTypeByte(int keyLength) {
+      return this.bytes[this.offset + keyLength - 1];
+    }
+
+
     private int getQualifierLength(int rlength, int flength) {
       return getKeyLength() - (int) getKeyDataStructureSize(rlength, flength, 0);
     }
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
index 6443d84..d6c8a75 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
@@ -69,7 +69,7 @@ public class TestByteBufferKeyValue {
     assertTrue(CellComparatorImpl.COMPARATOR.compare(cell1, cell3) < 0);
     Cell cell4 = getOffheapCell(row1, Bytes.toBytes("f"), qual2);
     assertTrue(CellComparatorImpl.COMPARATOR.compare(cell1, cell4) > 0);
-    BBKVComparator comparator = new BBKVComparator(null);
+    CellComparator comparator = CellComparator.getInstance();
     assertTrue(comparator.compare(cell1, cell2) < 0);
     assertTrue(comparator.compare(cell1, cell3) < 0);
     assertTrue(comparator.compare(cell1, cell4) > 0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
index 0fb183e..a8efa16 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
@@ -221,7 +221,8 @@ public class DataBlockEncodingTool {
 
       int kLen = currentKV.getKeyLength();
       int vLen = currentKV.getValueLength();
-      int cfLen = currentKV.getFamilyLength(currentKV.getFamilyOffset());
+      int cfOffset = currentKV.getFamilyOffset();
+      int cfLen = currentKV.getFamilyLength();
       int restLen = currentKV.getLength() - kLen - vLen;
 
       totalKeyLength += kLen;