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:56 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/branch-2 0454a389a -> 2468f3002


http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java
index b51df3f..87c79ec 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedImportExport2.java
@@ -70,7 +70,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -676,12 +676,12 @@ public class TestCellBasedImportExport2 {
       @Override
       public Void answer(InvocationOnMock invocation) throws Throwable {
         ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArguments()[0];
-        MapReduceCell key = (MapReduceCell) invocation.getArguments()[1];
+        MapReduceExtendedCell key = (MapReduceExtendedCell) invocation.getArguments()[1];
         assertEquals("Key", Bytes.toString(writer.get()));
         assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
         return null;
       }
-    }).when(ctx).write(any(ImmutableBytesWritable.class), any(MapReduceCell.class));
+    }).when(ctx).write(any(ImmutableBytesWritable.class), any(MapReduceExtendedCell.class));
 
     importer.setup(ctx);
     Result value = mock(Result.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
index 99bd5c1..03bd0b1 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
@@ -52,7 +52,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -185,12 +185,12 @@ public class TestCellBasedWALPlayer2 {
       @Override
       public Void answer(InvocationOnMock invocation) throws Throwable {
         ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArguments()[0];
-        MapReduceCell key = (MapReduceCell) invocation.getArguments()[1];
+        MapReduceExtendedCell key = (MapReduceExtendedCell) invocation.getArguments()[1];
         assertEquals("row", Bytes.toString(writer.get()));
         assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
         return null;
       }
-    }).when(context).write(any(ImmutableBytesWritable.class), any(MapReduceCell.class));
+    }).when(context).write(any(ImmutableBytesWritable.class), any(MapReduceExtendedCell.class));
 
     mapper.map(key, value, context);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 50d5ddc..cfc3dd9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-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;
@@ -383,14 +383,15 @@ public class HFileWriterImpl implements HFile.Writer {
           + CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
     }
     byte[] midRow;
-    boolean bufferBacked = left instanceof ByteBufferCell && right instanceof ByteBufferCell;
+    boolean bufferBacked = left instanceof ByteBufferExtendedCell
+        && right instanceof ByteBufferExtendedCell;
     if (diff < 0) {
       // Left row is < right row.
       if (bufferBacked) {
-        midRow = getMinimumMidpointArray(((ByteBufferCell) left).getRowByteBuffer(),
-            ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
-            ((ByteBufferCell) right).getRowByteBuffer(),
-            ((ByteBufferCell) right).getRowPosition(), right.getRowLength());
+        midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+            ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
+            ((ByteBufferExtendedCell) right).getRowByteBuffer(),
+            ((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength());
       } else {
         midRow = getMinimumMidpointArray(left.getRowArray(), left.getRowOffset(),
             left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
@@ -407,10 +408,10 @@ public class HFileWriterImpl implements HFile.Writer {
     }
     if (diff < 0) {
       if (bufferBacked) {
-        midRow = getMinimumMidpointArray(((ByteBufferCell) left).getFamilyByteBuffer(),
-            ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
-            ((ByteBufferCell) right).getFamilyByteBuffer(),
-            ((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
+        midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+            ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
+            ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+            ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
       } else {
         midRow = getMinimumMidpointArray(left.getFamilyArray(), left.getFamilyOffset(),
             left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
@@ -429,10 +430,10 @@ public class HFileWriterImpl implements HFile.Writer {
     }
     if (diff < 0) {
       if (bufferBacked) {
-        midRow = getMinimumMidpointArray(((ByteBufferCell) left).getQualifierByteBuffer(),
-            ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
-            ((ByteBufferCell) right).getQualifierByteBuffer(),
-            ((ByteBufferCell) right).getQualifierPosition(), right.getQualifierLength());
+        midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+            ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
+            ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+            ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
       } else {
         midRow = getMinimumMidpointArray(left.getQualifierArray(), left.getQualifierOffset(),
             left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
deleted file mode 100644
index 4597b55..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +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.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset
- * @see MemStoreLAB
- */
-//TODO : When moving this cell to CellChunkMap we will have the following things
-// to be serialized
-// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
-@InterfaceAudience.Private
-public class ByteBufferChunkCell extends ByteBufferKeyValue {
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java
new file mode 100644
index 0000000..8278c42
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java
@@ -0,0 +1,48 @@
+/**
+ * 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.regionserver;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset
+ * @see MemStoreLAB
+ */
+//TODO : When moving this cell to CellChunkMap we will have the following things
+// to be serialized
+// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
+@InterfaceAudience.Private
+public class ByteBufferChunkKeyValue extends ByteBufferKeyValue {
+  public ByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public ByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
index 388b675..782d27e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
@@ -123,6 +123,6 @@ public class CellChunkMap extends CellFlatMap {
           + chunk.isFromPool() + ". We were looking for a cell at index " + i);
     }
 
-    return new ByteBufferChunkCell(buf, offsetOfCell, lengthOfCell, cellSeqID);
+    return new ByteBufferChunkKeyValue(buf, offsetOfCell, lengthOfCell, cellSeqID);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index ca079ea..5d43069 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -155,9 +155,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
       // which directly return tagsLen as 0. So we avoid parsing many length components in
       // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
       // call getTagsLength().
-      return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+      return new NoTagByteBufferChunkKeyValue(buf, offset, len, cell.getSequenceId());
     } else {
-      return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+      return new ByteBufferChunkKeyValue(buf, offset, len, cell.getSequenceId());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
deleted file mode 100644
index dba8681..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +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.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
- * @see MemStoreLAB
- */
-@InterfaceAudience.Private
-public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java
new file mode 100644
index 0000000..1fb533a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java
@@ -0,0 +1,48 @@
+/**
+ * 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.regionserver;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
+ * @see MemStoreLAB
+ */
+@InterfaceAudience.Private
+public class NoTagByteBufferChunkKeyValue extends NoTagsByteBufferKeyValue {
+
+  public NoTagByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public NoTagByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 584d0a2..4091316 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -46,7 +46,7 @@ import java.util.concurrent.atomic.LongAdder;
 import org.apache.commons.lang3.mutable.MutableObject;
 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;
@@ -1295,8 +1295,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         // Since byte buffers can point all kinds of crazy places it's harder to keep track
         // of which blocks are kept alive by what byte buffer.
         // So we make a guess.
-        if (c instanceof ByteBufferCell) {
-          ByteBufferCell bbCell = (ByteBufferCell) c;
+        if (c instanceof ByteBufferExtendedCell) {
+          ByteBufferExtendedCell bbCell = (ByteBufferExtendedCell) c;
           ByteBuffer bb = bbCell.getValueByteBuffer();
           if (bb != lastBlock) {
             context.incrementResponseBlockSize(bb.capacity());

http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
index 7e14228..f43f147 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
-import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OffheapDecodedCell;
 import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell;
 import org.apache.hadoop.hbase.codec.Codec.Decoder;
 import org.apache.hadoop.hbase.codec.Codec.Encoder;
@@ -108,7 +107,8 @@ public class TestBufferedDataBlockEncoder {
         kv2.getTagsLength());
     KeyValue kv3 = new KeyValue(Bytes.toBytes("r3"), Bytes.toBytes("cf"), Bytes.toBytes("qual"),
         HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"));
-    OffheapDecodedCell c3 = new OffheapDecodedCell(ByteBuffer.wrap(kv2.getKey()),
+    BufferedDataBlockEncoder.OffheapDecodedExtendedCell
+        c3 = new BufferedDataBlockEncoder.OffheapDecodedExtendedCell(ByteBuffer.wrap(kv2.getKey()),
         kv2.getRowLength(), kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(),
         kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(),
         kv2.getTimestamp(), kv2.getTypeByte(), ByteBuffer.wrap(kv2.getValueArray()),


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

Posted by ch...@apache.org.
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 {


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

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2468f300/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..f024a14 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);
     }
@@ -1021,30 +1034,34 @@ public final class PrivateCellUtil {
    * @return length of common prefix
    */
   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) {
@@ -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);
+      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/2468f300/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/2468f300/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/2468f300/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/2468f300/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/2468f300/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/2468f300/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/2468f300/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/2468f300/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 c77a9d1..7c43e1f 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;
@@ -271,7 +271,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()));
@@ -482,7 +482,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) {
@@ -833,7 +833,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);
@@ -856,7 +856,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/2468f300/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 f4dfba5..f83bf9d 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
@@ -42,7 +42,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;
@@ -156,7 +156,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) {
@@ -347,7 +347,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/2468f300/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/2468f300/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);
+    }
+  }
+}