http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
index 363f0bd..69eaab0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
@@ -83,17 +83,17 @@ public final class PrivateCellUtil {
   /********************* misc *************************************/
 
   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];
   }
 
   public static byte getQualifierByte(Cell cell, int index) {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) cell).getQualifierByteBuffer()
-          .get(((ByteBufferCell) cell).getQualifierPosition() + index);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) cell).getQualifierByteBuffer()
+          .get(((ByteBufferExtendedCell) cell).getQualifierPosition() + index);
     }
     return cell.getQualifierArray()[cell.getQualifierOffset() + index];
   }
@@ -115,15 +115,16 @@ public final class PrivateCellUtil {
    * @return A new cell which is having the extra tags also added to it.
    */
   public static Cell createCell(Cell cell, byte[] tags) {
-    if (cell instanceof ByteBufferCell) {
-      return new TagRewriteByteBufferCell((ByteBufferCell) cell, tags);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) 
cell, tags);
     }
     return new TagRewriteCell(cell, tags);
   }
 
   public static Cell createCell(Cell cell, byte[] value, byte[] tags) {
-    if (cell instanceof ByteBufferCell) {
-      return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) cell, 
value, tags);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new 
ValueAndTagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell,
+          value, tags);
     }
     return new ValueAndTagRewriteCell(cell, value, tags);
   }
@@ -314,25 +315,24 @@ public final class PrivateCellUtil {
     }
   }
 
-  static class TagRewriteByteBufferCell extends ByteBufferCell implements 
ExtendedCell {
+  static class TagRewriteByteBufferExtendedCell extends ByteBufferExtendedCell 
{
 
-    protected ByteBufferCell cell;
+    protected ByteBufferExtendedCell cell;
     protected byte[] tags;
     private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * 
ClassSize.REFERENCE;
 
     /**
-     * @param cell The original ByteBufferCell which it rewrites
+     * @param cell The original ByteBufferExtendedCell which it rewrites
      * @param tags the tags bytes. The array suppose to contain the tags bytes 
alone.
      */
-    public TagRewriteByteBufferCell(ByteBufferCell cell, byte[] tags) {
-      assert cell instanceof ExtendedCell;
+    public TagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell, 
byte[] tags) {
       assert tags != null;
       this.cell = cell;
       this.tags = tags;
       // tag offset will be treated as 0 and length this.tags.length
-      if (this.cell instanceof TagRewriteByteBufferCell) {
+      if (this.cell instanceof TagRewriteByteBufferExtendedCell) {
         // Cleaning the ref so that the byte[] can be GCed
-        ((TagRewriteByteBufferCell) this.cell).tags = null;
+        ((TagRewriteByteBufferExtendedCell) this.cell).tags = null;
       }
     }
 
@@ -490,8 +490,9 @@ public final class PrivateCellUtil {
     @Override
     public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
-      if (clonedBaseCell instanceof ByteBufferCell) {
-        return new TagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, 
this.tags);
+      if (clonedBaseCell instanceof ByteBufferExtendedCell) {
+        return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) 
clonedBaseCell,
+            this.tags);
       }
       return new TagRewriteCell(clonedBaseCell, this.tags);
     }
@@ -585,7 +586,10 @@ public final class PrivateCellUtil {
       return write(out, withTags, this.cell, this.value, this.tags);
     }
 
-    // Made into a static method so as to reuse the logic within 
ValueAndTagRewriteByteBufferCell
+    /**
+     * Made into a static method so as to reuse the logic within
+     * ValueAndTagRewriteByteBufferExtendedCell
+     */
     static int write(OutputStream out, boolean withTags, Cell cell, byte[] 
value, byte[] tags)
         throws IOException {
       int valLen = value == null ? 0 : value.length;
@@ -617,7 +621,10 @@ public final class PrivateCellUtil {
       write(buf, offset, this.cell, this.value, this.tags);
     }
 
-    // Made into a static method so as to reuse the logic within 
ValueAndTagRewriteByteBufferCell
+    /**
+     * Made into a static method so as to reuse the logic
+     * within ValueAndTagRewriteByteBufferExtendedCell
+     */
     static void write(ByteBuffer buf, int offset, Cell cell, byte[] value, 
byte[] tags) {
       offset = ByteBufferUtils.putInt(buf, offset, 
KeyValueUtil.keyLength(cell));// Key length
       offset = ByteBufferUtils.putInt(buf, offset, value.length);// Value 
length
@@ -638,11 +645,12 @@ public final class PrivateCellUtil {
     }
   }
 
-  static class ValueAndTagRewriteByteBufferCell extends 
TagRewriteByteBufferCell {
+  static class ValueAndTagRewriteByteBufferExtendedCell extends 
TagRewriteByteBufferExtendedCell {
 
     protected byte[] value;
 
-    public ValueAndTagRewriteByteBufferCell(ByteBufferCell cell, byte[] value, 
byte[] tags) {
+    public ValueAndTagRewriteByteBufferExtendedCell(ByteBufferExtendedCell 
cell,
+        byte[] value, byte[] tags) {
       super(cell, tags);
       this.value = value;
     }
@@ -699,9 +707,9 @@ public final class PrivateCellUtil {
     @Override
     public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
-      if (clonedBaseCell instanceof ByteBufferCell) {
-        return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) 
clonedBaseCell, this.value,
-            this.tags);
+      if (clonedBaseCell instanceof ByteBufferExtendedCell) {
+        return new ValueAndTagRewriteByteBufferExtendedCell(
+            (ByteBufferExtendedCell) clonedBaseCell, this.value, this.tags);
       }
       return new ValueAndTagRewriteCell(clonedBaseCell, this.value, this.tags);
     }
@@ -709,19 +717,21 @@ public final class PrivateCellUtil {
 
   public static boolean matchingRows(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);
+        length);
   }
 
   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);
+    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);
@@ -740,10 +750,10 @@ public final class PrivateCellUtil {
     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);
@@ -759,24 +769,25 @@ public final class PrivateCellUtil {
 
   public static boolean matchingValue(final Cell left, final Cell right, int 
lvlength,
       int rvlength) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) 
left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), lvlength,
-        ((ByteBufferCell) right).getValueByteBuffer(), ((ByteBufferCell) 
right).getValuePosition(),
-        rvlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof 
ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) 
left).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
+          ((ByteBufferExtendedCell) right).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) 
left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), lvlength, 
right.getValueArray(),
-        right.getValueOffset(), rvlength);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) 
left).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, 
right.getValueArray(),
+          right.getValueOffset(), rvlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) 
right).getValueByteBuffer(),
-        ((ByteBufferCell) right).getValuePosition(), rvlength, 
left.getValueArray(),
-        left.getValueOffset(), lvlength);
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) 
right).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, 
left.getValueArray(),
+          left.getValueOffset(), lvlength);
     }
-    return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
-      right.getValueArray(), right.getValueOffset(), rvlength);
+    return Bytes
+        .equals(left.getValueArray(), left.getValueOffset(), lvlength, 
right.getValueArray(),
+            right.getValueOffset(), rvlength);
   }
 
   public static boolean matchingType(Cell a, Cell b) {
@@ -837,13 +848,13 @@ public final class PrivateCellUtil {
    */
   public static int copyTagsTo(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;
   }
@@ -857,9 +868,9 @@ public final class PrivateCellUtil {
    */
   public static int copyTagsTo(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);
@@ -887,14 +898,15 @@ public final class PrivateCellUtil {
    * @return null if there is no tag of the passed in tag type
    */
   public static Optional<Tag> getTag(Cell cell, byte type) {
-    boolean bufferBacked = cell instanceof ByteBufferCell;
+    boolean bufferBacked = cell instanceof ByteBufferExtendedCell;
     int length = cell.getTagsLength();
-    int offset = bufferBacked ? ((ByteBufferCell) cell).getTagsPosition() : 
cell.getTagsOffset();
+    int offset =
+      bufferBacked ? ((ByteBufferExtendedCell) cell).getTagsPosition() : 
cell.getTagsOffset();
     int pos = offset;
     while (pos < offset + length) {
       int tagLen;
       if (bufferBacked) {
-        ByteBuffer tagsBuffer = ((ByteBufferCell) cell).getTagsByteBuffer();
+        ByteBuffer tagsBuffer = ((ByteBufferExtendedCell) 
cell).getTagsByteBuffer();
         tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
         if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) 
{
           return Optional.of(new ByteBufferTag(tagsBuffer, pos, tagLen + 
TAG_LENGTH_SIZE));
@@ -922,9 +934,9 @@ public final class PrivateCellUtil {
     if (tagsLength == 0) {
       return TagUtil.EMPTY_TAGS_ITR;
     }
-    if (cell instanceof ByteBufferCell) {
-      return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return tagsIterator(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
     }
     return CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), 
cell.getTagsLength());
   }
@@ -1000,10 +1012,11 @@ public final class PrivateCellUtil {
    */
   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);
+    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);
     }
@@ -1028,23 +1041,27 @@ public final class PrivateCellUtil {
     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) {
@@ -1067,14 +1084,16 @@ public final class PrivateCellUtil {
       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) {
@@ -1085,22 +1104,25 @@ public final class PrivateCellUtil {
     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;
@@ -1195,9 +1217,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's row
    */
   public static int compareRow(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) 
cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
     }
     return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength());
   }
@@ -1209,9 +1231,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's column family
    */
   public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) 
cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) 
cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), 
cell.getFamilyLength());
     }
     return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
       cell.getFamilyLength());
@@ -1224,9 +1246,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's qualifier
    */
   public static int compareQualifier(Cell cell, ByteArrayComparable 
comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) 
cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), 
cell.getQualifierLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) 
cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), 
cell.getQualifierLength());
     }
     return comparator.compareTo(cell.getQualifierArray(), 
cell.getQualifierOffset(),
       cell.getQualifierLength());
@@ -1262,9 +1284,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's value
    */
   public static int compareValue(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) 
cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition(), 
cell.getValueLength());
     }
     return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), 
cell.getValueLength());
   }
@@ -1375,8 +1397,7 @@ public final class PrivateCellUtil {
    * These cells are used in reseeks/seeks to improve the read performance. 
They are not real cells
    * that are returned back to the clients
    */
-  private static abstract class EmptyByteBufferCell extends ByteBufferCell
-      implements ExtendedCell {
+  private static abstract class EmptyByteBufferExtendedCell extends 
ByteBufferExtendedCell {
 
     @Override
     public void setSequenceId(long seqId) {
@@ -1578,7 +1599,7 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class FirstOnRowByteBufferCell extends EmptyByteBufferCell {
+  private static class FirstOnRowByteBufferExtendedCell extends 
EmptyByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
         ClassSize.OBJECT // object
         + ClassSize.REFERENCE // row buffer
@@ -1588,7 +1609,7 @@ public final class PrivateCellUtil {
     private final int roffset;
     private final short rlength;
 
-    public FirstOnRowByteBufferCell(final ByteBuffer row, int roffset, short 
rlength) {
+    public FirstOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, 
short rlength) {
       this.rowBuff = row;
       this.roffset = roffset;
       this.rlength = rlength;
@@ -1633,7 +1654,7 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class LastOnRowByteBufferCell extends EmptyByteBufferCell {
+  private static class LastOnRowByteBufferExtendedCell extends 
EmptyByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
         ClassSize.OBJECT // object
       + ClassSize.REFERENCE // rowBuff
@@ -1643,7 +1664,7 @@ public final class PrivateCellUtil {
     private final int roffset;
     private final short rlength;
 
-    public LastOnRowByteBufferCell(final ByteBuffer row, int roffset, short 
rlength) {
+    public LastOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, 
short rlength) {
       this.rowBuff = row;
       this.roffset = roffset;
       this.rlength = rlength;
@@ -1688,9 +1709,10 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class FirstOnRowColByteBufferCell extends 
FirstOnRowByteBufferCell {
+  private static class FirstOnRowColByteBufferExtendedCell
+      extends FirstOnRowByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
-        FirstOnRowByteBufferCell.FIXED_OVERHEAD
+        FirstOnRowByteBufferExtendedCell.FIXED_OVERHEAD
         + ClassSize.REFERENCE * 2 // family buffer and column buffer
         + Bytes.SIZEOF_INT * 3 // famOffset, colOffset, colLength
         + Bytes.SIZEOF_BYTE; // famLength
@@ -1701,7 +1723,7 @@ public final class PrivateCellUtil {
     private final int colOffset;
     private final int colLength;
 
-    public FirstOnRowColByteBufferCell(final ByteBuffer row, int roffset, 
short rlength,
+    public FirstOnRowColByteBufferExtendedCell(final ByteBuffer row, int 
roffset, short rlength,
         final ByteBuffer famBuff, final int famOffset, final byte famLength, 
final ByteBuffer col,
         final int colOffset, final int colLength) {
       super(row, roffset, rlength);
@@ -1843,13 +1865,14 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class FirstOnRowColTSByteBufferCell extends 
FirstOnRowColByteBufferCell {
+  private static class FirstOnRowColTSByteBufferExtendedCell
+      extends FirstOnRowColByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
-        FirstOnRowColByteBufferCell.FIXED_OVERHEAD
+        FirstOnRowColByteBufferExtendedCell.FIXED_OVERHEAD
             + Bytes.SIZEOF_LONG; // ts
     private long ts;
 
-    public FirstOnRowColTSByteBufferCell(ByteBuffer rBuffer, int roffset, 
short rlength,
+    public FirstOnRowColTSByteBufferExtendedCell(ByteBuffer rBuffer, int 
roffset, short rlength,
         ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int 
qoffset, int qlength,
         long ts) {
       super(rBuffer, roffset, rlength, fBuffer, foffset, flength, qBuffer, 
qoffset, qlength);
@@ -1983,9 +2006,9 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class LastOnRowColByteBufferCell extends 
LastOnRowByteBufferCell {
+  private static class LastOnRowColByteBufferExtendedCell extends 
LastOnRowByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
-        LastOnRowByteBufferCell.FIXED_OVERHEAD
+        LastOnRowByteBufferExtendedCell.FIXED_OVERHEAD
             + ClassSize.REFERENCE * 2 // fBuffer and qBuffer
             + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
             + Bytes.SIZEOF_BYTE; // flength
@@ -1996,7 +2019,7 @@ public final class PrivateCellUtil {
     private final int qoffset;
     private final int qlength;
 
-    public LastOnRowColByteBufferCell(ByteBuffer rBuffer, int roffset, short 
rlength,
+    public LastOnRowColByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, 
short rlength,
         ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int 
qoffset,
         int qlength) {
       super(rBuffer, roffset, rlength);
@@ -2125,15 +2148,16 @@ public final class PrivateCellUtil {
     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(out, ((ByteBufferCell) 
cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), rowLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
       out.writeByte(fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
+      ByteBufferUtils
+        .copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
     } else {
       out.writeShort(rowLen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
@@ -2214,15 +2238,16 @@ public final class PrivateCellUtil {
     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) {
       StreamUtils.writeShort(out, rowLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), rowLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
       out.write(fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
+      ByteBufferUtils
+        .copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
     } else {
       StreamUtils.writeShort(out, rowLen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
@@ -2233,7 +2258,7 @@ public final class PrivateCellUtil {
     StreamUtils.writeLong(out, cell.getTimestamp());
     out.write(cell.getTypeByte());
     return Bytes.SIZEOF_SHORT + rowLen + Bytes.SIZEOF_BYTE + fLen + qLen + 
Bytes.SIZEOF_LONG
-        + Bytes.SIZEOF_BYTE;
+      + Bytes.SIZEOF_BYTE;
   }
 
   /**
@@ -2322,9 +2347,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeRow(OutputStream out, Cell cell, short rlength) 
throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), rlength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), rlength);
     } else {
       out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
     }
@@ -2338,9 +2363,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeFamily(OutputStream out, Cell cell, byte flength) 
throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), flength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), flength);
     } else {
       out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
     }
@@ -2354,9 +2379,10 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeQualifier(OutputStream out, Cell cell, int qlength) 
throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qlength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils
+        .copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qlength);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
     }
@@ -2371,13 +2397,14 @@ public final class PrivateCellUtil {
    */
   public static void writeQualifierSkippingBytes(DataOutputStream out, Cell 
cell, int qlength,
       int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength 
- commonPrefix);
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition() + 
commonPrefix,
+          qlength - commonPrefix);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset() + 
commonPrefix,
-        qlength - commonPrefix);
+          qlength - commonPrefix);
     }
   }
 
@@ -2389,9 +2416,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeValue(OutputStream out, Cell cell, int vlength) 
throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition(), vlength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition(), vlength);
     } else {
       out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
     }
@@ -2405,9 +2432,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeTags(OutputStream out, Cell cell, int tagsLength) 
throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) 
cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) 
cell).getTagsByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
     } else {
       out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
     }
@@ -2440,9 +2467,9 @@ public final class PrivateCellUtil {
    * @return rowkey as int
    */
   public static int getRowAsInt(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toInt(((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) 
cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition());
     }
     return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
   }
@@ -2453,9 +2480,9 @@ public final class PrivateCellUtil {
    * @return value as long
    */
   public static long getValueAsLong(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toLong(((ByteBufferCell) 
cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toLong(((ByteBufferExtendedCell) 
cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition());
     }
     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
   }
@@ -2466,9 +2493,9 @@ public final class PrivateCellUtil {
    * @return value as int
    */
   public static int getValueAsInt(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toInt(((ByteBufferCell) 
cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) 
cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition());
     }
     return Bytes.toInt(cell.getValueArray(), cell.getValueOffset());
   }
@@ -2479,9 +2506,9 @@ public final class PrivateCellUtil {
    * @return value as double
    */
   public static double getValueAsDouble(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toDouble(((ByteBufferCell) 
cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toDouble(((ByteBufferExtendedCell) 
cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition());
     }
     return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
   }
@@ -2492,9 +2519,9 @@ public final class PrivateCellUtil {
    * @return value as BigDecimal
    */
   public static BigDecimal getValueAsBigDecimal(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toBigDecimal(((ByteBufferCell) 
cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toBigDecimal(((ByteBufferExtendedCell) 
cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition(), 
cell.getValueLength());
     }
     return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), 
cell.getValueLength());
   }
@@ -2508,9 +2535,9 @@ public final class PrivateCellUtil {
    */
   public static void compressTags(OutputStream out, Cell cell,
       TagCompressionContext tagCompressionContext) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      tagCompressionContext.compressTags(out, ((ByteBufferCell) 
cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), cell.getTagsLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      tagCompressionContext.compressTags(out, ((ByteBufferExtendedCell) 
cell).getTagsByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getTagsPosition(), 
cell.getTagsLength());
     } else {
       tagCompressionContext.compressTags(out, cell.getTagsArray(), 
cell.getTagsOffset(),
         cell.getTagsLength());
@@ -2518,9 +2545,9 @@ public final class PrivateCellUtil {
   }
 
   public static void compressRow(OutputStream out, Cell cell, Dictionary dict) 
throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      Dictionary.write(out, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(), dict);
+    if (cell instanceof ByteBufferExtendedCell) {
+      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(), 
dict);
     } else {
       Dictionary.write(out, cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(), dict);
     }
@@ -2528,9 +2555,9 @@ public final class PrivateCellUtil {
 
   public static void compressFamily(OutputStream out, Cell cell, Dictionary 
dict)
       throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      Dictionary.write(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(), 
dict);
+    if (cell instanceof ByteBufferExtendedCell) {
+      Dictionary.write(out, ((ByteBufferExtendedCell) 
cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), 
cell.getFamilyLength(), dict);
     } else {
       Dictionary.write(out, cell.getFamilyArray(), cell.getFamilyOffset(), 
cell.getFamilyLength(),
         dict);
@@ -2539,9 +2566,9 @@ public final class PrivateCellUtil {
 
   public static void compressQualifier(OutputStream out, Cell cell, Dictionary 
dict)
       throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      Dictionary.write(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), 
cell.getQualifierLength(), dict);
+    if (cell instanceof ByteBufferExtendedCell) {
+      Dictionary.write(out, ((ByteBufferExtendedCell) 
cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), 
cell.getQualifierLength(), dict);
     } else {
       Dictionary.write(out, cell.getQualifierArray(), 
cell.getQualifierOffset(),
         cell.getQualifierLength(), dict);
@@ -2668,13 +2695,14 @@ public final class PrivateCellUtil {
   }
 
   static Cell createNextOnRowCol(Cell cell, long ts, byte type) {
-    if (cell instanceof ByteBufferCell) {
-      return new LastOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) 
cell).getRowPosition(),
-          cell.getRowLength(), ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), 
cell.getQualifierLength()) {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new LastOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), 
cell.getFamilyLength(),
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), 
cell.getQualifierLength()) {
         @Override
         public long getTimestamp() {
           return ts;
@@ -2686,8 +2714,8 @@ public final class PrivateCellUtil {
         }
       };
     }
-    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
-        cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), 
cell.getFamilyLength(),
+    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
         cell.getQualifierArray(), cell.getQualifierOffset(), 
cell.getQualifierLength()) {
       @Override
       public long getTimestamp() {
@@ -2788,13 +2816,12 @@ public final class PrivateCellUtil {
    * @return First possible Cell on passed Cell's row.
    */
   public static Cell createFirstOnRow(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) 
cell).getRowPosition(),
-          cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength());
     }
-    return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(),
-        cell.getRowLength());
+    return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength());
   }
 
   public static Cell createFirstOnRow(final byte[] row, int roffset, short 
rlength) {
@@ -2803,7 +2830,7 @@ public final class PrivateCellUtil {
 
   public static Cell createFirstOnRow(final byte[] row, final byte[] family, 
final byte[] col) {
     return createFirstOnRow(row, 0, (short) row.length, family, 0, (byte) 
family.length, col, 0,
-      col.length);
+        col.length);
   }
 
   public static Cell createFirstOnRow(final byte[] row, int roffset, short 
rlength,
@@ -2817,23 +2844,24 @@ public final class PrivateCellUtil {
   }
 
   public static Cell createFirstOnRowFamily(Cell cell, byte[] fArray, int 
foff, int flen) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) 
cell).getRowPosition(),
-          cell.getRowLength(), ByteBuffer.wrap(fArray), foff, (byte) flen,
-          HConstants.EMPTY_BYTE_BUFFER, 0, 0);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength(),
+          ByteBuffer.wrap(fArray), foff, (byte) flen, 
HConstants.EMPTY_BYTE_BUFFER, 0, 0);
     }
     return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(),
         fArray, foff, (byte) flen, HConstants.EMPTY_BYTE_ARRAY, 0, 0);
   }
 
   public static Cell createFirstOnRowCol(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) 
cell).getRowPosition(),
-          cell.getRowLength(), HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), 
cell.getQualifierLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength(),
+          HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), 
cell.getQualifierLength());
     }
     return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(),
         HConstants.EMPTY_BYTE_ARRAY, 0, (byte) 0, cell.getQualifierArray(),
@@ -2857,16 +2885,17 @@ public final class PrivateCellUtil {
    * @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
    */
   public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int 
qoffest, int qlength) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) 
cell).getRowPosition(),
-          cell.getRowLength(), ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), 
cell.getFamilyLength(),
           ByteBuffer.wrap(qArray), qoffest, qlength);
     }
-    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
-        cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), 
cell.getFamilyLength(),
-        qArray, qoffest, qlength);
+    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), 
qArray, qoffest,
+        qlength);
   }
 
   /**
@@ -2877,13 +2906,14 @@ public final class PrivateCellUtil {
    * @param ts
    */
   public static Cell createFirstOnRowColTS(Cell cell, long ts) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColTSByteBufferCell(((ByteBufferCell) 
cell).getRowByteBuffer(),
-          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), 
cell.getQualifierLength(), ts);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColTSByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), 
cell.getFamilyLength(),
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), 
cell.getQualifierLength(), ts);
     }
     return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(),
         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
@@ -2896,10 +2926,9 @@ public final class PrivateCellUtil {
    * @return Last possible Cell on passed Cell's row.
    */
   public static Cell createLastOnRow(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new LastOnRowByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) 
cell).getRowPosition(),
-          cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new LastOnRowByteBufferExtendedCell(((ByteBufferExtendedCell) 
cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength());
     }
     return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength());
   }
@@ -2916,13 +2945,14 @@ public final class PrivateCellUtil {
    * @return Last possible Cell on passed Cell's rk:cf:q.
    */
   public static Cell createLastOnRowCol(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new LastOnRowColByteBufferCell(((ByteBufferCell) 
cell).getRowByteBuffer(),
-          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), 
cell.getQualifierLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new LastOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), 
cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), 
cell.getFamilyLength(),
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), 
cell.getQualifierLength());
     }
     return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength(),
         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
index 34c78a5..9527173 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
@@ -99,9 +99,9 @@ public final class TagUtil {
     }
     byte[] b = new byte[tags.length + cellTagsLen];
     int pos = Bytes.putBytes(b, 0, tags, 0, tags.length);
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(b, ((ByteBufferCell) 
cell).getTagsByteBuffer(),
-          ((ByteBufferCell) cell).getTagsPosition(), pos, cellTagsLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToArray(b, ((ByteBufferExtendedCell) 
cell).getTagsByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getTagsPosition(), pos, cellTagsLen);
     } else {
       Bytes.putBytes(b, pos, cell.getTagsArray(), cell.getTagsOffset(), 
cellTagsLen);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index 9bcda01..7fe2b71 100644
--- 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -21,8 +21,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -252,7 +251,7 @@ abstract class BufferedDataBlockEncoder extends 
AbstractDataBlockEncoder {
 
     private Cell toOffheapCell(ByteBuffer valAndTagsBuffer, int vOffset,
         int tagsLenSerializationSize) {
-      ByteBuffer tagsBuf =  HConstants.EMPTY_BYTE_BUFFER;
+      ByteBuffer tagsBuf = HConstants.EMPTY_BYTE_BUFFER;
       int tOffset = 0;
       if (this.includeTags) {
         if (this.tagCompressionContext == null) {
@@ -263,8 +262,9 @@ abstract class BufferedDataBlockEncoder extends 
AbstractDataBlockEncoder {
           tOffset = 0;
         }
       }
-      return new OffheapDecodedCell(ByteBuffer.wrap(Bytes.copy(keyBuffer, 0, 
this.keyLength)),
-          currentKey.getRowLength(), currentKey.getFamilyOffset(), 
currentKey.getFamilyLength(),
+      return new OffheapDecodedExtendedCell(
+          ByteBuffer.wrap(Bytes.copy(keyBuffer, 0, this.keyLength)), 
currentKey.getRowLength(),
+          currentKey.getFamilyOffset(), currentKey.getFamilyLength(),
           currentKey.getQualifierOffset(), currentKey.getQualifierLength(),
           currentKey.getTimestamp(), currentKey.getTypeByte(), 
valAndTagsBuffer, vOffset,
           this.valueLength, memstoreTS, tagsBuf, tOffset, this.tagsLength);
@@ -477,7 +477,7 @@ abstract class BufferedDataBlockEncoder extends 
AbstractDataBlockEncoder {
     }
   }
 
-  protected static class OffheapDecodedCell extends ByteBufferCell implements 
ExtendedCell {
+  protected static class OffheapDecodedExtendedCell extends 
ByteBufferExtendedCell {
     private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
         + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * 
Bytes.SIZEOF_INT)
         + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * 
ClassSize.BYTE_BUFFER));
@@ -497,7 +497,7 @@ abstract class BufferedDataBlockEncoder extends 
AbstractDataBlockEncoder {
     private int tagsLength;
     private long seqId;
 
-    protected OffheapDecodedCell(ByteBuffer keyBuffer, short rowLength, int 
familyOffset,
+    protected OffheapDecodedExtendedCell(ByteBuffer keyBuffer, short 
rowLength, int familyOffset,
         byte familyLength, int qualOffset, int qualLength, long timeStamp, 
byte typeByte,
         ByteBuffer valueBuffer, int valueOffset, int valueLen, long seqId, 
ByteBuffer tagsBuffer,
         int tagsOffset, int tagsLength) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
index 02ed721..1dd4e19 100644
--- 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
+++ 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
@@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.io.encoding;
 
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
@@ -154,10 +154,10 @@ public class RowIndexSeekerV1 extends 
AbstractEncodedSeeker {
   }
 
   private int compareRows(ByteBuffer row, Cell seekCell) {
-    if (seekCell instanceof ByteBufferCell) {
+    if (seekCell instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
-          ((ByteBufferCell) seekCell).getRowByteBuffer(),
-          ((ByteBufferCell) seekCell).getRowPosition(),
+          ((ByteBufferExtendedCell) seekCell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) seekCell).getRowPosition(),
           seekCell.getRowLength());
     } else {
       return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
----------------------------------------------------------------------
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 030bc8f..97a4276 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
@@ -56,7 +56,7 @@ public class TestByteBufferKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0L, Type.Put, row1);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, 
kvCell.getBuffer().length);
-    ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 
0L);
+    ByteBufferExtendedCell offheapKV = new ByteBufferKeyValue(buf, 0, 
buf.capacity(), 0L);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -179,7 +179,7 @@ public class TestByteBufferKeyValue {
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getKeyLength());
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 
kvCell.getKeyOffset(),
       kvCell.getKeyLength());
-    ByteBufferCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, 
buf.capacity());
+    ByteBufferExtendedCell offheapKeyOnlyKV = new 
ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKeyOnlyKV.getRowByteBuffer(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
 
b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
index 5f4115e..96181e1 100644
--- 
a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
+++ 
b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
@@ -27,7 +27,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.ArrayBackedTag;
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
@@ -80,10 +80,10 @@ public class TestTagCompressionContext {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
     TagCompressionContext context = new 
TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
-    ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(2);
+    ByteBufferExtendedCell kv1 = 
(ByteBufferExtendedCell)createOffheapKVWithTags(2);
     int tagsLength1 = kv1.getTagsLength();
     context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), 
tagsLength1);
-    ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
+    ByteBufferExtendedCell kv2 = 
(ByteBufferExtendedCell)createOffheapKVWithTags(3);
     int tagsLength2 = kv2.getTagsLength();
     context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), 
tagsLength2);
 
@@ -129,10 +129,10 @@ public class TestTagCompressionContext {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
     TagCompressionContext context = new 
TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
-    ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(1);
+    ByteBufferExtendedCell kv1 = 
(ByteBufferExtendedCell)createOffheapKVWithTags(1);
     int tagsLength1 = kv1.getTagsLength();
     context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), 
tagsLength1);
-    ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
+    ByteBufferExtendedCell kv2 = 
(ByteBufferExtendedCell)createOffheapKVWithTags(3);
     int tagsLength2 = kv2.getTagsLength();
     context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), 
tagsLength2);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
----------------------------------------------------------------------
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
index 499accb..de961cf 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -53,7 +53,7 @@ public class CellSortReducer
     context.setStatus("Read " + map.getClass());
     int index = 0;
     for (Cell kv: map) {
-      context.write(row, new MapReduceCell(kv));
+      context.write(row, new MapReduceExtendedCell(kv));
       if (++index % 100 == 0) context.setStatus("Wrote " + index);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
----------------------------------------------------------------------
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index 4a67933..312b315 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -75,7 +75,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -586,7 +586,7 @@ public class HFileOutputFormat2
       Class<? extends OutputFormat<?, ?>> cls) throws IOException {
     Configuration conf = job.getConfiguration();
     job.setOutputKeyClass(ImmutableBytesWritable.class);
-    job.setOutputValueClass(MapReduceCell.class);
+    job.setOutputValueClass(MapReduceExtendedCell.class);
     job.setOutputFormatClass(cls);
 
     if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {
@@ -601,7 +601,7 @@ public class HFileOutputFormat2
     // sort the incoming values.
     // TODO it would be nice to pick one or the other of these formats.
     if (KeyValue.class.equals(job.getMapOutputValueClass())
-        || MapReduceCell.class.equals(job.getMapOutputValueClass())) {
+        || MapReduceExtendedCell.class.equals(job.getMapOutputValueClass())) {
       job.setReducerClass(CellSortReducer.class);
     } else if (Put.class.equals(job.getMapOutputValueClass())) {
       job.setReducerClass(PutSortReducer.class);
@@ -661,7 +661,7 @@ public class HFileOutputFormat2
     Configuration conf = job.getConfiguration();
 
     job.setOutputKeyClass(ImmutableBytesWritable.class);
-    job.setOutputValueClass(MapReduceCell.class);
+    job.setOutputValueClass(MapReduceExtendedCell.class);
     job.setOutputFormatClass(HFileOutputFormat2.class);
 
     ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index e8e1de0..e2b22d7 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
@@ -61,7 +62,6 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MapReduceCell;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableComparable;
@@ -180,7 +180,7 @@ public class Import extends Configured implements Tool {
       int index = 0;
       for (Cell kv : kvs) {
         context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),
-          new MapReduceCell(kv));
+          new MapReduceExtendedCell(kv));
         if (++index % 100 == 0)
           context.setStatus("Wrote " + index + " KeyValues, "
               + "and the rowkey whose is being wrote is " + 
Bytes.toString(kv.getRowArray()));
@@ -280,7 +280,7 @@ public class Import extends Configured implements Tool {
             kv = filterKv(filter, kv);
             // skip if we filtered it out
             if (kv == null) continue;
-            context.write(row, new MapReduceCell(convertKv(kv, cfRenameMap)));
+            context.write(row, new MapReduceExtendedCell(convertKv(kv, 
cfRenameMap)));
           }
         }
       } catch (InterruptedException e) {
@@ -631,7 +631,7 @@ public class Import extends Configured implements Tool {
         Path outputDir = new Path(hfileOutPath);
         FileOutputFormat.setOutputPath(job, outputDir);
         job.setMapOutputKeyClass(CellWritableComparable.class);
-        job.setMapOutputValueClass(MapReduceCell.class);
+        job.setMapOutputValueClass(MapReduceExtendedCell.class);
         
job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
             CellWritableComparable.CellWritableComparator.class,
             RawComparator.class);
@@ -654,7 +654,7 @@ public class Import extends Configured implements Tool {
         Path outputDir = new Path(hfileOutPath);
         FileOutputFormat.setOutputPath(job, outputDir);
         job.setMapOutputKeyClass(ImmutableBytesWritable.class);
-        job.setMapOutputValueClass(MapReduceCell.class);
+        job.setMapOutputValueClass(MapReduceExtendedCell.class);
         HFileOutputFormat2.configureIncrementalLoad(job, 
table.getDescriptor(), regionLocator);
         TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
             
org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 7212d4b..3c3a1c0 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.mapreduce.Job;
@@ -111,7 +111,7 @@ public class WALPlayer extends Configured implements Tool {
               continue;
             }
             context.write(new ImmutableBytesWritable(CellUtil.cloneRow(cell)),
-              new MapReduceCell(cell));
+              new MapReduceExtendedCell(cell));
           }
         }
       } catch (InterruptedException e) {
@@ -302,7 +302,7 @@ public class WALPlayer extends Configured implements Tool {
       job.setReducerClass(CellSortReducer.class);
       Path outputDir = new Path(hfileOutPath);
       FileOutputFormat.setOutputPath(job, outputDir);
-      job.setMapOutputValueClass(MapReduceCell.class);
+      job.setMapOutputValueClass(MapReduceExtendedCell.class);
       try (Connection conn = ConnectionFactory.createConnection(conf);
           Table table = conn.getTable(tableName);
           RegionLocator regionLocator = conn.getRegionLocator(tableName)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java
----------------------------------------------------------------------
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java
deleted file mode 100644
index 38ff59b..0000000
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java
+++ /dev/null
@@ -1,271 +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.util;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferCell;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ExtendedCell;
-import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * A wrapper for a cell to be used with mapreduce, as the output value class 
for mappers/reducers.
- */
-@InterfaceAudience.Private
-public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
-
-  private final Cell cell;
-
-  public MapReduceCell(Cell cell) {
-    this.cell = cell;
-  }
-
-  @Override
-  public byte[] getRowArray() {
-    return this.cell.getRowArray();
-  }
-
-  @Override
-  public int getRowOffset() {
-    return this.cell.getRowOffset();
-  }
-
-  @Override
-  public short getRowLength() {
-    return this.cell.getRowLength();
-  }
-
-  @Override
-  public byte[] getFamilyArray() {
-    return this.cell.getFamilyArray();
-  }
-
-  @Override
-  public int getFamilyOffset() {
-    return this.cell.getFamilyOffset();
-  }
-
-  @Override
-  public byte getFamilyLength() {
-    return this.cell.getFamilyLength();
-  }
-
-  @Override
-  public byte[] getQualifierArray() {
-    return this.cell.getQualifierArray();
-  }
-
-  @Override
-  public int getQualifierOffset() {
-    return this.cell.getQualifierOffset();
-  }
-
-  @Override
-  public int getQualifierLength() {
-    return this.cell.getQualifierLength();
-  }
-
-  @Override
-  public long getTimestamp() {
-    return this.cell.getTimestamp();
-  }
-
-  @Override
-  public byte getTypeByte() {
-    return this.cell.getTypeByte();
-  }
-
-  @Override
-  public long getSequenceId() {
-    return this.cell.getSequenceId();
-  }
-
-  @Override
-  public byte[] getValueArray() {
-    return this.cell.getValueArray();
-  }
-
-  @Override
-  public int getValueOffset() {
-    return this.cell.getValueOffset();
-  }
-
-  @Override
-  public int getValueLength() {
-    return this.cell.getValueLength();
-  }
-
-  @Override
-  public byte[] getTagsArray() {
-    return this.cell.getTagsArray();
-  }
-
-  @Override
-  public int getTagsOffset() {
-    return this.cell.getTagsOffset();
-  }
-
-  @Override
-  public int getTagsLength() {
-    return this.cell.getTagsLength();
-  }
-
-  @Override
-  public ByteBuffer getRowByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getRowByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneRow(this.cell));
-    }
-  }
-
-  @Override
-  public int getRowPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getRowPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getFamilyByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getFamilyByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneFamily(this.cell));
-    }
-  }
-
-  @Override
-  public int getFamilyPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getFamilyPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getQualifierByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getQualifierByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneQualifier(this.cell));
-    }
-  }
-
-  @Override
-  public int getQualifierPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getQualifierPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getValueByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getValueByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneValue(this.cell));
-    }
-  }
-
-  @Override
-  public int getValuePosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getValuePosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getTagsByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getTagsByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneTags(this.cell));
-    }
-  }
-
-  @Override
-  public int getTagsPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getTagsPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return this.cell.toString();
-  }
-
-  @Override
-  public void setSequenceId(long seqId) throws IOException {
-    PrivateCellUtil.setSequenceId(cell, seqId);
-  }
-
-  @Override
-  public void setTimestamp(long ts) throws IOException {
-    PrivateCellUtil.setTimestamp(cell, ts);
-  }
-
-  @Override
-  public void setTimestamp(byte[] ts) throws IOException {
-    PrivateCellUtil.setTimestamp(cell, ts);
-  }
-
-  @Override
-  public long heapSize() {
-    return PrivateCellUtil.estimatedHeapSizeOf(cell);
-  }
-
-  @Override
-  public int write(OutputStream out, boolean withTags) throws IOException {
-    return PrivateCellUtil.writeCell(cell, out, withTags);
-  }
-
-  @Override
-  public int getSerializedSize(boolean withTags) {
-    return PrivateCellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
-  }
-
-  @Override
-  public void write(ByteBuffer buf, int offset) {
-    PrivateCellUtil.writeCellToBuffer(cell, buf, offset);
-  }
-
-  @Override
-  public ExtendedCell deepClone() {
-    try {
-      return (ExtendedCell) PrivateCellUtil.deepClone(cell);
-    } catch (CloneNotSupportedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
----------------------------------------------------------------------
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
new file mode 100644
index 0000000..73eb7d8
--- /dev/null
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
@@ -0,0 +1,270 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A wrapper for a cell to be used with mapreduce, as the output value class 
for mappers/reducers.
+ */
+@InterfaceAudience.Private
+public class MapReduceExtendedCell extends ByteBufferExtendedCell {
+
+  private final Cell cell;
+
+  public MapReduceExtendedCell(Cell cell) {
+    this.cell = cell;
+  }
+
+  @Override
+  public byte[] getRowArray() {
+    return this.cell.getRowArray();
+  }
+
+  @Override
+  public int getRowOffset() {
+    return this.cell.getRowOffset();
+  }
+
+  @Override
+  public short getRowLength() {
+    return this.cell.getRowLength();
+  }
+
+  @Override
+  public byte[] getFamilyArray() {
+    return this.cell.getFamilyArray();
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    return this.cell.getFamilyOffset();
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return this.cell.getFamilyLength();
+  }
+
+  @Override
+  public byte[] getQualifierArray() {
+    return this.cell.getQualifierArray();
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    return this.cell.getQualifierOffset();
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return this.cell.getQualifierLength();
+  }
+
+  @Override
+  public long getTimestamp() {
+    return this.cell.getTimestamp();
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return this.cell.getTypeByte();
+  }
+
+  @Override
+  public long getSequenceId() {
+    return this.cell.getSequenceId();
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    return this.cell.getValueArray();
+  }
+
+  @Override
+  public int getValueOffset() {
+    return this.cell.getValueOffset();
+  }
+
+  @Override
+  public int getValueLength() {
+    return this.cell.getValueLength();
+  }
+
+  @Override
+  public byte[] getTagsArray() {
+    return this.cell.getTagsArray();
+  }
+
+  @Override
+  public int getTagsOffset() {
+    return this.cell.getTagsOffset();
+  }
+
+  @Override
+  public int getTagsLength() {
+    return this.cell.getTagsLength();
+  }
+
+  @Override
+  public ByteBuffer getRowByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getRowByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneRow(this.cell));
+    }
+  }
+
+  @Override
+  public int getRowPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getRowPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getFamilyByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getFamilyByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneFamily(this.cell));
+    }
+  }
+
+  @Override
+  public int getFamilyPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getFamilyPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getQualifierByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getQualifierByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneQualifier(this.cell));
+    }
+  }
+
+  @Override
+  public int getQualifierPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getQualifierPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getValueByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getValueByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneValue(this.cell));
+    }
+  }
+
+  @Override
+  public int getValuePosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getValuePosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getTagsByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getTagsByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneTags(this.cell));
+    }
+  }
+
+  @Override
+  public int getTagsPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getTagsPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return this.cell.toString();
+  }
+
+  @Override
+  public void setSequenceId(long seqId) throws IOException {
+    PrivateCellUtil.setSequenceId(cell, seqId);
+  }
+
+  @Override
+  public void setTimestamp(long ts) throws IOException {
+    PrivateCellUtil.setTimestamp(cell, ts);
+  }
+
+  @Override
+  public void setTimestamp(byte[] ts) throws IOException {
+    PrivateCellUtil.setTimestamp(cell, ts);
+  }
+
+  @Override
+  public long heapSize() {
+    return PrivateCellUtil.estimatedHeapSizeOf(cell);
+  }
+
+  @Override
+  public int write(OutputStream out, boolean withTags) throws IOException {
+    return PrivateCellUtil.writeCell(cell, out, withTags);
+  }
+
+  @Override
+  public int getSerializedSize(boolean withTags) {
+    return PrivateCellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
+  }
+
+  @Override
+  public void write(ByteBuffer buf, int offset) {
+    PrivateCellUtil.writeCellToBuffer(cell, buf, offset);
+  }
+
+  @Override
+  public ExtendedCell deepClone() {
+    try {
+      return (ExtendedCell) PrivateCellUtil.deepClone(cell);
+    } catch (CloneNotSupportedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

Reply via email to