You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2016/11/24 15:18:05 UTC

[3/4] hbase git commit: HBASE-15786 Create DBB backed MSLAB pool.

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 484eebd..f1aa091 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
@@ -131,10 +131,10 @@ public final class CellUtil {
 
   public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
     short rowLen = cell.getRowLength();
-    if (cell instanceof ByteBufferedCell) {
+    if (cell instanceof ByteBufferCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-          ((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition(), destinationOffset, rowLen);
+          ((ByteBufferCell) cell).getRowByteBuffer(),
+          ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
     } else {
       System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
           rowLen);
@@ -142,16 +142,28 @@ public final class CellUtil {
     return destinationOffset + rowLen;
   }
 
+  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);
+    } else {
+      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
+          cell.getRowOffset(), rowLen);
+    }
+    return destinationOffset + rowLen;
+  }
+
   /**
    * Copies the row to a new byte[]
    * @param cell the cell from which row has to copied
    * @return the byte[] containing the row
    */
   public static byte[] copyRow(Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return ByteBufferUtils.copyOfRange(((ByteBufferedCell) cell).getRowByteBuffer(),
-        ((ByteBufferedCell) cell).getRowPosition(),
-        ((ByteBufferedCell) cell).getRowPosition() + cell.getRowLength());
+    if (cell instanceof ByteBufferCell) {
+      return ByteBufferUtils.copyOfRange(((ByteBufferCell) cell).getRowByteBuffer(),
+        ((ByteBufferCell) cell).getRowPosition(),
+        ((ByteBufferCell) cell).getRowPosition() + cell.getRowLength());
     } else {
       return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
         cell.getRowOffset() + cell.getRowLength());
@@ -160,10 +172,10 @@ public final class CellUtil {
 
   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
     byte fLen = cell.getFamilyLength();
-    if (cell instanceof ByteBufferedCell) {
+    if (cell instanceof ByteBufferCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-          ((ByteBufferedCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferedCell) cell).getFamilyPosition(), destinationOffset, fLen);
+          ((ByteBufferCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferCell) cell).getFamilyPosition(), destinationOffset, fLen);
     } else {
       System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
           destinationOffset, fLen);
@@ -171,12 +183,24 @@ public final class CellUtil {
     return destinationOffset + fLen;
   }
 
+  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);
+    } else {
+      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
+          cell.getFamilyOffset(), fLen);
+    }
+    return destinationOffset + fLen;
+  }
+
   public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
     int qlen = cell.getQualifierLength();
-    if (cell instanceof ByteBufferedCell) {
+    if (cell instanceof ByteBufferCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-          ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferedCell) cell).getQualifierPosition(), destinationOffset, qlen);
+          ((ByteBufferCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
     } else {
       System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
           destinationOffset, qlen);
@@ -184,12 +208,24 @@ public final class CellUtil {
     return destinationOffset + qlen;
   }
 
+  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);
+    } else {
+      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
+          cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
+    }
+    return destinationOffset + qlen;
+  }
+
   public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
     int vlen = cell.getValueLength();
-    if (cell instanceof ByteBufferedCell) {
+    if (cell instanceof ByteBufferCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-          ((ByteBufferedCell) cell).getValueByteBuffer(),
-          ((ByteBufferedCell) cell).getValuePosition(), destinationOffset, vlen);
+          ((ByteBufferCell) cell).getValueByteBuffer(),
+          ((ByteBufferCell) cell).getValuePosition(), destinationOffset, vlen);
     } else {
       System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
           vlen);
@@ -197,6 +233,18 @@ public final class CellUtil {
     return destinationOffset + vlen;
   }
 
+  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);
+    } else {
+      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
+          cell.getValueOffset(), vlen);
+    }
+    return destinationOffset + vlen;
+  }
+
   /**
    * Copies the tags info into the tag portion of the cell
    * @param cell
@@ -206,10 +254,10 @@ public final class CellUtil {
    */
   public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
     int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferedCell) {
+    if (cell instanceof ByteBufferCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-          ((ByteBufferedCell) cell).getTagsByteBuffer(),
-          ((ByteBufferedCell) cell).getTagsPosition(), destinationOffset, tlen);
+          ((ByteBufferCell) cell).getTagsByteBuffer(),
+          ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
     } else {
       System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
           tlen);
@@ -217,22 +265,34 @@ public final class CellUtil {
     return destinationOffset + tlen;
   }
 
+  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);
+    } else {
+      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
+          cell.getTagsOffset(), tlen);
+    }
+    return destinationOffset + tlen;
+  }
+
   /********************* misc *************************************/
 
   @Private
   public static byte getRowByte(Cell cell, int index) {
-    if (cell instanceof ByteBufferedCell) {
-      return ((ByteBufferedCell) cell).getRowByteBuffer().get(
-          ((ByteBufferedCell) cell).getRowPosition() + index);
+    if (cell instanceof ByteBufferCell) {
+      return ((ByteBufferCell) cell).getRowByteBuffer().get(
+          ((ByteBufferCell) cell).getRowPosition() + index);
     }
     return cell.getRowArray()[cell.getRowOffset() + index];
   }
 
   @Private
   public static byte getQualifierByte(Cell cell, int index) {
-    if (cell instanceof ByteBufferedCell) {
-      return ((ByteBufferedCell) cell).getQualifierByteBuffer().get(
-          ((ByteBufferedCell) cell).getQualifierPosition() + index);
+    if (cell instanceof ByteBufferCell) {
+      return ((ByteBufferCell) cell).getQualifierByteBuffer().get(
+          ((ByteBufferCell) cell).getQualifierPosition() + index);
     }
     return cell.getQualifierArray()[cell.getQualifierOffset() + index];
   }
@@ -546,12 +606,12 @@ public final class CellUtil {
     }
 
     @Override
-    public void write(byte[] buf, int offset) {
-      offset = KeyValueUtil.appendToByteArray(this.cell, buf, offset, false);
+    public void write(ByteBuffer buf, int offset) {
+      offset = KeyValueUtil.appendToByteBuffer(this.cell, buf, offset, false);
       int tagsLen = this.tags.length;
       assert tagsLen > 0;
-      offset = Bytes.putAsShort(buf, offset, tagsLen);
-      System.arraycopy(this.tags, 0, buf, offset, tagsLen);
+      offset = ByteBufferUtils.putAsShort(buf, offset, tagsLen);
+      ByteBufferUtils.copyFromArrayToBuffer(buf, offset, this.tags, 0, tagsLen);
     }
 
     @Override
@@ -720,9 +780,9 @@ public final class CellUtil {
 
   public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
       final int length) {
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getRowByteBuffer(),
-          ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(), buf, offset,
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
+          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(), buf, offset,
           length);
     }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
@@ -732,20 +792,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 ByteBufferedCell && right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getFamilyByteBuffer(),
-          ((ByteBufferedCell) left).getFamilyPosition(), lfamlength,
-          ((ByteBufferedCell) right).getFamilyByteBuffer(),
-          ((ByteBufferedCell) right).getFamilyPosition(), rfamlength);
-    }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getFamilyByteBuffer(),
-          ((ByteBufferedCell) left).getFamilyPosition(), lfamlength,
+    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 ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
+          ((ByteBufferCell) left).getFamilyPosition(), lfamlength,
           right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
     }
-    if (right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) right).getFamilyByteBuffer(),
-          ((ByteBufferedCell) right).getFamilyPosition(), rfamlength,
+    if (right instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) right).getFamilyByteBuffer(),
+          ((ByteBufferCell) right).getFamilyPosition(), rfamlength,
           left.getFamilyArray(), left.getFamilyOffset(), lfamlength);
     }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
@@ -761,9 +821,9 @@ public final class CellUtil {
 
   public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
       final int length) {
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getFamilyByteBuffer(),
-          ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(), buf,
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
+          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), buf,
           offset, length);
     }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
@@ -773,20 +833,20 @@ public final class CellUtil {
   public static boolean matchingQualifier(final Cell left, final Cell right) {
     int lqlength = left.getQualifierLength();
     int rqlength = right.getQualifierLength();
-    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getQualifierByteBuffer(),
-          ((ByteBufferedCell) left).getQualifierPosition(), lqlength,
-          ((ByteBufferedCell) right).getQualifierByteBuffer(),
-          ((ByteBufferedCell) right).getQualifierPosition(), rqlength);
-    }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getQualifierByteBuffer(),
-          ((ByteBufferedCell) left).getQualifierPosition(), lqlength,
+    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 ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
+          ((ByteBufferCell) left).getQualifierPosition(), lqlength,
           right.getQualifierArray(), right.getQualifierOffset(), rqlength);
     }
-    if (right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) right).getQualifierByteBuffer(),
-          ((ByteBufferedCell) right).getQualifierPosition(), rqlength,
+    if (right instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) right).getQualifierByteBuffer(),
+          ((ByteBufferCell) right).getQualifierPosition(), rqlength,
           left.getQualifierArray(), left.getQualifierOffset(), lqlength);
     }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
@@ -822,9 +882,9 @@ public final class CellUtil {
     if (buf == null) {
       return left.getQualifierLength() == 0;
     }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getQualifierByteBuffer(),
-          ((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
+          ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
           buf, offset, length);
     }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
@@ -856,20 +916,20 @@ public final class CellUtil {
 
   public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
       int rvlength) {
-    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getValueByteBuffer(),
-        ((ByteBufferedCell) left).getValuePosition(), lvlength,
-        ((ByteBufferedCell) right).getValueByteBuffer(),
-        ((ByteBufferedCell) right).getValuePosition(), rvlength);
-    }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getValueByteBuffer(),
-        ((ByteBufferedCell) left).getValuePosition(), lvlength, right.getValueArray(),
+    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 ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
+        ((ByteBufferCell) left).getValuePosition(), lvlength, right.getValueArray(),
         right.getValueOffset(), rvlength);
     }
-    if (right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) right).getValueByteBuffer(),
-        ((ByteBufferedCell) right).getValuePosition(), rvlength, left.getValueArray(),
+    if (right instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) right).getValueByteBuffer(),
+        ((ByteBufferCell) right).getValuePosition(), rvlength, left.getValueArray(),
         left.getValueOffset(), lvlength);
     }
     return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
@@ -877,9 +937,9 @@ public final class CellUtil {
   }
 
   public static boolean matchingValue(final Cell left, final byte[] buf) {
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
-          ((ByteBufferedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getValueByteBuffer(),
+          ((ByteBufferCell) left).getValuePosition(), left.getValueLength(), buf, 0,
           buf.length) == 0;
     }
     return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
@@ -1105,9 +1165,9 @@ public final class CellUtil {
     if (tagsLength == 0) {
       return TagUtil.EMPTY_TAGS_ITR;
     }
-    if (cell instanceof ByteBufferedCell) {
-      return tagsIterator(((ByteBufferedCell) cell).getTagsByteBuffer(),
-          ((ByteBufferedCell) cell).getTagsPosition(), tagsLength);
+    if (cell instanceof ByteBufferCell) {
+      return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
+          ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
     }
     return tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
   }
@@ -1133,14 +1193,14 @@ public final class CellUtil {
    * @return null if there is no tag of the passed in tag type
    */
   public static Tag getTag(Cell cell, byte type){
-    boolean bufferBacked = cell instanceof ByteBufferedCell;
+    boolean bufferBacked = cell instanceof ByteBufferCell;
     int length = cell.getTagsLength();
-    int offset = bufferBacked? ((ByteBufferedCell)cell).getTagsPosition():cell.getTagsOffset();
+    int offset = bufferBacked? ((ByteBufferCell)cell).getTagsPosition():cell.getTagsOffset();
     int pos = offset;
     while (pos < offset + length) {
       int tagLen;
       if (bufferBacked) {
-        ByteBuffer tagsBuffer = ((ByteBufferedCell)cell).getTagsByteBuffer();
+        ByteBuffer tagsBuffer = ((ByteBufferCell)cell).getTagsByteBuffer();
         tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
         if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
           return new OffheapTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE);
@@ -1264,15 +1324,15 @@ 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 ByteBufferedCell) {
+    if (cell instanceof ByteBufferCell) {
       out.writeShort(rowLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(),
-        ((ByteBufferedCell) cell).getRowPosition(), rowLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
+        ((ByteBufferCell) cell).getRowPosition(), rowLen);
       out.writeByte(fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferedCell) cell).getFamilyPosition(), fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferedCell) cell).getQualifierPosition(), qLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferCell) cell).getFamilyPosition(), fLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferCell) cell).getQualifierPosition(), qLen);
     } else {
       out.writeShort(rowLen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
@@ -1292,9 +1352,9 @@ public final class CellUtil {
    * @throws IOException
    */
   public static void writeRow(DataOutputStream out, Cell cell, short rlength) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(),
-        ((ByteBufferedCell) cell).getRowPosition(), rlength);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
+        ((ByteBufferCell) cell).getRowPosition(), rlength);
     } else {
       out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
     }
@@ -1309,9 +1369,9 @@ public final class CellUtil {
    */
   public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
       int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(),
-        ((ByteBufferedCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
+        ((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
     } else {
       out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
     }
@@ -1325,9 +1385,9 @@ public final class CellUtil {
    * @throws IOException
    */
   public static void writeFamily(DataOutputStream out, Cell cell, byte flength) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferedCell) cell).getFamilyPosition(), flength);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferCell) cell).getFamilyPosition(), flength);
     } else {
       out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
     }
@@ -1342,9 +1402,9 @@ public final class CellUtil {
    */
   public static void writeQualifier(DataOutputStream out, Cell cell, int qlength)
       throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferedCell) cell).getQualifierPosition(), qlength);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferCell) cell).getQualifierPosition(), qlength);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
     }
@@ -1359,9 +1419,9 @@ public final class CellUtil {
    */
   public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell,
       int qlength, int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferedCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
         qlength - commonPrefix);
@@ -1376,9 +1436,9 @@ public final class CellUtil {
    * @throws IOException
    */
   public static void writeValue(DataOutputStream out, Cell cell, int vlength) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getValueByteBuffer(),
-        ((ByteBufferedCell) cell).getValuePosition(), vlength);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getValueByteBuffer(),
+        ((ByteBufferCell) cell).getValuePosition(), vlength);
     } else {
       out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
     }
@@ -1392,9 +1452,9 @@ public final class CellUtil {
    * @throws IOException
    */
   public static void writeTags(DataOutputStream out, Cell cell, int tagsLength) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getTagsByteBuffer(),
-        ((ByteBufferedCell) cell).getTagsPosition(), tagsLength);
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
+        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
     } else {
       out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
     }
@@ -1494,10 +1554,10 @@ public final class CellUtil {
     }
     // Compare the RKs
     int rkCommonPrefix = 0;
-    if (c1 instanceof ByteBufferedCell && c2 instanceof ByteBufferedCell) {
-      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferedCell) c1).getRowByteBuffer(),
-        ((ByteBufferedCell) c1).getRowPosition(), rLen1, ((ByteBufferedCell) c2).getRowByteBuffer(),
-        ((ByteBufferedCell) c2).getRowPosition(), rLen2);
+    if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
+      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getRowByteBuffer(),
+        ((ByteBufferCell) c1).getRowPosition(), rLen1, ((ByteBufferCell) c2).getRowByteBuffer(),
+        ((ByteBufferCell) 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
@@ -1526,12 +1586,12 @@ public final class CellUtil {
       commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
       // Compare the CF names
       int fCommonPrefix;
-      if (c1 instanceof ByteBufferedCell && c2 instanceof ByteBufferedCell) {
+      if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
         fCommonPrefix =
-            ByteBufferUtils.findCommonPrefix(((ByteBufferedCell) c1).getFamilyByteBuffer(),
-              ((ByteBufferedCell) c1).getFamilyPosition(), fLen1,
-              ((ByteBufferedCell) c2).getFamilyByteBuffer(),
-              ((ByteBufferedCell) c2).getFamilyPosition(), fLen2);
+            ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getFamilyByteBuffer(),
+              ((ByteBufferCell) c1).getFamilyPosition(), fLen1,
+              ((ByteBufferCell) c2).getFamilyByteBuffer(),
+              ((ByteBufferCell) c2).getFamilyPosition(), fLen2);
       } else {
         fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
           fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
@@ -1545,11 +1605,11 @@ public final class CellUtil {
     int qLen1 = c1.getQualifierLength();
     int qLen2 = c2.getQualifierLength();
     int qCommon;
-    if (c1 instanceof ByteBufferedCell && c2 instanceof ByteBufferedCell) {
-      qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferedCell) c1).getQualifierByteBuffer(),
-        ((ByteBufferedCell) c1).getQualifierPosition(), qLen1,
-        ((ByteBufferedCell) c2).getQualifierByteBuffer(),
-        ((ByteBufferedCell) c2).getQualifierPosition(), qLen2);
+    if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
+      qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getQualifierByteBuffer(),
+        ((ByteBufferCell) c1).getQualifierPosition(), qLen1,
+        ((ByteBufferCell) c2).getQualifierByteBuffer(),
+        ((ByteBufferCell) c2).getQualifierPosition(), qLen2);
     } else {
       qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
         qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
@@ -1658,20 +1718,20 @@ public final class CellUtil {
     short lrowlength = left.getRowLength();
     short rrowlength = right.getRowLength();
     if (lrowlength != rrowlength) return false;
-    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getRowByteBuffer(),
-          ((ByteBufferedCell) left).getRowPosition(), lrowlength,
-          ((ByteBufferedCell) right).getRowByteBuffer(),
-          ((ByteBufferedCell) right).getRowPosition(), rrowlength);
-    }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) left).getRowByteBuffer(),
-          ((ByteBufferedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
+    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 ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
+          ((ByteBufferCell) left).getRowPosition(), lrowlength, right.getRowArray(),
           right.getRowOffset(), rrowlength);
     }
-    if (right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.equals(((ByteBufferedCell) right).getRowByteBuffer(),
-          ((ByteBufferedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
+    if (right instanceof ByteBufferCell) {
+      return ByteBufferUtils.equals(((ByteBufferCell) right).getRowByteBuffer(),
+          ((ByteBufferCell) right).getRowPosition(), rrowlength, left.getRowArray(),
           left.getRowOffset(), lrowlength);
     }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength,
@@ -1704,9 +1764,9 @@ public final class CellUtil {
    * @return rowkey as int
    */
   public static int getRowAsInt(Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return ByteBufferUtils.toInt(((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition());
+    if (cell instanceof ByteBufferCell) {
+      return ByteBufferUtils.toInt(((ByteBufferCell) cell).getRowByteBuffer(),
+          ((ByteBufferCell) cell).getRowPosition());
     }
     return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
   }
@@ -1718,9 +1778,9 @@ public final class CellUtil {
    * @return value as long
    */
   public static long getValueAsLong(Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return ByteBufferUtils.toLong(((ByteBufferedCell) cell).getValueByteBuffer(),
-          ((ByteBufferedCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferCell) {
+      return ByteBufferUtils.toLong(((ByteBufferCell) cell).getValueByteBuffer(),
+          ((ByteBufferCell) cell).getValuePosition());
     }
     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
   }
@@ -1732,9 +1792,9 @@ public final class CellUtil {
    * @return value as double
    */
   public static double getValueAsDouble(Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return ByteBufferUtils.toDouble(((ByteBufferedCell) cell).getValueByteBuffer(),
-          ((ByteBufferedCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferCell) {
+      return ByteBufferUtils.toDouble(((ByteBufferCell) cell).getValueByteBuffer(),
+          ((ByteBufferCell) cell).getValuePosition());
     }
     return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
   }
@@ -1746,9 +1806,9 @@ public final class CellUtil {
    * @return value as BigDecimal
    */
   public static BigDecimal getValueAsBigDecimal(Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return ByteBufferUtils.toBigDecimal(((ByteBufferedCell) cell).getValueByteBuffer(),
-          ((ByteBufferedCell) cell).getValuePosition(), cell.getValueLength());
+    if (cell instanceof ByteBufferCell) {
+      return ByteBufferUtils.toBigDecimal(((ByteBufferCell) cell).getValueByteBuffer(),
+          ((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
     }
     return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }
@@ -1760,9 +1820,9 @@ public final class CellUtil {
    * @return First possible Cell on passed Cell's row.
    */
   public static Cell createFirstOnRow(final Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return new FirstOnRowByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
-        ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
+    if (cell instanceof ByteBufferCell) {
+      return new FirstOnRowByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
+        ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
     }
     return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
@@ -1796,12 +1856,12 @@ public final class CellUtil {
    * @return First possible Cell on passed Cell's row.
    */
   public static Cell createFirstOnRowCol(final Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return new FirstOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
+    if (cell instanceof ByteBufferCell) {
+      return new FirstOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
+          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
           HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
-          ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
+          ((ByteBufferCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
     }
     return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
         cell.getRowLength(), HConstants.EMPTY_BYTE_ARRAY, 0, (byte)0, cell.getQualifierArray(),
@@ -1829,11 +1889,11 @@ public final class CellUtil {
    * @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 ByteBufferedCell) {
-      return new FirstOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferedCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
+    if(cell instanceof ByteBufferCell) {
+      return new FirstOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
+          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
+          ((ByteBufferCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
           ByteBuffer.wrap(qArray), qoffest, qlength);
     }
     return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
@@ -1850,13 +1910,13 @@ public final class CellUtil {
    * @param ts
    */
   public static Cell createFirstOnRowColTS(Cell cell, long ts) {
-    if(cell instanceof ByteBufferedCell) {
-      return new FirstOnRowColTSByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferedCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength(),
+    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);
     }
     return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(),
@@ -1871,9 +1931,9 @@ public final class CellUtil {
    * @return Last possible Cell on passed Cell's row.
    */
   public static Cell createLastOnRow(final Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return new LastOnRowByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
-        ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
+    if (cell instanceof ByteBufferCell) {
+      return new LastOnRowByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
+        ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
     }
     return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
@@ -1891,13 +1951,13 @@ public final class CellUtil {
    * @return Last possible Cell on passed Cell's rk:cf:q.
    */
   public static Cell createLastOnRowCol(final Cell cell) {
-    if (cell instanceof ByteBufferedCell) {
-      return new LastOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferedCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferedCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
+    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());
     }
     return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
@@ -1926,9 +1986,9 @@ public final class CellUtil {
    */
   public static void compressTags(DataOutputStream out, Cell cell,
       TagCompressionContext tagCompressionContext) throws IOException {
-    if (cell instanceof ByteBufferedCell) {
-      tagCompressionContext.compressTags(out, ((ByteBufferedCell) cell).getTagsByteBuffer(),
-          ((ByteBufferedCell) cell).getTagsPosition(), cell.getTagsLength());
+    if (cell instanceof ByteBufferCell) {
+      tagCompressionContext.compressTags(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
+          ((ByteBufferCell) cell).getTagsPosition(), cell.getTagsLength());
     } else {
       tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(),
           cell.getTagsLength());
@@ -2032,7 +2092,7 @@ public final class CellUtil {
    * 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 EmptyByteBufferedCell extends ByteBufferedCell
+  private static abstract class EmptyByteBufferCell extends ByteBufferCell
       implements SettableSequenceId {
 
     @Override
@@ -2210,12 +2270,12 @@ public final class CellUtil {
   }
 
   @InterfaceAudience.Private
-  private static class FirstOnRowByteBufferedCell extends EmptyByteBufferedCell {
+  private static class FirstOnRowByteBufferCell extends EmptyByteBufferCell {
     private final ByteBuffer rowBuff;
     private final int roffset;
     private final short rlength;
 
-    public FirstOnRowByteBufferedCell(final ByteBuffer row, int roffset, short rlength) {
+    public FirstOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
       this.rowBuff = row;
       this.roffset = roffset;
       this.rlength = rlength;
@@ -2248,12 +2308,12 @@ public final class CellUtil {
   }
 
   @InterfaceAudience.Private
-  private static class LastOnRowByteBufferedCell extends EmptyByteBufferedCell {
+  private static class LastOnRowByteBufferCell extends EmptyByteBufferCell {
     private final ByteBuffer rowBuff;
     private final int roffset;
     private final short rlength;
 
-    public LastOnRowByteBufferedCell(final ByteBuffer row, int roffset, short rlength) {
+    public LastOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
       this.rowBuff = row;
       this.roffset = roffset;
       this.rlength = rlength;
@@ -2286,7 +2346,7 @@ public final class CellUtil {
   }
 
   @InterfaceAudience.Private
-  private static class FirstOnRowColByteBufferedCell extends FirstOnRowByteBufferedCell {
+  private static class FirstOnRowColByteBufferCell extends FirstOnRowByteBufferCell {
     private final ByteBuffer famBuff;
     private final int famOffset;
     private final byte famLength;
@@ -2294,7 +2354,7 @@ public final class CellUtil {
     private final int colOffset;
     private final int colLength;
 
-    public FirstOnRowColByteBufferedCell(final ByteBuffer row, int roffset, short rlength,
+    public FirstOnRowColByteBufferCell(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);
@@ -2406,11 +2466,11 @@ public final class CellUtil {
   }
 
   @InterfaceAudience.Private
-  private static class FirstOnRowColTSByteBufferedCell extends FirstOnRowColByteBufferedCell {
+  private static class FirstOnRowColTSByteBufferCell extends FirstOnRowColByteBufferCell {
 
     private long ts;
 
-    public FirstOnRowColTSByteBufferedCell(ByteBuffer rBuffer, int roffset, short rlength,
+    public FirstOnRowColTSByteBufferCell(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);
@@ -2513,7 +2573,7 @@ public final class CellUtil {
   }
 
   @InterfaceAudience.Private
-  private static class LastOnRowColByteBufferedCell extends LastOnRowByteBufferedCell {
+  private static class LastOnRowColByteBufferCell extends LastOnRowByteBufferCell {
     private final ByteBuffer fBuffer;
     private final int foffset;
     private final byte flength;
@@ -2521,7 +2581,7 @@ public final class CellUtil {
     private final int qoffset;
     private final int qlength;
 
-    public LastOnRowColByteBufferedCell(ByteBuffer rBuffer, int roffset, short rlength,
+    public LastOnRowColByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
         ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset,
         int qlength) {
       super(rBuffer, roffset, rlength);
@@ -2604,4 +2664,34 @@ public final class CellUtil {
       return Type.DeleteFamily.getCode();
     }
   }
+
+  /**
+   * Clone the passed cell by copying its data into the passed buf.
+   */
+  public static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) {
+    int tagsLen = cell.getTagsLength();
+    if (cell instanceof ExtendedCell) {
+      ((ExtendedCell) cell).write(buf, offset);
+    } else {
+      // Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
+      // other case also. The data fragments within Cell is copied into buf as in KeyValue
+      // serialization format only.
+      KeyValueUtil.appendToByteBuffer(cell, buf, offset, true);
+    }
+    if (buf.hasArray()) {
+      KeyValue newKv;
+      if (tagsLen == 0) {
+        // When tagsLen is 0, make a NoTagsKeyValue version of Cell. This is an optimized class
+        // 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().
+        newKv = new NoTagsKeyValue(buf.array(), buf.arrayOffset() + offset, len);
+      } else {
+        newKv = new KeyValue(buf.array(), buf.arrayOffset() + offset, len);
+      }
+      newKv.setSequenceId(cell.getSequenceId());
+      return newKv;
+    }
+    return new OffheapKeyValue(buf, offset, len, tagsLen > 0, cell.getSequenceId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index f60da14..0c7fbb0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -65,7 +66,7 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
    * @param buf The buffer where to write the Cell.
    * @param offset The offset within buffer, to write the Cell.
    */
-  void write(byte[] buf, int offset);
+  void write(ByteBuffer buf, int offset);
 
   /**
    * @return The heap size overhead associated with this Cell.

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
index 7b94c3d..4baaabe 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 
@@ -81,7 +80,6 @@ public class HBaseConfiguration extends Configuration {
     conf.addResource("hbase-site.xml");
 
     checkDefaultsVersion(conf);
-    HeapMemorySizeUtil.checkForClusterFreeMemoryLimit(conf);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index a95f814..6a07513 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -2491,8 +2491,8 @@ public class KeyValue implements ExtendedCell {
   }
 
   @Override
-  public void write(byte[] buf, int offset) {
-    System.arraycopy(this.bytes, this.offset, buf, offset, this.length);
+  public void write(ByteBuffer buf, int offset) {
+    ByteBufferUtils.copyFromArrayToBuffer(buf, offset, this.bytes, this.offset, this.length);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index 077f9ee..d4c047c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -153,7 +153,6 @@ public class KeyValueUtil {
     return nextOffset;
   }
 
-
   /**************** copy key and value *********************/
 
   public static int appendToByteArray(Cell cell, byte[] output, int offset, boolean withTags) {
@@ -170,15 +169,25 @@ public class KeyValueUtil {
   }
 
   /**
-   * The position will be set to the beginning of the new ByteBuffer
-   * @param cell
-   * @return the ByteBuffer containing the cell
+   * Copy the Cell content into the passed buf in KeyValue serialization format.
    */
-  public static ByteBuffer copyToNewByteBuffer(final Cell cell) {
-    byte[] bytes = new byte[length(cell)];
-    appendToByteArray(cell, bytes, 0, true);
-    ByteBuffer buffer = ByteBuffer.wrap(bytes);
-    return buffer;
+  public static int appendToByteBuffer(Cell cell, ByteBuffer buf, int offset, boolean withTags) {
+    offset = ByteBufferUtils.putInt(buf, offset, keyLength(cell));// Key length
+    offset = ByteBufferUtils.putInt(buf, offset, cell.getValueLength());// Value length
+    offset = ByteBufferUtils.putShort(buf, offset, cell.getRowLength());// RK length
+    offset = CellUtil.copyRowTo(cell, buf, offset);// Row bytes
+    offset = ByteBufferUtils.putByte(buf, offset, cell.getFamilyLength());// CF length
+    offset = CellUtil.copyFamilyTo(cell, buf, offset);// CF bytes
+    offset = CellUtil.copyQualifierTo(cell, buf, offset);// Qualifier bytes
+    offset = ByteBufferUtils.putLong(buf, offset, cell.getTimestamp());// TS
+    offset = ByteBufferUtils.putByte(buf, offset, cell.getTypeByte());// Type
+    offset = CellUtil.copyValueTo(cell, buf, offset);// Value bytes
+    int tagsLength = cell.getTagsLength();
+    if (withTags && (tagsLength > 0)) {
+      offset = ByteBufferUtils.putAsShort(buf, offset, tagsLength);// Tags length
+      offset = CellUtil.copyTagTo(cell, buf, offset);// Tags bytes
+    }
+    return offset;
   }
 
   public static void appendToByteBuffer(final ByteBuffer bb, final KeyValue kv,
@@ -660,29 +669,4 @@ public class KeyValueUtil {
       return size;
     }
   }
-
-  /**
-   * Write the given cell in KeyValue serialization format into the given buf and return a new
-   * KeyValue object around that.
-   */
-  public static KeyValue copyCellTo(Cell cell, byte[] buf, int offset, int len) {
-    int tagsLen = cell.getTagsLength();
-    if (cell instanceof ExtendedCell) {
-      ((ExtendedCell) cell).write(buf, offset);
-    } else {
-      appendToByteArray(cell, buf, offset, true);
-    }
-    KeyValue newKv;
-    if (tagsLen == 0) {
-      // When tagsLen is 0, make a NoTagsKeyValue version of Cell. This is an optimized class 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().
-      newKv = new NoTagsKeyValue(buf, offset, len);
-    } else {
-      newKv = new KeyValue(buf, offset, len);
-    }
-    newKv.setSequenceId(cell.getSequenceId());
-    return newKv;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
index 06a0ed6..4277c3f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java
@@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 
 /**
- * This Cell is an implementation of {@link ByteBufferedCell} where the data resides in off heap
+ * This Cell is an implementation of {@link ByteBufferCell} where the data resides in off heap
  * memory.
  */
 @InterfaceAudience.Private
-public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
+public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell {
 
   protected final ByteBuffer buf;
   protected final int offset;
@@ -266,8 +266,8 @@ public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
   }
 
   @Override
-  public void write(byte[] buf, int offset) {
-    ByteBufferUtils.copyFromBufferToArray(buf, this.buf, this.offset, offset, this.length);
+  public void write(ByteBuffer buf, int offset) {
+    ByteBufferUtils.copyFromBufferToBuffer(this.buf, buf, this.offset, offset, this.length);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
index 8a3dadd..011e501 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.io.encoding;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
+import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
@@ -55,7 +55,7 @@ public abstract class AbstractDataBlockEncoder implements DataBlockEncoder {
       return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset()
           + key.position(), keyLength);
     } else {
-      return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength);
+      return new ByteBufferKeyOnlyKeyValue(key, key.position(), keyLength);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 216a82d..8f81f08 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
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -452,7 +452,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     }
 
     @Override
-    public void write(byte[] buf, int offset) {
+    public void write(ByteBuffer buf, int offset) {
       // This is not used in actual flow. Throwing UnsupportedOperationException
       throw new UnsupportedOperationException();
     }
@@ -475,7 +475,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     }
   }
 
-  protected static class OffheapDecodedCell extends ByteBufferedCell implements ExtendedCell {
+  protected static class OffheapDecodedCell extends ByteBufferCell implements ExtendedCell {
     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));
@@ -708,7 +708,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     }
 
     @Override
-    public void write(byte[] buf, int offset) {
+    public void write(ByteBuffer buf, int offset) {
       // This is not used in actual flow. Throwing UnsupportedOperationException
       throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 389ce01..9ad098c 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,8 +18,8 @@ package org.apache.hadoop.hbase.io.encoding;
 
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.ByteBufferedCell;
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
+import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -153,10 +153,10 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
   }
 
   private int compareRows(ByteBuffer row, Cell seekCell) {
-    if (seekCell instanceof ByteBufferedCell) {
+    if (seekCell instanceof ByteBufferCell) {
       return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
-          ((ByteBufferedCell) seekCell).getRowByteBuffer(),
-          ((ByteBufferedCell) seekCell).getRowPosition(),
+          ((ByteBufferCell) seekCell).getRowByteBuffer(),
+          ((ByteBufferCell) seekCell).getRowPosition(),
           seekCell.getRowLength());
     } else {
       return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
@@ -315,7 +315,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
     protected long memstoreTS;
     protected int nextKvOffset;
     // buffer backed keyonlyKV
-    private ByteBufferedKeyOnlyKeyValue currentKey = new ByteBufferedKeyOnlyKeyValue();
+    private ByteBufferKeyOnlyKeyValue currentKey = new ByteBufferKeyOnlyKeyValue();
 
     protected boolean isValid() {
       return valueOffset != -1;
@@ -323,7 +323,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
 
     protected void invalidate() {
       valueOffset = -1;
-      currentKey = new ByteBufferedKeyOnlyKeyValue();
+      currentKey = new ByteBufferKeyOnlyKeyValue();
       currentBuffer = null;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
deleted file mode 100644
index 5bec65d..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/HeapMemorySizeUtil.java
+++ /dev/null
@@ -1,152 +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.io.util;
-
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryUsage;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class HeapMemorySizeUtil {
-
-  public static final String MEMSTORE_SIZE_KEY = "hbase.regionserver.global.memstore.size";
-  public static final String MEMSTORE_SIZE_OLD_KEY =
-      "hbase.regionserver.global.memstore.upperLimit";
-  public static final String MEMSTORE_SIZE_LOWER_LIMIT_KEY =
-      "hbase.regionserver.global.memstore.size.lower.limit";
-  public static final String MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY =
-      "hbase.regionserver.global.memstore.lowerLimit";
-
-  public static final float DEFAULT_MEMSTORE_SIZE = 0.4f;
-  // Default lower water mark limit is 95% size of memstore size.
-  public static final float DEFAULT_MEMSTORE_SIZE_LOWER_LIMIT = 0.95f;
-
-  private static final Log LOG = LogFactory.getLog(HeapMemorySizeUtil.class);
-  // a constant to convert a fraction to a percentage
-  private static final int CONVERT_TO_PERCENTAGE = 100;
-
-  /**
-   * Checks whether we have enough heap memory left out after portion for Memstore and Block cache.
-   * We need atleast 20% of heap left out for other RS functions.
-   * @param conf
-   */
-  public static void checkForClusterFreeMemoryLimit(Configuration conf) {
-    if (conf.get(MEMSTORE_SIZE_OLD_KEY) != null) {
-      LOG.warn(MEMSTORE_SIZE_OLD_KEY + " is deprecated by " + MEMSTORE_SIZE_KEY);
-    }
-    float globalMemstoreSize = getGlobalMemStorePercent(conf, false);
-    int gml = (int)(globalMemstoreSize * CONVERT_TO_PERCENTAGE);
-    float blockCacheUpperLimit = getBlockCacheHeapPercent(conf);
-    int bcul = (int)(blockCacheUpperLimit * CONVERT_TO_PERCENTAGE);
-    if (CONVERT_TO_PERCENTAGE - (gml + bcul)
-            < (int)(CONVERT_TO_PERCENTAGE *
-                    HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD)) {
-      throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds "
-          + "the threshold required for successful cluster operation. "
-          + "The combined value cannot exceed 0.8. Please check "
-          + "the settings for hbase.regionserver.global.memstore.size and "
-          + "hfile.block.cache.size in your configuration. "
-          + "hbase.regionserver.global.memstore.size is " + globalMemstoreSize
-          + " hfile.block.cache.size is " + blockCacheUpperLimit);
-    }
-  }
-
-  /**
-   * Retrieve global memstore configured size as percentage of total heap.
-   * @param c
-   * @param logInvalid
-   */
-  public static float getGlobalMemStorePercent(final Configuration c, final boolean logInvalid) {
-    float limit = c.getFloat(MEMSTORE_SIZE_KEY,
-        c.getFloat(MEMSTORE_SIZE_OLD_KEY, DEFAULT_MEMSTORE_SIZE));
-    if (limit > 0.8f || limit <= 0.0f) {
-      if (logInvalid) {
-        LOG.warn("Setting global memstore limit to default of " + DEFAULT_MEMSTORE_SIZE
-            + " because supplied value outside allowed range of (0 -> 0.8]");
-      }
-      limit = DEFAULT_MEMSTORE_SIZE;
-    }
-    return limit;
-  }
-
-  /**
-   * Retrieve configured size for global memstore lower water mark as fraction of global memstore
-   * size.
-   */
-  public static float getGlobalMemStoreLowerMark(final Configuration conf, float globalMemStorePercent) {
-    String lowMarkPercentStr = conf.get(MEMSTORE_SIZE_LOWER_LIMIT_KEY);
-    if (lowMarkPercentStr != null) {
-      float lowMarkPercent = Float.parseFloat(lowMarkPercentStr);
-      if (lowMarkPercent > 1.0f) {
-        LOG.error("Bad configuration value for " + MEMSTORE_SIZE_LOWER_LIMIT_KEY + ": " +
-            lowMarkPercent + ". Using 1.0f instead.");
-        lowMarkPercent = 1.0f;
-      }
-      return lowMarkPercent;
-    }
-    String lowerWaterMarkOldValStr = conf.get(MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY);
-    if (lowerWaterMarkOldValStr != null) {
-      LOG.warn(MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY + " is deprecated. Instead use "
-          + MEMSTORE_SIZE_LOWER_LIMIT_KEY);
-      float lowerWaterMarkOldVal = Float.parseFloat(lowerWaterMarkOldValStr);
-      if (lowerWaterMarkOldVal > globalMemStorePercent) {
-        lowerWaterMarkOldVal = globalMemStorePercent;
-        LOG.error("Value of " + MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY + " (" + lowerWaterMarkOldVal
-            + ") is greater than global memstore limit (" + globalMemStorePercent + ") set by "
-            + MEMSTORE_SIZE_KEY + "/" + MEMSTORE_SIZE_OLD_KEY + ". Setting memstore lower limit "
-            + "to " + globalMemStorePercent);
-      }
-      return lowerWaterMarkOldVal / globalMemStorePercent;
-    }
-    return DEFAULT_MEMSTORE_SIZE_LOWER_LIMIT;
-  }
-
-  /**
-   * Retrieve configured size for on heap block cache as percentage of total heap.
-   * @param conf
-   */
-  public static float getBlockCacheHeapPercent(final Configuration conf) {
-    // L1 block cache is always on heap
-    float l1CachePercent = conf.getFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY,
-        HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
-    float l2CachePercent = getL2BlockCacheHeapPercent(conf);
-    return l1CachePercent + l2CachePercent;
-  }
-
-  /**
-   * @param conf
-   * @return The on heap size for L2 block cache.
-   */
-  public static float getL2BlockCacheHeapPercent(Configuration conf) {
-    float l2CachePercent = 0.0F;
-    String bucketCacheIOEngineName = conf.get(HConstants.BUCKET_CACHE_IOENGINE_KEY, null);
-    // L2 block cache can be on heap when IOEngine is "heap"
-    if (bucketCacheIOEngineName != null && bucketCacheIOEngineName.startsWith("heap")) {
-      float bucketCachePercentage = conf.getFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0F);
-      MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
-      l2CachePercent = bucketCachePercentage < 1 ? bucketCachePercentage
-          : (bucketCachePercentage * 1024 * 1024) / mu.getMax();
-    }
-    return l2CachePercent;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index c9a19ff..760afd4 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -876,6 +876,14 @@ public final class ByteBufferUtils {
     }
   }
 
+  public static int putInt(ByteBuffer buffer, int index, int val) {
+    if (UNSAFE_UNALIGNED) {
+      return UnsafeAccess.putInt(buffer, index, val);
+    }
+    buffer.putInt(index, val);
+    return index + Bytes.SIZEOF_INT;
+  }
+
   /**
    * Reads a double value at the given buffer's offset.
    * @param buffer
@@ -919,6 +927,21 @@ public final class ByteBufferUtils {
     }
   }
 
+  public static int putShort(ByteBuffer buffer, int index, short val) {
+    if (UNSAFE_UNALIGNED) {
+      return UnsafeAccess.putShort(buffer, index, val);
+    }
+    buffer.putShort(index, val);
+    return index + Bytes.SIZEOF_SHORT;
+  }
+
+  public static int putAsShort(ByteBuffer buf, int index, int val) {
+    buf.put(index + 1, (byte) val);
+    val >>= 8;
+    buf.put(index, (byte) val);
+    return index + Bytes.SIZEOF_SHORT;
+  }
+
   /**
    * Put a long value out to the given ByteBuffer's current position in big-endian format.
    * This also advances the position in buffer by long size.
@@ -933,6 +956,15 @@ public final class ByteBufferUtils {
       buffer.putLong(val);
     }
   }
+
+  public static int putLong(ByteBuffer buffer, int index, long val) {
+    if (UNSAFE_UNALIGNED) {
+      return UnsafeAccess.putLong(buffer, index, val);
+    }
+    buffer.putLong(index, val);
+    return index + Bytes.SIZEOF_LONG;
+  }
+
   /**
    * Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes
    * to buffer's current position. This also advances the position in the 'out' buffer by 'length'

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index 80fc33e..8740cfb 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
+import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -102,15 +102,15 @@ public class TestCellComparator {
     byte[] v = Bytes.toBytes("val1");
     KeyValue kv = new KeyValue(r1, f1, q1, v);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell1 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     kv = new KeyValue(r2, f1, q1, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertEquals(0, CellComparator.compareColumns(bbCell1, bbCell2));
     assertEquals(0, CellComparator.compareColumns(bbCell1, kv));
     kv = new KeyValue(r2, f1, q2, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell3 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell3 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertEquals(0, CellComparator.compareFamilies(bbCell2, bbCell3));
     assertTrue(CellComparator.compareQualifiers(bbCell2, bbCell3) < 0);
     assertTrue(CellComparator.compareColumns(bbCell2, bbCell3) < 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
index c1d0252..41a011d 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellUtil.java
@@ -416,7 +416,7 @@ public class TestCellUtil {
     byte[] tags = Bytes.toBytes("tag1");
     KeyValue kv = new KeyValue(r, f, q, 0, q.length, 1234L, Type.Put, v, 0, v.length, tags);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     byte[] rDest = CellUtil.cloneRow(bbCell);
     assertTrue(Bytes.equals(r, rDest));
     byte[] fDest = CellUtil.cloneFamily(bbCell);
@@ -440,10 +440,10 @@ public class TestCellUtil {
     byte[] tags = Bytes.toBytes("tag1");
     KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell1 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     kv = new KeyValue(r, f, q2, 0, q2.length, 1234L, Type.Put, v, 0, v.length, tags);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
     assertTrue(CellUtil.matchingRows(kv, bbCell2));
     assertTrue(CellUtil.matchingRow(bbCell1, r));
@@ -473,30 +473,30 @@ public class TestCellUtil {
     byte[] v = Bytes.toBytes(vl);
     KeyValue kv = new KeyValue(r, f, q, v);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertEquals(ri, CellUtil.getRowAsInt(bbCell));
     assertEquals(vl, CellUtil.getValueAsLong(bbCell));
     double vd = 3005.5;
     v = Bytes.toBytes(vd);
     kv = new KeyValue(r, f, q, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertEquals(vd, CellUtil.getValueAsDouble(bbCell), 0.0);
     BigDecimal bd = new BigDecimal(9999);
     v = Bytes.toBytes(bd);
     kv = new KeyValue(r, f, q, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
   }
 
   // TODO remove this test impl once we have a Cell implementation backed by ByteBuffer
-  public static class ByteBufferedCellImpl extends ByteBufferedCell {
+  public static class ByteBufferCellImpl extends ByteBufferCell {
 
     private final ByteBuffer buffer;
     private final int offset, length;
 
-    public ByteBufferedCellImpl(ByteBuffer buffer, int offset, int length) {
+    public ByteBufferCellImpl(ByteBuffer buffer, int offset, int length) {
       this.buffer = buffer;
       this.offset = offset;
       this.length = length;

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
index 9e76fc5..ec44408 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java
@@ -56,7 +56,7 @@ public class TestOffheapKeyValue {
     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);
-    ByteBufferedCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
+    ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -138,7 +138,7 @@ public class TestOffheapKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1, tags);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    ByteBufferedCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), true, 0l);
+    ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), true, 0l);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -178,7 +178,7 @@ public class TestOffheapKeyValue {
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getKeyLength());
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), kvCell.getKeyOffset(),
       kvCell.getKeyLength());
-    ByteBufferedCell offheapKeyOnlyKV = new ByteBufferedKeyOnlyKeyValue(buf, 0, buf.capacity());
+    ByteBufferCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKeyOnlyKV.getRowByteBuffer(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 67434a0..0a14443 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
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.OffheapKeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.ArrayBackedTag;
-import org.apache.hadoop.hbase.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
@@ -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);
-    ByteBufferedCell kv1 = (ByteBufferedCell)createOffheapKVWithTags(2);
+    ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(2);
     int tagsLength1 = kv1.getTagsLength();
     context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
-    ByteBufferedCell kv2 = (ByteBufferedCell)createOffheapKVWithTags(3);
+    ByteBufferCell kv2 = (ByteBufferCell)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);
-    ByteBufferedCell kv1 = (ByteBufferedCell)createOffheapKVWithTags(1);
+    ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(1);
     int tagsLength1 = kv1.getTagsLength();
     context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
-    ByteBufferedCell kv2 = (ByteBufferedCell)createOffheapKVWithTags(3);
+    ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
     int tagsLength2 = kv2.getTagsLength();
     context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
index 8dae4ea..da56e8c 100644
--- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
+++ b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.codec.prefixtree;
 
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -93,7 +93,7 @@ public class PrefixTreeSeeker implements EncodedSeeker {
     // The PrefixTreecell is of type BytebufferedCell and the value part of the cell
     // determines whether we are offheap cell or onheap cell.  All other parts of the cell-
     // row, fam and col are all represented as onheap byte[]
-    ByteBufferedCell cell = (ByteBufferedCell)ptSearcher.current();
+    ByteBufferCell cell = (ByteBufferCell)ptSearcher.current();
     if (cell == null) {
       return null;
     }
@@ -373,7 +373,7 @@ public class PrefixTreeSeeker implements EncodedSeeker {
     }
   }
 
-  private static class OffheapPrefixTreeCell extends ByteBufferedCell implements Cell,
+  private static class OffheapPrefixTreeCell extends ByteBufferCell implements Cell,
       SettableSequenceId, HeapSize {
     private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
         + (5 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_INT)

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
index f1da601..3ca4236 100644
--- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
+++ b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.decode;
 
 
 import java.nio.ByteBuffer;
-import org.apache.hadoop.hbase.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.util.ObjectIntPair;
  * without allocating new memory for every Cell iterated through.
  */
 @InterfaceAudience.Private
-public class PrefixTreeCell extends ByteBufferedCell implements SettableSequenceId,
+public class PrefixTreeCell extends ByteBufferCell implements SettableSequenceId,
     Comparable<Cell> {
   // Create a reference here? Can be removed too
   protected CellComparator comparator = CellComparator.COMPARATOR;

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index 88fe5e1..6eb293d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -34,7 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
+import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -745,7 +745,7 @@ public class HFileBlockIndex {
       // If we imagine that keys[-1] = -Infinity and
       // keys[numEntries] = Infinity, then we are maintaining an invariant that
       // keys[low - 1] < key < keys[high + 1] while narrowing down the range.
-      ByteBufferedKeyOnlyKeyValue nonRootIndexkeyOnlyKV = new ByteBufferedKeyOnlyKeyValue();
+      ByteBufferKeyOnlyKeyValue nonRootIndexkeyOnlyKV = new ByteBufferKeyOnlyKeyValue();
       ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
       while (low <= high) {
         mid = (low + high) >>> 1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 4887550..4cf1bf2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -30,7 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
+import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -491,7 +491,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     protected final HFile.Reader reader;
     private int currTagsLen;
     // buffer backed keyonlyKV
-    private ByteBufferedKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferedKeyOnlyKeyValue();
+    private ByteBufferKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferKeyOnlyKeyValue();
     // A pair for reusing in blockSeek() so that we don't garbage lot of objects
     final ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
 
@@ -1208,7 +1208,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         return new KeyValue.KeyOnlyKeyValue(keyBuff.array(), keyBuff.arrayOffset()
             + keyBuff.position(), klen);
       } else {
-        return new ByteBufferedKeyOnlyKeyValue(keyBuff, keyBuff.position(), klen);
+        return new ByteBufferKeyOnlyKeyValue(keyBuff, keyBuff.position(), klen);
       }
     }