You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/04/11 17:31:21 UTC

[16/54] [abbrv] hbase git commit: HBASE-17881 Remove the ByteBufferCellImpl

HBASE-17881 Remove the ByteBufferCellImpl


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

Branch: refs/heads/HBASE-16961
Commit: 18c5ecf6ed57e80b32568ca1a1a12c7af36bab46
Parents: 1a701ce
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Wed Apr 5 21:11:29 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Apr 7 21:14:19 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/TestComparators.java    |  14 +-
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  |   4 +-
 .../apache/hadoop/hbase/TestCellComparator.java |   7 +-
 .../org/apache/hadoop/hbase/TestCellUtil.java   | 184 +------------------
 .../filter/TestSingleColumnValueFilter.java     |  36 ++--
 5 files changed, 36 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/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 d9e4033..0c69ece 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
@@ -21,11 +21,11 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
-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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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/18c5ecf6/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 f22e5d4..33e3cd9 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
@@ -23,12 +23,12 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.CellUtil;
 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.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -71,7 +71,7 @@ public class TestKeyOnlyFilter {
         v.length, tags);
 
     ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
-    ByteBufferCellImpl bbCell = new ByteBufferCellImpl(buffer, 0,
+    ByteBufferKeyValue bbCell = new ByteBufferKeyValue(buffer, 0,
         buffer.remaining());
 
     // KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>

http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/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 8740cfb..791b9ee 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,6 @@ import static org.junit.Assert.assertTrue;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
-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 +101,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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell1 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     kv = new KeyValue(r2, f1, q1, v);
     buffer = ByteBuffer.wrap(kv.getBuffer());
-    Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell2 = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell3 = new ByteBufferKeyValue(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/18c5ecf6/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 441d1b5..bbc7a31 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
@@ -33,7 +33,6 @@ import java.util.TreeMap;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -416,7 +415,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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     byte[] rDest = CellUtil.cloneRow(bbCell);
     assertTrue(Bytes.equals(r, rDest));
     byte[] fDest = CellUtil.cloneFamily(bbCell);
@@ -440,10 +439,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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell1 = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell2 = new ByteBufferKeyValue(buffer, 0, buffer.remaining());
     assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
     assertTrue(CellUtil.matchingRows(kv, bbCell2));
     assertTrue(CellUtil.matchingRow(bbCell1, r));
@@ -473,191 +472,20 @@ 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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    Cell bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl(buffer, 0, buffer.remaining());
+    bbCell = new ByteBufferKeyValue(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 ByteBufferCellImpl extends ByteBufferCell {
-
-    private final ByteBuffer buffer;
-    private final int offset, length;
-
-    public ByteBufferCellImpl(ByteBuffer buffer, int offset, int length) {
-      this.buffer = buffer;
-      this.offset = offset;
-      this.length = length;
-    }
-
-    @Override
-    public byte[] getRowArray() {
-      return CellUtil.cloneRow(this);
-    }
-
-    @Override
-    public int getRowOffset() {
-      return 0;
-    }
-
-    @Override
-    public short getRowLength() {
-      return ByteBufferUtils.toShort(this.buffer, this.offset + KeyValue.ROW_OFFSET);
-    }
-
-    @Override
-    public byte[] getFamilyArray() {
-      return CellUtil.cloneFamily(this);
-    }
-
-    @Override
-    public int getFamilyOffset() {
-      return 0;
-    }
-
-    @Override
-    public byte getFamilyLength() {
-      return this.buffer.get(getFamilyPosition() - 1);
-    }
-
-    @Override
-    public byte[] getQualifierArray() {
-      return CellUtil.cloneQualifier(this);
-    }
-
-    @Override
-    public int getQualifierOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getQualifierLength() {
-      return getKeyLength()
-          - (int) KeyValue.getKeyDataStructureSize(getRowLength(), getFamilyLength(), 0);
-    }
-
-    private int getKeyLength() {
-      return ByteBufferUtils.toInt(this.buffer, this.offset);
-    }
-
-    @Override
-    public long getTimestamp() {
-      int tsOffset = this.offset + KeyValue.ROW_OFFSET + getKeyLength()
-          - KeyValue.TIMESTAMP_TYPE_SIZE;
-      return ByteBufferUtils.toLong(buffer, tsOffset);
-    }
-
-    @Override
-    public byte getTypeByte() {
-      return KeyValue.Type.Put.getCode();
-    }
-
-    @Override
-    public long getSequenceId() {
-      return 0;
-    }
-
-    @Override
-    public byte[] getValueArray() {
-      return CellUtil.cloneValue(this);
-    }
-
-    @Override
-    public int getValueOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getValueLength() {
-      return ByteBufferUtils.toInt(this.buffer, this.offset + KeyValue.KEY_LENGTH_SIZE);
-    }
-
-    @Override
-    public byte[] getTagsArray() {
-      byte[] tDest = new byte[getTagsLength()];
-      CellUtil.copyTagTo(this, tDest, 0);
-      return tDest;
-    }
-
-    @Override
-    public int getTagsOffset() {
-      return 0;
-    }
-
-    @Override
-    public int getTagsLength() {
-      int tagsLen = this.length
-          - (getKeyLength() + getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
-      if (tagsLen > 0) {
-        tagsLen -= KeyValue.TAGS_LENGTH_SIZE;
-      }
-      return tagsLen;
-    }
-
-    @Override
-    public ByteBuffer getRowByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getRowPosition() {
-      return this.offset + KeyValue.ROW_KEY_OFFSET;
-    }
-
-    @Override
-    public ByteBuffer getFamilyByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getFamilyPosition() {
-      return this.offset + KeyValue.ROW_KEY_OFFSET + getRowLength() + Bytes.SIZEOF_BYTE;
-    }
-
-    @Override
-    public ByteBuffer getQualifierByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getQualifierPosition() {
-      return getFamilyPosition() + getFamilyLength();
-    }
-
-    @Override
-    public ByteBuffer getValueByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getValuePosition() {
-      return this.offset + KeyValue.ROW_OFFSET + getKeyLength();
-    }
-
-    @Override
-    public ByteBuffer getTagsByteBuffer() {
-      return this.buffer;
-    }
-
-    @Override
-    public int getTagsPosition() {
-      int tagsLen = getTagsLength();
-      if (tagsLen == 0) {
-        return this.offset + this.length;
-      }
-      return this.offset + this.length - tagsLen;
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/18c5ecf6/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
index 48a5b6f..134e8d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
@@ -24,10 +24,10 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.regex.Pattern;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -108,7 +108,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
 
@@ -117,7 +117,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     filter.reset();
 
@@ -126,7 +126,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     filter.reset();
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
   }
 
@@ -135,29 +135,29 @@ public class TestSingleColumnValueFilter {
     KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
     assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
     assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
     assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterNotNull", filter.filterRow());
     filter.reset();
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
     assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
     assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
     assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
     assertTrue("basicFilterNotNull", filter.filterRow());
@@ -166,12 +166,12 @@ public class TestSingleColumnValueFilter {
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
     assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
     assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("basicFilterNotNull", filter.filterRow());
   }
@@ -181,14 +181,14 @@ public class TestSingleColumnValueFilter {
     KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
     assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("null1FilterRow", filter.filterRow());
     filter.reset();
     kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
     assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertTrue("null2FilterRow", filter.filterRow());
   }
@@ -200,13 +200,13 @@ public class TestSingleColumnValueFilter {
     assertTrue("substrTrue",
       filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_2);
     assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("substrFilterNotNull", filter.filterRow());
@@ -219,13 +219,13 @@ public class TestSingleColumnValueFilter {
     assertTrue("regexTrue",
       filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
       FULLSTRING_2);
     assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     buffer = kv.getBuffer();
-    c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("regexFilterNotNull", filter.filterRow());
@@ -238,7 +238,7 @@ public class TestSingleColumnValueFilter {
     assertTrue("regexTrue",
       filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
     byte[] buffer = kv.getBuffer();
-    Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
+    Cell c = new ByteBufferKeyValue(ByteBuffer.wrap(buffer), 0, buffer.length);
     assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
     assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
     assertFalse("regexFilterNotNull", filter.filterRow());