You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2017/12/27 15:04:58 UTC

[3/3] hbase git commit: HBASE-19628 ByteBufferCell should extend ExtendedCell

HBASE-19628 ByteBufferCell should extend ExtendedCell


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

Branch: refs/heads/branch-2
Commit: 2468f300299fc37553cfb292d780aa9b255343e7
Parents: 0454a38
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Wed Dec 27 22:45:46 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Dec 27 23:02:18 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |   8 +-
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |  57 ++-
 .../hadoop/hbase/filter/PrefixFilter.java       |   8 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  20 +-
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  |   6 +-
 .../org/apache/hadoop/hbase/ByteBufferCell.java | 120 -----
 .../hadoop/hbase/ByteBufferExtendedCell.java    | 121 +++++
 .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java |  44 +-
 .../apache/hadoop/hbase/ByteBufferKeyValue.java |   6 +-
 .../apache/hadoop/hbase/CellComparatorImpl.java |  72 +--
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 298 +++++------
 .../hadoop/hbase/ExtendedCellBuilder.java       |   2 +-
 .../apache/hadoop/hbase/PrivateCellUtil.java    | 506 ++++++++++---------
 .../java/org/apache/hadoop/hbase/TagUtil.java   |   6 +-
 .../io/encoding/BufferedDataBlockEncoder.java   |  14 +-
 .../hbase/io/encoding/RowIndexSeekerV1.java     |   8 +-
 .../hadoop/hbase/TestByteBufferKeyValue.java    |   4 +-
 .../hbase/io/TestTagCompressionContext.java     |  10 +-
 .../hadoop/hbase/mapreduce/CellSortReducer.java |   4 +-
 .../hbase/mapreduce/HFileOutputFormat2.java     |   8 +-
 .../apache/hadoop/hbase/mapreduce/Import.java   |  10 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   6 +-
 .../apache/hadoop/hbase/util/MapReduceCell.java | 271 ----------
 .../hbase/util/MapReduceExtendedCell.java       | 270 ++++++++++
 .../mapreduce/TestCellBasedImportExport2.java   |   6 +-
 .../mapreduce/TestCellBasedWALPlayer2.java      |   6 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |  29 +-
 .../hbase/regionserver/ByteBufferChunkCell.java |  48 --
 .../regionserver/ByteBufferChunkKeyValue.java   |  48 ++
 .../hadoop/hbase/regionserver/CellChunkMap.java |   2 +-
 .../hbase/regionserver/MemStoreLABImpl.java     |   4 +-
 .../regionserver/NoTagByteBufferChunkCell.java  |  48 --
 .../NoTagByteBufferChunkKeyValue.java           |  48 ++
 .../hbase/regionserver/RSRpcServices.java       |   6 +-
 .../encoding/TestBufferedDataBlockEncoder.java  |   4 +-
 35 files changed, 1123 insertions(+), 1005 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
index 4811691..b4e7a0f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -95,9 +95,9 @@ public class ColumnPrefixFilter extends FilterBase {
   }
 
   private static int compareQualifierPart(Cell cell, int length, byte[] prefix) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), length, prefix, 0, length);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), length, prefix, 0, length);
     }
     return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0,
         length);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index 32286b6..4487292 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -18,21 +18,24 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-
-import org.apache.hadoop.hbase.ByteBufferCell;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Optional;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 /**
  * A filter that will only return the key component of each KV (the value will
@@ -60,8 +63,8 @@ public class KeyOnlyFilter extends FilterBase {
   }
 
   private Cell createKeyOnlyCell(Cell c) {
-    if (c instanceof ByteBufferCell) {
-      return new KeyOnlyByteBufferCell((ByteBufferCell) c, lenAsVal);
+    if (c instanceof ByteBufferExtendedCell) {
+      return new KeyOnlyByteBufferExtendedCell((ByteBufferExtendedCell) c, lenAsVal);
     } else {
       return new KeyOnlyCell(c, lenAsVal);
     }
@@ -77,7 +80,7 @@ public class KeyOnlyFilter extends FilterBase {
   public ReturnCode filterCell(final Cell ignored) throws IOException {
     return ReturnCode.INCLUDE;
   }
-  
+
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
     Preconditions.checkArgument((filterArguments.isEmpty() || filterArguments.size() == 1),
                                 "Expected: 0 or 1 but got: %s", filterArguments.size());
@@ -244,11 +247,13 @@ public class KeyOnlyFilter extends FilterBase {
     }
   }
 
-  static class KeyOnlyByteBufferCell extends ByteBufferCell {
-    private ByteBufferCell cell;
+  static class KeyOnlyByteBufferExtendedCell extends ByteBufferExtendedCell {
+    public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
+        + Bytes.SIZEOF_BOOLEAN;
+    private ByteBufferExtendedCell cell;
     private boolean lenAsVal;
 
-    public KeyOnlyByteBufferCell(ByteBufferCell c, boolean lenAsVal) {
+    public KeyOnlyByteBufferExtendedCell(ByteBufferExtendedCell c, boolean lenAsVal) {
       this.cell = c;
       this.lenAsVal = lenAsVal;
     }
@@ -309,6 +314,21 @@ public class KeyOnlyFilter extends FilterBase {
     }
 
     @Override
+    public void setSequenceId(long seqId) throws IOException {
+      cell.setSequenceId(seqId);
+    }
+
+    @Override
+    public void setTimestamp(long ts) throws IOException {
+      cell.setTimestamp(ts);
+    }
+
+    @Override
+    public void setTimestamp(byte[] ts) throws IOException {
+      cell.setTimestamp(ts);
+    }
+
+    @Override
     public long getSequenceId() {
       return 0;
     }
@@ -409,6 +429,21 @@ public class KeyOnlyFilter extends FilterBase {
     public int getTagsPosition() {
       return 0;
     }
+
+    @Override
+    public Iterator<Tag> getTags() {
+      return Collections.emptyIterator();
+    }
+
+    @Override
+    public Optional<Tag> getTag(byte type) {
+      return Optional.empty();
+    }
+
+    @Override
+    public long heapSize() {
+      return ClassSize.align(FIXED_OVERHEAD + cell.heapSize());
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
index 161c1a5..09a3304 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter;
 
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -61,9 +61,9 @@ public class PrefixFilter extends FilterBase {
     // else return true, filter row
     // if we are passed the prefix, set flag
     int cmp;
-    if (firstRowCell instanceof ByteBufferCell) {
-      cmp = ByteBufferUtils.compareTo(((ByteBufferCell) firstRowCell).getRowByteBuffer(),
-          ((ByteBufferCell) firstRowCell).getRowPosition(), this.prefix.length,
+    if (firstRowCell instanceof ByteBufferExtendedCell) {
+      cmp = ByteBufferUtils.compareTo(((ByteBufferExtendedCell) firstRowCell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) firstRowCell).getRowPosition(), this.prefix.length,
           this.prefix, 0, this.prefix.length);
     } else {
       cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 17b1141..8f44135 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -40,7 +40,7 @@ import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
 import org.apache.hadoop.hbase.Cell;
@@ -2039,17 +2039,17 @@ public final class ProtobufUtil {
     // Doing this is going to kill us if we do it for all data passed.
     // St.Ack 20121205
     CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder();
-    if (kv instanceof ByteBufferCell) {
-      kvbuilder.setRow(wrap(((ByteBufferCell) kv).getRowByteBuffer(),
-        ((ByteBufferCell) kv).getRowPosition(), kv.getRowLength()));
-      kvbuilder.setFamily(wrap(((ByteBufferCell) kv).getFamilyByteBuffer(),
-        ((ByteBufferCell) kv).getFamilyPosition(), kv.getFamilyLength()));
-      kvbuilder.setQualifier(wrap(((ByteBufferCell) kv).getQualifierByteBuffer(),
-        ((ByteBufferCell) kv).getQualifierPosition(), kv.getQualifierLength()));
+    if (kv instanceof ByteBufferExtendedCell) {
+      kvbuilder.setRow(wrap(((ByteBufferExtendedCell) kv).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getRowPosition(), kv.getRowLength()));
+      kvbuilder.setFamily(wrap(((ByteBufferExtendedCell) kv).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getFamilyPosition(), kv.getFamilyLength()));
+      kvbuilder.setQualifier(wrap(((ByteBufferExtendedCell) kv).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getQualifierPosition(), kv.getQualifierLength()));
       kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
       kvbuilder.setTimestamp(kv.getTimestamp());
-      kvbuilder.setValue(wrap(((ByteBufferCell) kv).getValueByteBuffer(),
-        ((ByteBufferCell) kv).getValuePosition(), kv.getValueLength()));
+      kvbuilder.setValue(wrap(((ByteBufferExtendedCell) kv).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getValuePosition(), kv.getValueLength()));
       // TODO : Once tags become first class then we may have to set tags to kvbuilder.
     } else {
       kvbuilder.setRow(

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
index f957b59..28fe2bf 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
@@ -20,9 +20,7 @@ package org.apache.hadoop.hbase.filter;
 import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.CellUtil;
@@ -30,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferExtendedCell;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -82,7 +80,7 @@ public class TestKeyOnlyFilter {
     KeyValue KeyOnlyKeyValue = new KeyValue(newBuffer);
 
     KeyOnlyCell keyOnlyCell = new KeyOnlyCell(kv, lenAsVal);
-    KeyOnlyByteBufferCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferCell(
+    KeyOnlyByteBufferExtendedCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferExtendedCell(
         bbCell, lenAsVal);
 
     assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyCell));

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
deleted file mode 100644
index b3be0ac..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
+++ /dev/null
@@ -1,120 +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.nio.ByteBuffer;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * This class is a server side extension to the {@link Cell} interface. It is used when the
- * Cell is backed by a {@link ByteBuffer}: i.e. <code>cell instanceof ByteBufferedCell</code>.
- *
- * <p>This class has getters for the row, column family, column qualifier, value and tags hosting
- * ByteBuffers. It also has getters of the *position* within a ByteBuffer where these
- * field bytes begin. These are needed because a single ByteBuffer may back one or many Cell
- * instances -- it depends on the implementation -- so the ByteBuffer position as returned by
- * {@link ByteBuffer#arrayOffset()} cannot be relied upon. Also, do not confuse these position
- * methods with the getXXXOffset methods from the super Interface, {@link Cell}; dependent up on
- * implementation, the Cell getXXXOffset methods can return the same value as a call to its
- * equivalent position method from below BUT they can also stray; if a ByteBufferedCell, use the
- * below position methods to find where a field begins.
- *
- * <p>Use the getXXXLength methods from Cell to find a fields length.
- *
- * <p>A Cell object can be of this type only on the server side.
- *
- * <p>WARNING: If a Cell is backed by an offheap ByteBuffer, any call to getXXXArray() will result
- * in a temporary byte array creation and a bytes copy. Avoid these allocations by using the
- * appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell
- * when it is not.
- */
-/*
- * Even though all the methods are abstract, ByteBufferCell is not made to be an interface with
- * intent. In CellComparator compare method, we have instance of check to decide whether to use
- * getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
- * if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      ....
-    }
-    if (left instanceof ByteBufferCell) {
-      ....
-    }
-    if (right instanceof ByteBufferCell) {
-      ....
-    }
-    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
- * We did JMH micro benchmark tests with both left and right cells as ByteBufferCell, one only
- * ByteBufferCell and both as Cells. This is compared against JMH results on compare logic with out
- * any instance of checks. We noticed that if ByteBufferCell is an interface, the benchmark result
- * seems to be very bad for case of both right and left are Cell only (Not ByteBufferCell). When
- * ByteBufferCell is an abstract class all 4 possible cases giving almost similar performance number
- * compared with compare logic with no instance of checks.
- */
-@InterfaceAudience.Private
-public abstract class ByteBufferCell implements Cell {
-  /**
-   * @return The {@link ByteBuffer} containing the row bytes.
-   */
-  public abstract ByteBuffer getRowByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where row bytes start
-   */
-  public abstract int getRowPosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the column family bytes.
-   */
-  public abstract ByteBuffer getFamilyByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where column family bytes start
-   */
-  public abstract int getFamilyPosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the column qualifier bytes.
-   */
-  public abstract ByteBuffer getQualifierByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where column qualifier bytes start
-   */
-  public abstract int getQualifierPosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the value bytes.
-   */
-  public abstract ByteBuffer getValueByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where value bytes start
-   */
-  public abstract int getValuePosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the tag bytes.
-   */
-  public abstract ByteBuffer getTagsByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where tag bytes start
-   */
-  public abstract int getTagsPosition();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java
new file mode 100644
index 0000000..3e4cc95
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java
@@ -0,0 +1,121 @@
+/**
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * This class is a server side extension to the {@link Cell} interface. It is used when the
+ * Cell is backed by a {@link ByteBuffer}: i.e. <code>cell instanceof ByteBufferedCell</code>.
+ *
+ * <p>This class has getters for the row, column family, column qualifier, value and tags hosting
+ * ByteBuffers. It also has getters of the *position* within a ByteBuffer where these
+ * field bytes begin. These are needed because a single ByteBuffer may back one or many Cell
+ * instances -- it depends on the implementation -- so the ByteBuffer position as returned by
+ * {@link ByteBuffer#arrayOffset()} cannot be relied upon. Also, do not confuse these position
+ * methods with the getXXXOffset methods from the super Interface, {@link Cell}; dependent up on
+ * implementation, the Cell getXXXOffset methods can return the same value as a call to its
+ * equivalent position method from below BUT they can also stray; if a ByteBufferedCell, use the
+ * below position methods to find where a field begins.
+ *
+ * <p>Use the getXXXLength methods from Cell to find a fields length.
+ *
+ * <p>A Cell object can be of this type only on the server side.
+ *
+ * <p>WARNING: If a Cell is backed by an offheap ByteBuffer, any call to getXXXArray() will result
+ * in a temporary byte array creation and a bytes copy. Avoid these allocations by using the
+ * appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell
+ * when it is not.
+ */
+/*
+ * Even though all the methods are abstract, ByteBufferExtendedCell is not made to be an interface
+ * with intent. In CellComparator compare method, we have instance of check to decide whether to
+ * use getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
+ * if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      ....
+    }
+    if (left instanceof ByteBufferExtendedCell) {
+      ....
+    }
+    if (right instanceof ByteBufferExtendedCell) {
+      ....
+    }
+    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
+        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+ * We did JMH micro benchmark tests with both left and right cells as ByteBufferExtendedCell, one
+ * only ByteBufferExtendedCell and both as Cells. This is compared against JMH results on compare
+ * logic with out any instance of checks. We noticed that if ByteBufferExtendedCell is an
+ * interface, the benchmark result seems to be very bad for case of both right and left are Cell
+ * only (Not ByteBufferExtendedCell). When ByteBufferExtendedCell is an abstract class all 4
+ * possible cases giving almost similar performance number compared with compare logic with no
+ * instance of checks.
+ */
+@InterfaceAudience.Private
+public abstract class ByteBufferExtendedCell implements ExtendedCell {
+  /**
+   * @return The {@link ByteBuffer} containing the row bytes.
+   */
+  public abstract ByteBuffer getRowByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where row bytes start
+   */
+  public abstract int getRowPosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the column family bytes.
+   */
+  public abstract ByteBuffer getFamilyByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where column family bytes start
+   */
+  public abstract int getFamilyPosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the column qualifier bytes.
+   */
+  public abstract ByteBuffer getQualifierByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where column qualifier bytes start
+   */
+  public abstract int getQualifierPosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the value bytes.
+   */
+  public abstract ByteBuffer getValueByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where value bytes start
+   */
+  public abstract int getValuePosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the tag bytes.
+   */
+  public abstract ByteBuffer getTagsByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where tag bytes start
+   */
+  public abstract int getTagsPosition();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
----------------------------------------------------------------------
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 713314e..7ecb946 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
@@ -18,10 +18,14 @@
  */
 package org.apache.hadoop.hbase;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
-
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Optional;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -30,8 +34,9 @@ import org.apache.yetus.audience.InterfaceAudience;
  * (onheap and offheap).
  */
 @InterfaceAudience.Private
-public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
-
+public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
+  public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
+      + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT;
   private ByteBuffer buf;
   private int offset = 0; // offset into buffer where key starts at
   private int length = 0; // length of this.
@@ -152,6 +157,21 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
   }
 
   @Override
+  public void setSequenceId(long seqId) throws IOException {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public void setTimestamp(long ts) throws IOException {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public void setTimestamp(byte[] ts) throws IOException {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
   public long getSequenceId() {
     return 0;
   }
@@ -246,4 +266,22 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
   public String toString() {
     return CellUtil.toString(this, false);
   }
+
+  @Override
+  public Iterator<Tag> getTags() {
+    return Collections.emptyIterator();
+  }
+
+  @Override
+  public Optional<Tag> getTag(byte type) {
+    return Optional.empty();
+  }
+
+  @Override
+  public long heapSize() {
+    if (this.buf.hasArray()) {
+      return ClassSize.align(FIXED_OVERHEAD + length);
+    }
+    return ClassSize.align(FIXED_OVERHEAD);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
index 798180a..54af38f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
@@ -28,11 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
- * This Cell is an implementation of {@link ByteBufferCell} where the data resides in
+ * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
  * off heap/ on heap ByteBuffer
  */
 @InterfaceAudience.Private
-public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
+public class ByteBufferKeyValue extends ByteBufferExtendedCell {
 
   protected final ByteBuffer buf;
   protected final int offset;
@@ -329,7 +329,7 @@ public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
     return calculateHashForKey(this);
   }
 
-  private int calculateHashForKey(ByteBufferCell cell) {
+  private int calculateHashForKey(ByteBufferExtendedCell cell) {
     int rowHash = ByteBufferUtils.hashCode(cell.getRowByteBuffer(), cell.getRowPosition(),
       cell.getRowLength());
     int familyHash = ByteBufferUtils.hashCode(cell.getFamilyByteBuffer(), cell.getFamilyPosition(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
----------------------------------------------------------------------
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 771fdaa..f1abbef 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
@@ -108,26 +108,26 @@ public class CellComparatorImpl implements CellComparator {
    */
   @Override
   public final int compareFamilies(Cell left, Cell right) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
-          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
-          ((ByteBufferCell) right).getFamilyByteBuffer(),
-          ((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
+          ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
-          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
           right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
     }
-    if (right instanceof ByteBufferCell) {
+    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(), left.getFamilyLength(),
-          ((ByteBufferCell)right).getFamilyByteBuffer(),
-          ((ByteBufferCell)right).getFamilyPosition(), right.getFamilyLength());
+          ((ByteBufferExtendedCell)right).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell)right).getFamilyPosition(), right.getFamilyLength());
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
@@ -141,28 +141,28 @@ public class CellComparatorImpl implements CellComparator {
    */
   @Override
   public final int compareQualifiers(Cell left, Cell right) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils
-          .compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
-              ((ByteBufferCell) left).getQualifierPosition(),
-              left.getQualifierLength(), ((ByteBufferCell) right).getQualifierByteBuffer(),
-              ((ByteBufferCell) right).getQualifierPosition(),
+          .compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+              ((ByteBufferExtendedCell) left).getQualifierPosition(),
+              left.getQualifierLength(), ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+              ((ByteBufferExtendedCell) right).getQualifierPosition(),
               right.getQualifierLength());
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
-          ((ByteBufferCell) left).getQualifierPosition(), left.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 ByteBufferCell) {
+    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(),
-          ((ByteBufferCell)right).getQualifierByteBuffer(),
-          ((ByteBufferCell)right).getQualifierPosition(), right.getQualifierLength());
+          ((ByteBufferExtendedCell)right).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell)right).getQualifierPosition(), right.getQualifierLength());
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
@@ -183,25 +183,25 @@ public class CellComparatorImpl implements CellComparator {
     if (left == right) {
       return 0;
     }
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
-          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
-          ((ByteBufferCell) right).getRowByteBuffer(),
-          ((ByteBufferCell) right).getRowPosition(), right.getRowLength());
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
+          ((ByteBufferExtendedCell) right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength());
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
-          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
           right.getRowArray(), right.getRowOffset(), right.getRowLength());
     }
-    if (right instanceof ByteBufferCell) {
+    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.getRowArray(), left.getRowOffset(), left.getRowLength(),
-          ((ByteBufferCell)right).getRowByteBuffer(),
-          ((ByteBufferCell)right).getRowPosition(), right.getRowLength());
+          ((ByteBufferExtendedCell)right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell)right).getRowPosition(), right.getRowLength());
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
         right.getRowArray(), right.getRowOffset(), right.getRowLength());
@@ -225,9 +225,9 @@ public class CellComparatorImpl implements CellComparator {
    */
   @Override
   public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
-          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(), right,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), right,
           roffset, rlength);
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/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 c116f31..d67bfa5 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
@@ -211,9 +211,10 @@ public final class CellUtil {
    */
   public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
     short rowLen = cell.getRowLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(destination, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
     } else {
       System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
         rowLen);
@@ -230,9 +231,9 @@ public final class CellUtil {
    */
   public static int copyRowTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     short rowLen = cell.getRowLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getRowByteBuffer(),
-        destination, ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
         cell.getRowOffset(), rowLen);
@@ -246,10 +247,10 @@ public final class CellUtil {
    * @return the byte[] containing the row
    */
   public static byte[] copyRow(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.copyOfRange(((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(),
-        ((ByteBufferCell) cell).getRowPosition() + cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.copyOfRange(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(),
+        ((ByteBufferExtendedCell) cell).getRowPosition() + cell.getRowLength());
     } else {
       return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
         cell.getRowOffset() + cell.getRowLength());
@@ -265,10 +266,10 @@ public final class CellUtil {
    */
   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
     byte fLen = cell.getFamilyLength();
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getFamilyByteBuffer(), ((ByteBufferCell) cell).getFamilyPosition(),
-        destinationOffset, fLen);
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
     } else {
       System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
         destinationOffset, fLen);
@@ -285,9 +286,9 @@ public final class CellUtil {
    */
   public static int copyFamilyTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     byte fLen = cell.getFamilyLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getFamilyByteBuffer(),
-        destination, ((ByteBufferCell) cell).getFamilyPosition(), destinationOffset, fLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
         cell.getFamilyOffset(), fLen);
@@ -304,10 +305,10 @@ public final class CellUtil {
    */
   public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
     int qlen = cell.getQualifierLength();
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
+        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), destinationOffset, qlen);
     } else {
       System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
         destinationOffset, qlen);
@@ -324,9 +325,11 @@ public final class CellUtil {
    */
   public static int copyQualifierTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     int qlen = cell.getQualifierLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getQualifierByteBuffer(),
-        destination, ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          destination, ((ByteBufferExtendedCell) cell).getQualifierPosition(),
+          destinationOffset, qlen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
         cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
@@ -343,10 +346,10 @@ public final class CellUtil {
    */
   public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
     int vlen = cell.getValueLength();
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getValueByteBuffer(), ((ByteBufferCell) cell).getValuePosition(),
-        destinationOffset, vlen);
+          ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
     } else {
       System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
         vlen);
@@ -363,9 +366,9 @@ public final class CellUtil {
    */
   public static int copyValueTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     int vlen = cell.getValueLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getValueByteBuffer(),
-        destination, ((ByteBufferCell) cell).getValuePosition(), destinationOffset, vlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
         cell.getValueOffset(), vlen);
@@ -384,13 +387,13 @@ public final class CellUtil {
   @Deprecated
   public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
     int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getTagsByteBuffer(), ((ByteBufferCell) cell).getTagsPosition(),
-        destinationOffset, tlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils
+        .copyFromBufferToArray(destination, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
     } else {
-      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
-        tlen);
+      System
+        .arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset, tlen);
     }
     return destinationOffset + tlen;
   }
@@ -406,9 +409,9 @@ public final class CellUtil {
   @Deprecated
   public static int copyTagTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getTagsByteBuffer(),
-        destination, ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
         cell.getTagsOffset(), tlen);
@@ -424,9 +427,9 @@ public final class CellUtil {
    */
   @Deprecated
   public static byte getRowByte(Cell cell, int index) {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) cell).getRowByteBuffer()
-          .get(((ByteBufferCell) cell).getRowPosition() + index);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) cell).getRowByteBuffer()
+          .get(((ByteBufferExtendedCell) cell).getRowPosition() + index);
     }
     return cell.getRowArray()[cell.getRowOffset() + index];
   }
@@ -775,9 +778,9 @@ public final class CellUtil {
   @Deprecated
   public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
       final int length) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
-        ((ByteBufferCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
     }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
       length);
@@ -786,20 +789,20 @@ public final class CellUtil {
   public static boolean matchingFamily(final Cell left, final Cell right) {
     byte lfamlength = left.getFamilyLength();
     byte rfamlength = right.getFamilyLength();
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
-        ((ByteBufferCell) left).getFamilyPosition(), lfamlength,
-        ((ByteBufferCell) right).getFamilyByteBuffer(),
-        ((ByteBufferCell) right).getFamilyPosition(), rfamlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
+        ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
-        ((ByteBufferCell) left).getFamilyPosition(), lfamlength, right.getFamilyArray(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength, right.getFamilyArray(),
         right.getFamilyOffset(), rfamlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getFamilyByteBuffer(),
-        ((ByteBufferCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(),
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength, left.getFamilyArray(),
         left.getFamilyOffset(), lfamlength);
     }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
@@ -818,32 +821,34 @@ public final class CellUtil {
    */
   @Deprecated
   public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
-      final int length) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
-        ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset, length);
+    final int length) {
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset,
+        length);
     }
-    return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
-      offset, length);
+    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 (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
-        ((ByteBufferCell) left).getQualifierPosition(), lqlength,
-        ((ByteBufferCell) right).getQualifierByteBuffer(),
-        ((ByteBufferCell) right).getQualifierPosition(), rqlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
+        ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
-        ((ByteBufferCell) left).getQualifierPosition(), lqlength, right.getQualifierArray(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength, right.getQualifierArray(),
         right.getQualifierOffset(), rqlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getQualifierByteBuffer(),
-        ((ByteBufferCell) right).getQualifierPosition(), rqlength, left.getQualifierArray(),
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength, left.getQualifierArray(),
         left.getQualifierOffset(), lqlength);
     }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), lqlength,
@@ -874,17 +879,18 @@ public final class CellUtil {
    */
   @Deprecated
   public static boolean matchingQualifier(final Cell left, final byte[] buf, final int offset,
-      final int length) {
+    final int length) {
     if (buf == null) {
       return left.getQualifierLength() == 0;
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
-        ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(), buf, offset,
-        length);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(), buf,
+        offset, length);
     }
-    return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
-      left.getQualifierLength(), buf, offset, length);
+    return Bytes
+      .equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(), buf,
+        offset, length);
   }
 
   public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
@@ -913,9 +919,10 @@ public final class CellUtil {
   }
 
   public static boolean matchingValue(final Cell left, final byte[] buf) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), left.getValueLength(), buf, 0, buf.length) == 0;
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
+        buf.length) == 0;
     }
     return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
       buf.length);
@@ -1239,18 +1246,18 @@ public final class CellUtil {
     int qLen = cell.getQualifierLength();
     // Using just one if/else loop instead of every time checking before writing every
     // component of cell
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       out.writeShort(rowLen);
-      ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-        rowLen);
+      ByteBufferUtils
+        .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
       out.writeByte(fLen);
+      ByteBufferUtils
+        .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
       ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getFamilyByteBuffer(), ((ByteBufferCell) cell).getFamilyPosition(),
-        fLen);
-      ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qLen);
+        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
     } else {
       out.writeShort(rowLen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
@@ -1272,11 +1279,11 @@ public final class CellUtil {
    */
   @Deprecated
   public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
-      int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
+    int commonPrefix) throws IOException {
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils
+        .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
     } else {
       out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
     }
@@ -1367,30 +1374,34 @@ public final class CellUtil {
    */
   @Deprecated
   public static int findCommonPrefixInFlatKey(Cell c1, Cell c2, boolean bypassFamilyCheck,
-      boolean withTsType) {
+    boolean withTsType) {
     // Compare the 2 bytes in RK length part
     short rLen1 = c1.getRowLength();
     short rLen2 = c2.getRowLength();
     int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
     if (rLen1 != rLen2) {
       // early out when the RK length itself is not matching
-      return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
-        Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
+      return ByteBufferUtils
+        .findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE, Bytes.toBytes(rLen2),
+          0, KeyValue.ROW_LENGTH_SIZE);
     }
     // Compare the RKs
     int rkCommonPrefix = 0;
-    if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
-      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getRowByteBuffer(),
-        ((ByteBufferCell) c1).getRowPosition(), rLen1, ((ByteBufferCell) c2).getRowByteBuffer(),
-        ((ByteBufferCell) c2).getRowPosition(), rLen2);
+    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
+      rkCommonPrefix = ByteBufferUtils
+        .findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
+          ((ByteBufferExtendedCell) c2).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
     } else {
       // There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
       // either
       // in flush or compactions. In flushes both cells are KV and in case of compaction it will be
       // either
       // KV or BBCell
-      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1,
-        c2.getRowArray(), c2.getRowOffset(), rLen2);
+      rkCommonPrefix = ByteBufferUtils
+        .findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1, c2.getRowArray(),
+          c2.getRowOffset(), rLen2);
     }
     commonPrefix += rkCommonPrefix;
     if (rkCommonPrefix != rLen1) {
@@ -1413,14 +1424,16 @@ public final class CellUtil {
       commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
       // Compare the CF names
       int fCommonPrefix;
-      if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
-        fCommonPrefix = ByteBufferUtils.findCommonPrefix(
-          ((ByteBufferCell) c1).getFamilyByteBuffer(), ((ByteBufferCell) c1).getFamilyPosition(),
-          fLen1, ((ByteBufferCell) c2).getFamilyByteBuffer(),
-          ((ByteBufferCell) c2).getFamilyPosition(), fLen2);
+      if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
+        fCommonPrefix = ByteBufferUtils
+          .findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
+            ((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
+            ((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
+            ((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
       } else {
-        fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
-          fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
+        fCommonPrefix = ByteBufferUtils
+          .findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(), fLen1, c2.getFamilyArray(),
+            c2.getFamilyOffset(), fLen2);
       }
       commonPrefix += fCommonPrefix;
       if (fCommonPrefix != fLen1) {
@@ -1431,22 +1444,25 @@ public final class CellUtil {
     int qLen1 = c1.getQualifierLength();
     int qLen2 = c2.getQualifierLength();
     int qCommon;
-    if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
-      qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getQualifierByteBuffer(),
-        ((ByteBufferCell) c1).getQualifierPosition(), qLen1,
-        ((ByteBufferCell) c2).getQualifierByteBuffer(),
-        ((ByteBufferCell) c2).getQualifierPosition(), qLen2);
+    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
+      qCommon = ByteBufferUtils
+        .findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
+          ((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
     } else {
-      qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
-        qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
+      qCommon = ByteBufferUtils
+        .findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(), qLen1,
+          c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
     }
     commonPrefix += qCommon;
     if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
       return commonPrefix;
     }
     // Compare the timestamp parts
-    int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
-      KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
+    int tsCommonPrefix = ByteBufferUtils
+      .findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE,
+        Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
     commonPrefix += tsCommonPrefix;
     if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
       return commonPrefix;
@@ -1544,24 +1560,24 @@ public final class CellUtil {
     short lrowlength = left.getRowLength();
     short rrowlength = right.getRowLength();
     if (lrowlength != rrowlength) return false;
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
-        ((ByteBufferCell) left).getRowPosition(), lrowlength,
-        ((ByteBufferCell) right).getRowByteBuffer(), ((ByteBufferCell) right).getRowPosition(),
-        rrowlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
+          ((ByteBufferExtendedCell) right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
-        ((ByteBufferCell) left).getRowPosition(), lrowlength, right.getRowArray(),
-        right.getRowOffset(), rrowlength);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
+          right.getRowOffset(), rrowlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getRowByteBuffer(),
-        ((ByteBufferCell) right).getRowPosition(), rrowlength, left.getRowArray(),
-        left.getRowOffset(), lrowlength);
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
+          left.getRowOffset(), lrowlength);
     }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
-      right.getRowOffset(), rrowlength);
+        right.getRowOffset(), rrowlength);
   }
 
   /**
@@ -1622,10 +1638,10 @@ public final class CellUtil {
    *         cell's qualifier is lesser than byte[] and 0 otherwise
    */
   public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
-        ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(), right, rOffset,
-        rLength);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(),
+          left.getQualifierLength(), right, rOffset, rLength);
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
       left.getQualifierLength(), right, rOffset, rLength);
@@ -1670,9 +1686,9 @@ public final class CellUtil {
    *         cell's family is lesser than byte[] and 0 otherwise
    */
   public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
-        ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
         rlength);
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
index b964d67..0afe273 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
@@ -26,7 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * {@link Tag} and memstoreTS/mvcc are internal implementation detail
  *  that should not be exposed publicly.
  * Use {@link ExtendedCellBuilderFactory} to get ExtendedCellBuilder instance.
- * TODO: ditto for ByteBufferCell?
+ * TODO: ditto for ByteBufferExtendedCell?
  */
 @InterfaceAudience.Private
 public interface ExtendedCellBuilder extends RawCellBuilder {