hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From anoopsamj...@apache.org
Subject [4/4] hbase git commit: HBASE-15786 Create DBB backed MSLAB pool.
Date Thu, 24 Nov 2016 15:18:06 GMT
HBASE-15786 Create DBB backed MSLAB pool.


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

Branch: refs/heads/master
Commit: 86e17858f73e742970e59936d34bfdb31fa286e1
Parents: a45b3c3
Author: anoopsamjohn <anoopsamjohn@gmail.com>
Authored: Thu Nov 24 20:47:41 2016 +0530
Committer: anoopsamjohn <anoopsamjohn@gmail.com>
Committed: Thu Nov 24 20:47:41 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |   8 +-
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |  12 +-
 .../hadoop/hbase/filter/PrefixFilter.java       |   8 +-
 .../hadoop/hbase/filter/TestComparators.java    |  14 +-
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  |   8 +-
 .../org/apache/hadoop/hbase/ByteBufferCell.java | 120 +++++
 .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java | 245 ++++++++++
 .../apache/hadoop/hbase/ByteBufferedCell.java   |  98 ----
 .../hbase/ByteBufferedKeyOnlyKeyValue.java      | 245 ----------
 .../org/apache/hadoop/hbase/CellComparator.java | 110 ++---
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 476 +++++++++++--------
 .../org/apache/hadoop/hbase/ExtendedCell.java   |   3 +-
 .../apache/hadoop/hbase/HBaseConfiguration.java |   2 -
 .../java/org/apache/hadoop/hbase/KeyValue.java  |   4 +-
 .../org/apache/hadoop/hbase/KeyValueUtil.java   |  52 +-
 .../apache/hadoop/hbase/OffheapKeyValue.java    |   8 +-
 .../io/encoding/AbstractDataBlockEncoder.java   |   4 +-
 .../io/encoding/BufferedDataBlockEncoder.java   |   8 +-
 .../hbase/io/encoding/RowIndexSeekerV1.java     |  14 +-
 .../hbase/io/util/HeapMemorySizeUtil.java       | 152 ------
 .../hadoop/hbase/util/ByteBufferUtils.java      |  32 ++
 .../apache/hadoop/hbase/TestCellComparator.java |   8 +-
 .../org/apache/hadoop/hbase/TestCellUtil.java   |  16 +-
 .../hadoop/hbase/TestOffheapKeyValue.java       |   6 +-
 .../hbase/io/TestTagCompressionContext.java     |  10 +-
 .../codec/prefixtree/PrefixTreeSeeker.java      |   6 +-
 .../codec/prefixtree/decode/PrefixTreeCell.java |   4 +-
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |   4 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |   6 +-
 .../hadoop/hbase/io/util/MemorySizeUtil.java    | 193 ++++++++
 .../apache/hadoop/hbase/regionserver/Chunk.java |  40 +-
 .../regionserver/DefaultHeapMemoryTuner.java    |  15 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   5 +-
 .../hbase/regionserver/HRegionServer.java       |  48 +-
 .../hbase/regionserver/HeapMemStoreLAB.java     | 245 ----------
 .../hbase/regionserver/HeapMemoryManager.java   |  20 +-
 .../hbase/regionserver/MemStoreChunkPool.java   | 109 ++---
 .../hbase/regionserver/MemStoreFlusher.java     |  28 +-
 .../hadoop/hbase/regionserver/MemStoreLAB.java  |  28 ++
 .../hbase/regionserver/MemStoreLABImpl.java     | 243 ++++++++++
 .../hadoop/hbase/regionserver/OffheapChunk.java |  54 +++
 .../hadoop/hbase/regionserver/OnheapChunk.java  |  53 +++
 .../hbase/regionserver/RSRpcServices.java       |   6 +-
 .../hbase/regionserver/SegmentFactory.java      |  19 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java   |   5 +-
 .../filter/TestSingleColumnValueFilter.java     |  36 +-
 .../hbase/regionserver/TestCellFlatSet.java     |   4 +-
 .../regionserver/TestCompactingMemStore.java    |   9 +-
 .../hbase/regionserver/TestDefaultMemStore.java |   4 +-
 .../regionserver/TestHeapMemoryManager.java     |   8 +-
 .../regionserver/TestMemStoreChunkPool.java     |  17 +-
 .../hbase/regionserver/TestMemStoreLAB.java     |  31 +-
 52 files changed, 1608 insertions(+), 1295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 806841f..7230d3a 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.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -91,9 +91,9 @@ public class ColumnPrefixFilter extends FilterBase {
   }
 
   private static int compareQualifierPart(Cell cell, int length, byte[] prefix) {
-    if (cell instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferedCell) cell).getQualifierPosition(), length, prefix, 0, length);
+    if (cell instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferCell) 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/86e17858/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 5ed5802..a22750d 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
@@ -23,7 +23,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -62,8 +62,8 @@ public class KeyOnlyFilter extends FilterBase {
   }
 
   private Cell createKeyOnlyCell(Cell c) {
-    if (c instanceof ByteBufferedCell) {
-      return new KeyOnlyByteBufferedCell((ByteBufferedCell) c, lenAsVal);
+    if (c instanceof ByteBufferCell) {
+      return new KeyOnlyByteBufferCell((ByteBufferCell) c, lenAsVal);
     } else {
       return new KeyOnlyCell(c, lenAsVal);
     }
@@ -232,11 +232,11 @@ public class KeyOnlyFilter extends FilterBase {
     }
   }
 
-  static class KeyOnlyByteBufferedCell extends ByteBufferedCell {
-    private ByteBufferedCell cell;
+  static class KeyOnlyByteBufferCell extends ByteBufferCell {
+    private ByteBufferCell cell;
     private boolean lenAsVal;
 
-    public KeyOnlyByteBufferedCell(ByteBufferedCell c, boolean lenAsVal) {
+    public KeyOnlyByteBufferCell(ByteBufferCell c, boolean lenAsVal) {
       this.cell = c;
       this.lenAsVal = lenAsVal;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 9bc6236..e3cefe5 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.ByteBufferedCell;
+import org.apache.hadoop.hbase.ByteBufferCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -62,9 +62,9 @@ public class PrefixFilter extends FilterBase {
     // else return true, filter row
     // if we are passed the prefix, set flag
     int cmp;
-    if (firstRowCell instanceof ByteBufferedCell) {
-      cmp = ByteBufferUtils.compareTo(((ByteBufferedCell) firstRowCell).getRowByteBuffer(),
-          ((ByteBufferedCell) firstRowCell).getRowPosition(), this.prefix.length,
+    if (firstRowCell instanceof ByteBufferCell) {
+      cmp = ByteBufferUtils.compareTo(((ByteBufferCell) firstRowCell).getRowByteBuffer(),
+          ((ByteBufferCell) 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/86e17858/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
index e3aa36b..d9e4033 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestComparators.java
@@ -25,7 +25,7 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
-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;
@@ -51,18 +51,18 @@ public class TestComparators {
     // Row compare
     KeyValue kv = new KeyValue(r1, f, q1, v1);
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     ByteArrayComparable comparable = new BinaryComparator(r1);
     assertEquals(0, CellComparator.compareRow(bbCell, comparable));
     assertEquals(0, CellComparator.compareRow(kv, comparable));
     kv = new KeyValue(r0, f, q1, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareRow(bbCell, comparable) > 0);
     assertTrue(CellComparator.compareRow(kv, comparable) > 0);
     kv = new KeyValue(r2, f, q1, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareRow(bbCell, comparable) < 0);
     assertTrue(CellComparator.compareRow(kv, comparable) < 0);
     // Qualifier compare
@@ -71,12 +71,12 @@ public class TestComparators {
     assertEquals(0, CellComparator.compareQualifier(kv, comparable));
     kv = new KeyValue(r2, f, q2, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertEquals(0, CellComparator.compareQualifier(bbCell, comparable));
     assertEquals(0, CellComparator.compareQualifier(kv, comparable));
     kv = new KeyValue(r2, f, q3, v1);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareQualifier(bbCell, comparable) < 0);
     assertTrue(CellComparator.compareQualifier(kv, comparable) < 0);
     // Value compare
@@ -85,7 +85,7 @@ public class TestComparators {
     assertEquals(0, CellComparator.compareValue(kv, comparable));
     kv = new KeyValue(r1, f, q1, v2);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
     assertTrue(CellComparator.compareValue(bbCell, comparable) < 0);
     assertTrue(CellComparator.compareValue(kv, comparable) < 0);
     // Family compare

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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 31f7904..0e88c4c 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
@@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferedCell;
+import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -71,7 +71,7 @@ public class TestKeyOnlyFilter {
         v.length, tags);
 
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    ByteBufferedCellImpl bbCell = new ByteBufferedCellImpl(buffer, 0,
+    ByteBufferCellImpl bbCell = new ByteBufferCellImpl(buffer, 0,
         buffer.remaining());
 
     // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
@@ -89,7 +89,7 @@ public class TestKeyOnlyFilter {
     KeyValue KeyOnlyKeyValue = new KeyValue(newBuffer);
 
     KeyOnlyCell keyOnlyCell = new KeyOnlyCell(kv, lenAsVal);
-    KeyOnlyByteBufferedCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferedCell(
+    KeyOnlyByteBufferCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferCell(
         bbCell, lenAsVal);
 
     assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyCell));

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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
new file mode 100644
index 0000000..918a805
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
@@ -0,0 +1,120 @@
+/**
+ * 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.hadoop.hbase.classification.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();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/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
new file mode 100644
index 0000000..0398047
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
@@ -0,0 +1,245 @@
+/**
+ *
+ * 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.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This is a key only Cell implementation which is identical to {@link KeyValue.KeyOnlyKeyValue}
+ * with respect to key serialization but have its data in the form of Byte buffer
+ * (onheap and offheap).
+ */
+@InterfaceAudience.Private
+public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
+
+  private ByteBuffer buf;
+  private int offset = 0; // offset into buffer where key starts at
+  private int length = 0; // length of this.
+  private short rowLen;
+
+  /**
+   * Used in cases where we want to avoid lot of garbage by allocating new objects with different
+   * keys. Use the emtpy construtor and set the keys using {@link #setKey(ByteBuffer, int, int)}
+   */
+  public ByteBufferKeyOnlyKeyValue() {
+  }
+
+  public ByteBufferKeyOnlyKeyValue(ByteBuffer buf, int offset, int length) {
+    setKey(buf, offset, length);
+  }
+
+  /**
+   * A setter that helps to avoid object creation every time and whenever
+   * there is a need to create new OffheapKeyOnlyKeyValue.
+   * @param key
+   * @param offset
+   * @param length
+   */
+  public void setKey(ByteBuffer key, int offset, int length) {
+    this.buf = key;
+    this.offset = offset;
+    this.length = length;
+    this.rowLen = ByteBufferUtils.toShort(this.buf, this.offset);
+  }
+
+  @Override
+  public byte[] getRowArray() {
+    if (this.buf.hasArray()) {
+      return this.buf.array();
+    }
+    return CellUtil.cloneRow(this);
+  }
+
+  @Override
+  public int getRowOffset() {
+    if (this.buf.hasArray()) {
+      return getRowPosition() + this.buf.arrayOffset();
+    }
+    return 0;
+  }
+
+  @Override
+  public short getRowLength() {
+    return this.rowLen;
+  }
+
+  @Override
+  public byte[] getFamilyArray() {
+    if (this.buf.hasArray()) {
+      return this.buf.array();
+    }
+    return CellUtil.cloneFamily(this);
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    if (this.buf.hasArray()) {
+      return getFamilyPosition() + this.buf.arrayOffset();
+    }
+    return 0;
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return getFamilyLength(getFamilyLengthPosition());
+  }
+
+  private byte getFamilyLength(int famLenPos) {
+    return ByteBufferUtils.toByte(this.buf, famLenPos);
+  }
+
+  @Override
+  public byte[] getQualifierArray() {
+    if (this.buf.hasArray()) {
+      return this.buf.array();
+    }
+    return CellUtil.cloneQualifier(this);
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    if (this.buf.hasArray()) {
+      return getQualifierPosition() + this.buf.arrayOffset();
+    }
+    return 0;
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return getQualifierLength(getRowLength(), getFamilyLength());
+  }
+
+  private int getQualifierLength(int rlength, int flength) {
+    return this.length - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
+  }
+
+  @Override
+  public long getTimestamp() {
+    return ByteBufferUtils.toLong(this.buf, getTimestampOffset());
+  }
+
+  private int getTimestampOffset() {
+    return this.offset + this.length - KeyValue.TIMESTAMP_TYPE_SIZE;
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
+  }
+
+  @Override
+  public long getSequenceId() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public int getValueOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getValueLength() {
+    return 0;
+  }
+
+  @Override
+  public byte[] getTagsArray() {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public int getTagsOffset() {
+    return 0;
+  }
+
+  @Override
+  public int getTagsLength() {
+    return 0;
+  }
+
+  @Override
+  public ByteBuffer getRowByteBuffer() {
+    return this.buf;
+  }
+
+  @Override
+  public int getRowPosition() {
+    return this.offset + Bytes.SIZEOF_SHORT;
+  }
+
+  @Override
+  public ByteBuffer getFamilyByteBuffer() {
+    return this.buf;
+  }
+
+  @Override
+  public int getFamilyPosition() {
+    return getFamilyLengthPosition() + Bytes.SIZEOF_BYTE;
+  }
+
+  // The position in BB where the family length is added.
+  private int getFamilyLengthPosition() {
+    return this.offset + Bytes.SIZEOF_SHORT + getRowLength();
+  }
+
+  @Override
+  public ByteBuffer getQualifierByteBuffer() {
+    return this.buf;
+  }
+
+  @Override
+  public int getQualifierPosition() {
+    int famLenPos = getFamilyLengthPosition();
+    return famLenPos + Bytes.SIZEOF_BYTE + getFamilyLength(famLenPos);
+  }
+
+  @Override
+  public ByteBuffer getValueByteBuffer() {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public int getValuePosition() {
+    return 0;
+  }
+
+  @Override
+  public ByteBuffer getTagsByteBuffer() {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public int getTagsPosition() {
+    return 0;
+  }
+
+  @Override
+  public String toString() {
+    return CellUtil.toString(this, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java
deleted file mode 100644
index cca1271..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedCell.java
+++ /dev/null
@@ -1,98 +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.hadoop.hbase.classification.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.
- */
-@InterfaceAudience.Private
-public abstract class ByteBufferedCell 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();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedKeyOnlyKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedKeyOnlyKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedKeyOnlyKeyValue.java
deleted file mode 100644
index c6c02ad..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferedKeyOnlyKeyValue.java
+++ /dev/null
@@ -1,245 +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.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * This is a key only Cell implementation which is identical to {@link KeyValue.KeyOnlyKeyValue}
- * with respect to key serialization but have its data in the form of Byte buffer
- * (onheap and offheap).
- */
-@InterfaceAudience.Private
-public class ByteBufferedKeyOnlyKeyValue extends ByteBufferedCell {
-
-  private ByteBuffer buf;
-  private int offset = 0; // offset into buffer where key starts at
-  private int length = 0; // length of this.
-  private short rowLen;
-
-  /**
-   * Used in cases where we want to avoid lot of garbage by allocating new objects with different
-   * keys. Use the emtpy construtor and set the keys using {@link #setKey(ByteBuffer, int, int)}
-   */
-  public ByteBufferedKeyOnlyKeyValue() {
-  }
-
-  public ByteBufferedKeyOnlyKeyValue(ByteBuffer buf, int offset, int length) {
-    setKey(buf, offset, length);
-  }
-
-  /**
-   * A setter that helps to avoid object creation every time and whenever
-   * there is a need to create new OffheapKeyOnlyKeyValue.
-   * @param key
-   * @param offset
-   * @param length
-   */
-  public void setKey(ByteBuffer key, int offset, int length) {
-    this.buf = key;
-    this.offset = offset;
-    this.length = length;
-    this.rowLen = ByteBufferUtils.toShort(this.buf, this.offset);
-  }
-
-  @Override
-  public byte[] getRowArray() {
-    if (this.buf.hasArray()) {
-      return this.buf.array();
-    }
-    return CellUtil.cloneRow(this);
-  }
-
-  @Override
-  public int getRowOffset() {
-    if (this.buf.hasArray()) {
-      return getRowPosition() + this.buf.arrayOffset();
-    }
-    return 0;
-  }
-
-  @Override
-  public short getRowLength() {
-    return this.rowLen;
-  }
-
-  @Override
-  public byte[] getFamilyArray() {
-    if (this.buf.hasArray()) {
-      return this.buf.array();
-    }
-    return CellUtil.cloneFamily(this);
-  }
-
-  @Override
-  public int getFamilyOffset() {
-    if (this.buf.hasArray()) {
-      return getFamilyPosition() + this.buf.arrayOffset();
-    }
-    return 0;
-  }
-
-  @Override
-  public byte getFamilyLength() {
-    return getFamilyLength(getFamilyLengthPosition());
-  }
-
-  private byte getFamilyLength(int famLenPos) {
-    return ByteBufferUtils.toByte(this.buf, famLenPos);
-  }
-
-  @Override
-  public byte[] getQualifierArray() {
-    if (this.buf.hasArray()) {
-      return this.buf.array();
-    }
-    return CellUtil.cloneQualifier(this);
-  }
-
-  @Override
-  public int getQualifierOffset() {
-    if (this.buf.hasArray()) {
-      return getQualifierPosition() + this.buf.arrayOffset();
-    }
-    return 0;
-  }
-
-  @Override
-  public int getQualifierLength() {
-    return getQualifierLength(getRowLength(), getFamilyLength());
-  }
-
-  private int getQualifierLength(int rlength, int flength) {
-    return this.length - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
-  }
-
-  @Override
-  public long getTimestamp() {
-    return ByteBufferUtils.toLong(this.buf, getTimestampOffset());
-  }
-
-  private int getTimestampOffset() {
-    return this.offset + this.length - KeyValue.TIMESTAMP_TYPE_SIZE;
-  }
-
-  @Override
-  public byte getTypeByte() {
-    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 1);
-  }
-
-  @Override
-  public long getSequenceId() {
-    return 0;
-  }
-
-  @Override
-  public byte[] getValueArray() {
-    throw new IllegalArgumentException("This is a key only Cell");
-  }
-
-  @Override
-  public int getValueOffset() {
-    return 0;
-  }
-
-  @Override
-  public int getValueLength() {
-    return 0;
-  }
-
-  @Override
-  public byte[] getTagsArray() {
-    throw new IllegalArgumentException("This is a key only Cell");
-  }
-
-  @Override
-  public int getTagsOffset() {
-    return 0;
-  }
-
-  @Override
-  public int getTagsLength() {
-    return 0;
-  }
-
-  @Override
-  public ByteBuffer getRowByteBuffer() {
-    return this.buf;
-  }
-
-  @Override
-  public int getRowPosition() {
-    return this.offset + Bytes.SIZEOF_SHORT;
-  }
-
-  @Override
-  public ByteBuffer getFamilyByteBuffer() {
-    return this.buf;
-  }
-
-  @Override
-  public int getFamilyPosition() {
-    return getFamilyLengthPosition() + Bytes.SIZEOF_BYTE;
-  }
-
-  // The position in BB where the family length is added.
-  private int getFamilyLengthPosition() {
-    return this.offset + Bytes.SIZEOF_SHORT + getRowLength();
-  }
-
-  @Override
-  public ByteBuffer getQualifierByteBuffer() {
-    return this.buf;
-  }
-
-  @Override
-  public int getQualifierPosition() {
-    int famLenPos = getFamilyLengthPosition();
-    return famLenPos + Bytes.SIZEOF_BYTE + getFamilyLength(famLenPos);
-  }
-
-  @Override
-  public ByteBuffer getValueByteBuffer() {
-    throw new IllegalArgumentException("This is a key only Cell");
-  }
-
-  @Override
-  public int getValuePosition() {
-    return 0;
-  }
-
-  @Override
-  public ByteBuffer getTagsByteBuffer() {
-    throw new IllegalArgumentException("This is a key only Cell");
-  }
-
-  @Override
-  public int getTagsPosition() {
-    return 0;
-  }
-
-  @Override
-  public String toString() {
-    return CellUtil.toString(this, false);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/86e17858/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index 4a5c0b7..bb08d6c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -159,35 +159,35 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public final static int compareFamilies(Cell left, Cell right) {
-    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
-          ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(),
-          ((ByteBufferedCell) right).getFamilyByteBuffer(),
-          ((ByteBufferedCell) right).getFamilyPosition(), right.getFamilyLength());
-    }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
-          ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(),
+    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 ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
+          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
           right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
     }
-    if (right instanceof ByteBufferedCell) {
+    if (right instanceof ByteBufferCell) {
       // 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(),
-          ((ByteBufferedCell)right).getFamilyByteBuffer(),
-          ((ByteBufferedCell)right).getFamilyPosition(), right.getFamilyLength());
+          ((ByteBufferCell)right).getFamilyByteBuffer(),
+          ((ByteBufferCell)right).getFamilyPosition(), right.getFamilyLength());
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
   }
 
   private final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
-          ((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(), right,
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
+          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), right,
           roffset, rlength);
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
@@ -201,28 +201,28 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
    */
   public final static int compareQualifiers(Cell left, Cell right) {
-    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
+    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
       return ByteBufferUtils
-          .compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
-              ((ByteBufferedCell) left).getQualifierPosition(),
-              left.getQualifierLength(), ((ByteBufferedCell) right).getQualifierByteBuffer(),
-              ((ByteBufferedCell) right).getQualifierPosition(),
+          .compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
+              ((ByteBufferCell) left).getQualifierPosition(),
+              left.getQualifierLength(), ((ByteBufferCell) right).getQualifierByteBuffer(),
+              ((ByteBufferCell) right).getQualifierPosition(),
               right.getQualifierLength());
     }
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
-          ((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
+          ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
           right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
     }
-    if (right instanceof ByteBufferedCell) {
+    if (right instanceof ByteBufferCell) {
       // 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(),
-          ((ByteBufferedCell)right).getQualifierByteBuffer(),
-          ((ByteBufferedCell)right).getQualifierPosition(), right.getQualifierLength());
+          ((ByteBufferCell)right).getQualifierByteBuffer(),
+          ((ByteBufferCell)right).getQualifierPosition(), right.getQualifierLength());
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
@@ -230,9 +230,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
   }
 
   public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
-          ((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
+          ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
           right, rOffset, rLength);
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
@@ -328,25 +328,25 @@ public class CellComparator implements Comparator<Cell>, Serializable {
     if (left == right) {
       return 0;
     }
-    if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
-          ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),
-          ((ByteBufferedCell) right).getRowByteBuffer(),
-          ((ByteBufferedCell) right).getRowPosition(), right.getRowLength());
+    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 ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
-          ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
+          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
           right.getRowArray(), right.getRowOffset(), right.getRowLength());
     }
-    if (right instanceof ByteBufferedCell) {
+    if (right instanceof ByteBufferCell) {
       // 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(),
-          ((ByteBufferedCell)right).getRowByteBuffer(),
-          ((ByteBufferedCell)right).getRowPosition(), right.getRowLength());
+          ((ByteBufferCell)right).getRowByteBuffer(),
+          ((ByteBufferCell)right).getRowPosition(), right.getRowLength());
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
         right.getRowArray(), right.getRowOffset(), right.getRowLength());
@@ -369,9 +369,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    *         than byte[], -1 otherwise
    */
   public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
-    if (left instanceof ByteBufferedCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
-          ((ByteBufferedCell) left).getRowPosition(), left.getRowLength(), right,
+    if (left instanceof ByteBufferCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
+          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(), right,
           roffset, rlength);
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
@@ -521,9 +521,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return result comparing cell's row
    */
   public static int compareRow(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferedCell) {
-      return comparator.compareTo(((ByteBufferedCell) cell).getRowByteBuffer(),
-          ((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
+    if (cell instanceof ByteBufferCell) {
+      return comparator.compareTo(((ByteBufferCell) cell).getRowByteBuffer(),
+          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
     }
     return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
@@ -535,9 +535,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return result comparing cell's column family
    */
   public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferedCell) {
-      return comparator.compareTo(((ByteBufferedCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength());
+    if (cell instanceof ByteBufferCell) {
+      return comparator.compareTo(((ByteBufferCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength());
     }
     return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
         cell.getFamilyLength());
@@ -550,9 +550,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return result comparing cell's qualifier
    */
   public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferedCell) {
-      return comparator.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
+    if (cell instanceof ByteBufferCell) {
+      return comparator.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
     }
     return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
         cell.getQualifierLength());
@@ -565,9 +565,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
    * @return result comparing cell's value
    */
   public static int compareValue(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferedCell) {
-      return comparator.compareTo(((ByteBufferedCell) cell).getValueByteBuffer(),
-          ((ByteBufferedCell) cell).getValuePosition(), cell.getValueLength());
+    if (cell instanceof ByteBufferCell) {
+      return comparator.compareTo(((ByteBufferCell) cell).getValueByteBuffer(),
+          ((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
     }
     return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }


Mime
View raw message