Index: src/test/java/org/apache/hadoop/hbase/TestKeyValue.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (revision 1310067) +++ src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (working copy) @@ -19,6 +19,8 @@ */ 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; @@ -385,4 +387,23 @@ } } } + + /** + * 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())); + } } Index: src/main/java/org/apache/hadoop/hbase/KeyValue.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/KeyValue.java (revision 1310067) +++ src/main/java/org/apache/hadoop/hbase/KeyValue.java (working copy) @@ -27,7 +27,6 @@ import java.util.HashMap; import java.util.Map; -import com.google.common.primitives.Longs; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.io.HeapSize; @@ -37,6 +36,8 @@ import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.Writable; +import com.google.common.primitives.Longs; + /** * An HBase Key/Value. This is the fundamental HBase Type. * @@ -2005,6 +2006,7 @@ // 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.bytes = new byte[this.length];