Index: hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (revision 1415561) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (working copy) @@ -66,17 +66,6 @@ assertNotNull(ff); } - @Test public void testKeyValue() throws Exception { - final String name = "testKeyValue"; - byte [] row = Bytes.toBytes(name); - byte [] family = Bytes.toBytes(name); - byte [] qualifier = Bytes.toBytes(name); - KeyValue original = new KeyValue(row, family, qualifier); - byte [] bytes = Writables.getBytes(original); - KeyValue newone = (KeyValue)Writables.getWritable(bytes, new KeyValue()); - assertTrue(KeyValue.COMPARATOR.compare(original, newone) == 0); - } - @Test public void testTableDescriptor() throws Exception { final String name = "testTableDescriptor"; HTableDescriptor htd = createTableDescriptor(name); @@ -518,24 +507,6 @@ } - @Test public void testKeyValue2() throws Exception { - final String name = "testKeyValue2"; - byte[] row = name.getBytes(); - byte[] fam = "fam".getBytes(); - byte[] qf = "qf".getBytes(); - long ts = System.currentTimeMillis(); - byte[] val = "val".getBytes(); - - KeyValue kv = new KeyValue(row, fam, qf, ts, val); - - byte [] mb = Writables.getBytes(kv); - KeyValue deserializedKv = - (KeyValue)Writables.getWritable(mb, new KeyValue()); - assertTrue(Bytes.equals(kv.getBuffer(), deserializedKv.getBuffer())); - assertEquals(kv.getOffset(), deserializedKv.getOffset()); - assertEquals(kv.getLength(), deserializedKv.getLength()); - } - protected static final int MAXVERSIONS = 3; protected final static byte [] fam1 = Bytes.toBytes("colfamily1"); protected final static byte [] fam2 = Bytes.toBytes("colfamily2"); @@ -576,6 +547,4 @@ ); return htd; } - } - Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java (revision 1415561) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java (working copy) @@ -117,6 +117,7 @@ scopes.clear(); } int versionOrLength = in.readInt(); + // TODO: Change version when we protobuf. Also, change way we serialize KV! Pb it too. if (versionOrLength == VERSION_2) { // this is new style HLog entry containing multiple KeyValues. int numEdits = in.readInt(); @@ -124,9 +125,7 @@ if (compressionContext != null) { this.add(KeyValueCompression.readKV(in, compressionContext)); } else { - KeyValue kv = new KeyValue(); - kv.readFields(in); - this.add(kv); + this.add(KeyValue.create(in)); } } int numFamilies = in.readInt(); @@ -143,9 +142,7 @@ } else { // this is an old style HLog entry. The int that we just // read is actually the length of a single KeyValue - KeyValue kv = new KeyValue(); - kv.readFields(versionOrLength, in); - this.add(kv); + this.add(KeyValue.create(versionOrLength, in)); } } @@ -158,7 +155,7 @@ if (compressionContext != null) { KeyValueCompression.writeKV(out, kv, compressionContext); } else{ - kv.write(out); + KeyValue.write(kv, out); } } if (scopes == null) { Index: hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java =================================================================== --- hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (revision 1415561) +++ hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (working copy) @@ -18,8 +18,6 @@ */ package org.apache.hadoop.hbase; -import java.io.ByteArrayInputStream; -import java.io.DataInputStream; import java.io.IOException; import java.util.Set; import java.util.TreeSet; @@ -32,7 +30,6 @@ import org.apache.hadoop.hbase.KeyValue.MetaComparator; import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.WritableUtils; public class TestKeyValue extends TestCase { private final Log LOG = LogFactory.getLog(this.getClass().getName()); @@ -483,25 +480,6 @@ } /** - * The row cache is cleared and re-read for the new value - * - * @throws IOException - */ - public void testReadFields() throws IOException { - KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("cf1"), - Bytes.toBytes("qualifier1"), 12345L, Bytes.toBytes("value1")); - kv1.getRow(); // set row cache of kv1 - KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("cf2"), - Bytes.toBytes("qualifier2"), 12345L, Bytes.toBytes("value2")); - kv1.readFields(new DataInputStream(new ByteArrayInputStream(WritableUtils - .toByteArray(kv2)))); - // check equality - assertEquals(kv1, kv2); - // check cache state (getRow() return the cached value if the cache is set) - assertTrue(Bytes.equals(kv1.getRow(), kv2.getRow())); - } - - /** * Tests that getTimestamp() does always return the proper timestamp, even after updating it. * See HBASE-6265. */ @@ -515,5 +493,4 @@ assertEquals(HConstants.LATEST_TIMESTAMP, time1); assertEquals(12345L, time2); } -} - +} \ No newline at end of file Index: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java =================================================================== --- hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (revision 1415561) +++ hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (working copy) @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.io.RawComparator; -import org.apache.hadoop.io.Writable; import org.apache.hbase.Cell; import org.apache.hbase.cell.CellComparator; @@ -65,7 +64,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Evolving -public class KeyValue implements Cell, Writable, HeapSize { +public class KeyValue implements Cell, HeapSize { static final Log LOG = LogFactory.getLog(KeyValue.class); // TODO: Group Key-only comparators and operations into a Key class, just // for neatness sake, if can figure what to call it. @@ -2303,6 +2302,43 @@ } /** + * @param in Where to read bytes from + * @return KeyValue created by deserializing from in + * @throws IOException + */ + public static KeyValue create(final DataInput in) throws IOException { + return create(in.readInt(), in); + } + + /** + * Create a KeyValue reading length from in + * @param length + * @param in + * @return Created KeyValue + * @throws IOException + */ + public static KeyValue create(int length, final DataInput in) throws IOException { + // This is how the old Writables.readFrom used to deserialize. Didn't even vint. + byte [] bytes = new byte[length]; + in.readFully(bytes); + return new KeyValue(bytes, 0, length); + } + + /** + * Write out a KeyValue in the manner in which we used to when KeyValue was a Writable. + * @param kv + * @param out + * @throws IOException + */ + public static void write(final KeyValue kv, final DataOutput out) throws IOException { + // This is how the old Writables write used to serialize KVs. Need to figure way to make it work for all + // implementations. + int length = kv.getLength(); + out.writeInt(length); + out.write(kv.getBuffer(), kv.getOffset(), length); + } + + /** * Compare key portion of a {@link KeyValue} for keys in -ROOT- * table. */ @@ -2647,30 +2683,4 @@ sum += 2 * Bytes.SIZEOF_LONG;// timestampCache, memstoreTS return ClassSize.align(sum); } - - // this overload assumes that the length bytes have already been read, - // and it expects the length of the KeyValue to be explicitly passed - // to it. - public void readFields(int length, final DataInput in) throws IOException { - this.rowCache = null; - this.length = length; - this.offset = 0; - this.timestampCache = -1; - this.keyLength = 0; - this.bytes = new byte[this.length]; - in.readFully(this.bytes, 0, this.length); - } - - // Writable - @Override - public void readFields(final DataInput in) throws IOException { - int length = in.readInt(); - readFields(length, in); - } - - @Override - public void write(final DataOutput out) throws IOException { - out.writeInt(this.length); - out.write(this.bytes, this.offset, this.length); - } }