You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by op...@apache.org on 2018/11/12 03:41:20 UTC

hbase git commit: Revert "HBASE-21401 Sanity check in BaseDecoder#parseCell"

Repository: hbase
Updated Branches:
  refs/heads/master 5e84997f2 -> 362b5dd25


Revert "HBASE-21401 Sanity check in BaseDecoder#parseCell"

This reverts commit f17382792fc9d9eb7aeedbaa7faa48ce6dbd42d4.


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

Branch: refs/heads/master
Commit: 362b5dd25980af11ed6dde9c046bb10893a20a56
Parents: 5e84997
Author: huzheng <op...@gmail.com>
Authored: Mon Nov 12 11:37:46 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Mon Nov 12 11:37:46 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/KeyValue.java  |   9 -
 .../org/apache/hadoop/hbase/KeyValueUtil.java   | 149 +---------
 .../hadoop/hbase/codec/KeyValueCodec.java       |   3 +-
 .../hbase/codec/KeyValueCodecWithTags.java      |   2 +-
 .../hbase/io/encoding/RowIndexSeekerV1.java     |   2 +-
 .../org/apache/hadoop/hbase/TestKeyValue.java   | 295 ++++++++++---------
 .../hadoop/hbase/regionserver/HStore.java       |   1 +
 7 files changed, 178 insertions(+), 283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/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 f913124..f7f6c0d 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
@@ -252,15 +252,6 @@ public class KeyValue implements ExtendedCell, Cloneable {
     }
 
     /**
-     * True to indicate that the byte b is a valid type.
-     * @param b byte to check
-     * @return true or false
-     */
-    static boolean isValidType(byte b) {
-      return codeArray[b & 0xff] != null;
-    }
-
-    /**
      * Cannot rely on enum ordinals . They change if item is removed or moved.
      * Do our own codes.
      * @param b

http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/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 fbec792..1b61d1e 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
@@ -518,145 +518,17 @@ public class KeyValueUtil {
     return (long) length + Bytes.SIZEOF_INT;
   }
 
-  static String bytesToHex(byte[] buf, int offset, int length) {
-    return ", KeyValueBytesHex=" + Bytes.toStringBinary(buf, offset, length) + ", offset=" + offset
-        + ", length=" + length;
-  }
-
-  private static void checkKeyValueBytes(byte[] buf, int offset, int length, boolean withTags)
-      throws IOException {
-    int pos = offset, endOffset = offset + length;
-    // check the key
-    if (pos + Bytes.SIZEOF_INT > endOffset) {
-      throw new IOException(
-          "Overflow when reading key length at position=" + pos + bytesToHex(buf, offset, length));
-    }
-    int keyLen = Bytes.toInt(buf, pos, Bytes.SIZEOF_INT);
-    pos += Bytes.SIZEOF_INT;
-    if (keyLen <= 0 || pos + keyLen > endOffset) {
-      throw new IOException(
-          "Invalid key length in KeyValue. keyLength=" + keyLen + bytesToHex(buf, offset, length));
-    }
-    // check the value
-    if (pos + Bytes.SIZEOF_INT > endOffset) {
-      throw new IOException("Overflow when reading value length at position=" + pos
-          + bytesToHex(buf, offset, length));
-    }
-    int valLen = Bytes.toInt(buf, pos, Bytes.SIZEOF_INT);
-    pos += Bytes.SIZEOF_INT;
-    if (valLen < 0 || pos + valLen > endOffset) {
-      throw new IOException("Invalid value length in KeyValue, valueLength=" + valLen
-          + bytesToHex(buf, offset, length));
-    }
-    // check the row
-    if (pos + Bytes.SIZEOF_SHORT > endOffset) {
-      throw new IOException(
-          "Overflow when reading row length at position=" + pos + bytesToHex(buf, offset, length));
-    }
-    short rowLen = Bytes.toShort(buf, pos, Bytes.SIZEOF_SHORT);
-    pos += Bytes.SIZEOF_SHORT;
-    if (rowLen < 0 || pos + rowLen > endOffset) {
-      throw new IOException(
-          "Invalid row length in KeyValue, rowLength=" + rowLen + bytesToHex(buf, offset, length));
-    }
-    pos += rowLen;
-    // check the family
-    if (pos + Bytes.SIZEOF_BYTE > endOffset) {
-      throw new IOException("Overflow when reading family length at position=" + pos
-          + bytesToHex(buf, offset, length));
-    }
-    int familyLen = buf[pos];
-    pos += Bytes.SIZEOF_BYTE;
-    if (familyLen < 0 || pos + familyLen > endOffset) {
-      throw new IOException("Invalid family length in KeyValue, familyLength=" + familyLen
-          + bytesToHex(buf, offset, length));
-    }
-    pos += familyLen;
-    // check the qualifier
-    int qualifierLen = keyLen - Bytes.SIZEOF_SHORT - rowLen - Bytes.SIZEOF_BYTE - familyLen
-        - Bytes.SIZEOF_LONG - Bytes.SIZEOF_BYTE;
-    if (qualifierLen < 0 || pos + qualifierLen > endOffset) {
-      throw new IOException("Invalid qualifier length in KeyValue, qualifierLen=" + qualifierLen
-          + bytesToHex(buf, offset, length));
-    }
-    pos += qualifierLen;
-    // check the timestamp
-    if (pos + Bytes.SIZEOF_LONG > endOffset) {
-      throw new IOException(
-          "Overflow when reading timestamp at position=" + pos + bytesToHex(buf, offset, length));
-    }
-    long timestamp = Bytes.toLong(buf, pos, Bytes.SIZEOF_LONG);
-    if (timestamp < 0) {
-      throw new IOException(
-          "Timestamp cannot be negative, ts=" + timestamp + bytesToHex(buf, offset, length));
-    }
-    pos += Bytes.SIZEOF_LONG;
-    // check the type
-    if (pos + Bytes.SIZEOF_BYTE > endOffset) {
-      throw new IOException(
-          "Overflow when reading type at position=" + pos + bytesToHex(buf, offset, length));
-    }
-    byte type = buf[pos];
-    if (!Type.isValidType(type)) {
-      throw new IOException(
-          "Invalid type in KeyValue, type=" + type + bytesToHex(buf, offset, length));
-    }
-    pos += Bytes.SIZEOF_BYTE;
-    // check the value
-    if (pos + valLen > endOffset) {
-      throw new IOException(
-          "Overflow when reading value part at position=" + pos + bytesToHex(buf, offset, length));
-    }
-    pos += valLen;
-    // check the tags
-    if (withTags) {
-      if (pos == endOffset) {
-        // withTags is true but no tag in the cell.
-        return;
-      }
-      if (pos + Bytes.SIZEOF_SHORT > endOffset) {
-        throw new IOException("Overflow when reading tags length at position=" + pos
-            + bytesToHex(buf, offset, length));
-      }
-      short tagsLen = Bytes.toShort(buf, pos);
-      pos += Bytes.SIZEOF_SHORT;
-      if (tagsLen < 0 || pos + tagsLen > endOffset) {
-        throw new IOException("Invalid tags length in KeyValue at position="
-            + (pos - Bytes.SIZEOF_SHORT) + bytesToHex(buf, offset, length));
-      }
-      int tagsEndOffset = pos + tagsLen;
-      for (; pos < tagsEndOffset;) {
-        if (pos + Tag.TAG_LENGTH_SIZE > endOffset) {
-          throw new IOException("Overflow when reading tag length at position=" + pos
-              + bytesToHex(buf, offset, length));
-        }
-        short tagLen = Bytes.toShort(buf, pos);
-        pos += Tag.TAG_LENGTH_SIZE;
-        // tagLen contains one byte tag type, so must be not less than 1.
-        if (tagLen < 1 || pos + tagLen > endOffset) {
-          throw new IOException("Invalid tag length at position=" + (pos - Tag.TAG_LENGTH_SIZE)
-              + ", tagLength=" + tagLen + bytesToHex(buf, offset, length));
-        }
-        pos += tagLen;
-      }
-    }
-    if (pos != endOffset) {
-      throw new IOException("Some redundant bytes in KeyValue's buffer, startOffset=" + pos
-          + ", endOffset=" + endOffset + bytesToHex(buf, offset, length));
-    }
-  }
-
   /**
    * Create a KeyValue reading from the raw InputStream. Named
-   * <code>createKeyValueFromInputStream</code> so doesn't clash with {@link #create(DataInput)}
-   * @param in inputStream to read.
+   * <code>iscreate</code> so doesn't clash with {@link #create(DataInput)}
+   *
+   * @param in
    * @param withTags whether the keyvalue should include tags are not
-   * @return Created KeyValue OR if we find a length of zero, we will return null which can be
-   *         useful marking a stream as done.
+   * @return Created KeyValue OR if we find a length of zero, we will return
+   *         null which can be useful marking a stream as done.
    * @throws IOException
    */
-  public static KeyValue createKeyValueFromInputStream(InputStream in, boolean withTags)
-      throws IOException {
+  public static KeyValue iscreate(final InputStream in, boolean withTags) throws IOException {
     byte[] intBytes = new byte[Bytes.SIZEOF_INT];
     int bytesRead = 0;
     while (bytesRead < intBytes.length) {
@@ -669,11 +541,14 @@ public class KeyValueUtil {
       }
       bytesRead += n;
     }
+    // TODO: perhaps some sanity check is needed here.
     byte[] bytes = new byte[Bytes.toInt(intBytes)];
     IOUtils.readFully(in, bytes, 0, bytes.length);
-    checkKeyValueBytes(bytes, 0, bytes.length, withTags);
-    return withTags ? new KeyValue(bytes, 0, bytes.length)
-        : new NoTagsKeyValue(bytes, 0, bytes.length);
+    if (withTags) {
+      return new KeyValue(bytes, 0, bytes.length);
+    } else {
+      return new NoTagsKeyValue(bytes, 0, bytes.length);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
index 19a80b0..9a5db3c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
@@ -72,7 +72,7 @@ public class KeyValueCodec implements Codec {
     @Override
     protected Cell parseCell() throws IOException {
       // No tags here
-      return KeyValueUtil.createKeyValueFromInputStream(in, false);
+      return KeyValueUtil.iscreate(in, false);
     }
   }
 
@@ -114,6 +114,7 @@ public class KeyValueCodec implements Codec {
       // We know there is not going to be any tags.
       return new NoTagsByteBufferKeyValue(bb, pos, len);
     }
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
index 34c2b87..bf9b375 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
@@ -78,7 +78,7 @@ public class KeyValueCodecWithTags implements Codec {
     @Override
     protected Cell parseCell() throws IOException {
       // create KeyValue with tags
-      return KeyValueUtil.createKeyValueFromInputStream(in, true);
+      return KeyValueUtil.iscreate(in, true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/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 e4c48fa..e1f00e2 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
@@ -356,7 +356,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
 
     protected int getCellBufSize() {
       int kvBufSize = KEY_VALUE_LEN_SIZE + keyLength + valueLength;
-      if (includesTags() && tagsLength > 0) {
+      if (includesTags()) {
         kvBufSize += Bytes.SIZEOF_SHORT + tagsLength;
       }
       return kvBufSize;

http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
index 5596990..167a030 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
@@ -18,13 +18,11 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -592,152 +590,181 @@ public class TestKeyValue {
 
   @Test
   public void testKeyValueSerialization() throws Exception {
-    KeyValue[] keyValues = new KeyValue[] {
-        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
-            Bytes.toBytes("1")),
-        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
-            Bytes.toBytes("2")),
-        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
-            System.currentTimeMillis(), Bytes.toBytes("2"),
-            new Tag[] { new ArrayBackedTag((byte) 120, "tagA"),
-                new ArrayBackedTag((byte) 121, Bytes.toBytes("tagB")) }),
-        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
-            System.currentTimeMillis(), Bytes.toBytes("2"),
-            new Tag[] { new ArrayBackedTag((byte) 0, "tagA") }),
-        new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes(""),
-            Bytes.toBytes("1")) };
+    KeyValue kvA1 = new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
+        Bytes.toBytes("1"));
+    KeyValue kvA2 = new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
+        Bytes.toBytes("2"));
+    MockKeyValue mkvA1 = new MockKeyValue(kvA1);
+    MockKeyValue mkvA2 = new MockKeyValue(kvA2);
     ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    for (KeyValue kv : keyValues) {
-      DataOutputStream os = new DataOutputStream(byteArrayOutputStream);
-      ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(kv, true));
-      KeyValueUtil.oswrite(kv, os, true);
+    DataOutputStream os = new DataOutputStream(byteArrayOutputStream);
+    ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(mkvA1, true));
+    KeyValueUtil.oswrite(mkvA1, os, true);
+    ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(mkvA2, true));
+    KeyValueUtil.oswrite(mkvA2, os, true);
+    DataInputStream is = new DataInputStream(new ByteArrayInputStream(
+        byteArrayOutputStream.toByteArray()));
+    KeyValue deSerKV1 = KeyValueUtil.iscreate(is, true);
+    assertTrue(kvA1.equals(deSerKV1));
+    KeyValue deSerKV2 = KeyValueUtil.iscreate(is, true);
+    assertTrue(kvA2.equals(deSerKV2));
+  }
+
+  private static class MockKeyValue implements Cell {
+    private final KeyValue kv;
+
+    public MockKeyValue(KeyValue kv) {
+      this.kv = kv;
     }
-    DataInputStream is =
-        new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
-    for (int i = 0; i < keyValues.length; i++) {
-      LOG.info("Case#" + i + ": deserialize the kv: " + keyValues[i]);
-      KeyValue destKv = KeyValueUtil.createKeyValueFromInputStream(is, true);
-      assertEquals(keyValues[i], destKv);
-      assertArrayEquals(CellUtil.cloneValue(keyValues[i]), CellUtil.cloneValue(destKv));
-      assertArrayEquals(PrivateCellUtil.cloneTags(keyValues[i]), PrivateCellUtil.cloneTags(destKv));
+
+    /**
+     * This returns the offset where the tag actually starts.
+     */
+    @Override
+    public int getTagsOffset() {
+      return this.kv.getTagsOffset();
     }
-  }
 
-  private static class FailureCase {
-    byte[] buf;
-    int offset;
-    int length;
-    boolean withTags;
-    String expectedMessage;
-
-    public FailureCase(byte[] buf, int offset, int length, boolean withTags,
-        String expectedMessage) {
-      this.buf = buf;
-      this.offset = offset;
-      this.length = length;
-      this.withTags = withTags;
-      this.expectedMessage = expectedMessage;
+    /**
+     * used to achieve atomic operations in the memstore.
+     */
+    @Override
+    public long getSequenceId() {
+      return this.kv.getSequenceId();
     }
 
+    /**
+     * This returns the total length of the tag bytes
+     */
     @Override
-    public String toString() {
-      return "FailureCaseDetails: [buf=" + Bytes.toStringBinary(buf, offset, length) + ", offset="
-          + offset + ", " + "length=" + length + ", expectedMessage=" + expectedMessage
-          + ", withtags=" + withTags + "]";
+    public int getTagsLength() {
+      return this.kv.getTagsLength();
     }
 
-    public String getExpectedMessage() {
-      return this.expectedMessage + KeyValueUtil.bytesToHex(buf, offset, length);
+    /**
+     *
+     * @return Timestamp
+     */
+    @Override
+    public long getTimestamp() {
+      return this.kv.getTimestamp();
     }
-  }
 
-  @Test
-  public void testCheckKeyValueBytesFailureCase() throws Exception {
-    byte[][] inputs = new byte[][] { HConstants.EMPTY_BYTE_ARRAY, // case.0
-        Bytes.toBytesBinary("a"), // case.1
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01"), // case.2
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00"), // case.3
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01"), // case.4
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00"), // case.5
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00\\x01"), // case.6
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00\\x03ROW"), // case.7
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x01\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01"), // case.8
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\xFF"
-            + "\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\xFF\\x03"), // case.9
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x03"), // case.10
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04"), // case.11
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04VALUE"), // case.12
-    };
-    String[] outputs = new String[] { "Overflow when reading key length at position=0",
-        "Overflow when reading key length at position=0",
-        "Invalid key length in KeyValue. keyLength=1",
-        "Overflow when reading value length at position=4",
-        "Invalid value length in KeyValue, valueLength=1",
-        "Overflow when reading row length at position=8",
-        "Invalid row length in KeyValue, rowLength=1",
-        "Overflow when reading family length at position=13",
-        "Invalid family length in KeyValue, familyLength=1", "Timestamp cannot be negative, ts=-1",
-        "Invalid type in KeyValue, type=3", "Overflow when reading value part at position=25",
-        "Some redundant bytes in KeyValue's buffer, startOffset=26, endOffset=30", };
-    byte[][] withTagsInputs = new byte[][] {
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x01"), // case.13
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x01"), // case.14
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x04\\x00\\x03\\x00A"), // case.15
-        // case.16
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x0A\\x00\\x04\\x00TAG\\x00\\x04"
-            + "\\xFFT"),
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x0C\\x00\\x04\\x00TAG\\x00\\x05"
-            + "\\xF0COME\\x00"), // case.17
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x0C\\x00\\x04\\x00TAG\\x00\\x05"
-            + "\\xF0COME"), // case.18
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x00"), // case.19
-        Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
-            + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x00\\x1B\\x00\\x05\\x01TAG1\\x00\\x05"
-            + "\\x02TAG2\\x00\\x05\\x03TAG3\\x00\\x05\\x04TAG4"), // case.20
-    };
-    String[] withTagsOutputs = new String[] { "Overflow when reading tags length at position=26",
-        "Invalid tags length in KeyValue at position=26",
-        "Invalid tag length at position=28, tagLength=3",
-        "Invalid tag length at position=34, tagLength=4",
-        "Some redundant bytes in KeyValue's buffer, startOffset=41, endOffset=42", null, null,
-        null, };
-    assertEquals(inputs.length, outputs.length);
-    assertEquals(withTagsInputs.length, withTagsOutputs.length);
-
-    FailureCase[] cases = new FailureCase[inputs.length + withTagsInputs.length];
-    for (int i = 0; i < inputs.length; i++) {
-      cases[i] = new FailureCase(inputs[i], 0, inputs[i].length, false, outputs[i]);
+    /**
+     * @return KeyValue.TYPE byte representation
+     */
+    @Override
+    public byte getTypeByte() {
+      return this.kv.getTypeByte();
     }
-    for (int i = 0; i < withTagsInputs.length; i++) {
-      cases[inputs.length + i] =
-          new FailureCase(withTagsInputs[i], 0, withTagsInputs[i].length, true, withTagsOutputs[i]);
+
+    /**
+     * @return the backing array of the entire KeyValue (all KeyValue fields are
+     *         in a single array)
+     */
+    @Override
+    public byte[] getValueArray() {
+      return this.kv.getValueArray();
     }
 
-    for (int i = 0; i < cases.length; i++) {
-      FailureCase c = cases[i];
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      DataOutputStream os = new DataOutputStream(baos);
-      ByteBufferUtils.putInt(os, c.length);
-      os.write(c.buf, c.offset, c.length);
-      try {
-        KeyValueUtil.createKeyValueFromInputStream(
-          new DataInputStream(new ByteArrayInputStream(baos.toByteArray())), c.withTags);
-        if (c.expectedMessage != null) {
-          fail("Should fail when parse key value from an invalid bytes for case#" + i + ". " + c);
-        }
-      } catch (IOException e) {
-        assertEquals("Case#" + i + " failed," + c, c.getExpectedMessage(), e.getMessage());
-      }
+    /**
+     * @return the value offset
+     */
+    @Override
+    public int getValueOffset() {
+      return this.kv.getValueOffset();
+    }
+
+    /**
+     * @return Value length
+     */
+    @Override
+    public int getValueLength() {
+      return this.kv.getValueLength();
+    }
+
+    /**
+     * @return the backing array of the entire KeyValue (all KeyValue fields are
+     *         in a single array)
+     */
+    @Override
+    public byte[] getRowArray() {
+      return this.kv.getRowArray();
+    }
+
+    /**
+     * @return Row offset
+     */
+    @Override
+    public int getRowOffset() {
+      return this.kv.getRowOffset();
+    }
+
+    /**
+     * @return Row length
+     */
+    @Override
+    public short getRowLength() {
+      return this.kv.getRowLength();
+    }
+
+    /**
+     * @return the backing array of the entire KeyValue (all KeyValue fields are
+     *         in a single array)
+     */
+    @Override
+    public byte[] getFamilyArray() {
+      return this.kv.getFamilyArray();
+    }
+
+    /**
+     * @return Family offset
+     */
+    @Override
+    public int getFamilyOffset() {
+      return this.kv.getFamilyOffset();
+    }
+
+    /**
+     * @return Family length
+     */
+    @Override
+    public byte getFamilyLength() {
+      return this.kv.getFamilyLength();
+    }
+
+    /**
+     * @return the backing array of the entire KeyValue (all KeyValue fields are
+     *         in a single array)
+     */
+    @Override
+    public byte[] getQualifierArray() {
+      return this.kv.getQualifierArray();
+    }
+
+    /**
+     * @return Qualifier offset
+     */
+    @Override
+    public int getQualifierOffset() {
+      return this.kv.getQualifierOffset();
+    }
+
+    /**
+     * @return Qualifier length
+     */
+    @Override
+    public int getQualifierLength() {
+      return this.kv.getQualifierLength();
+    }
+
+    /**
+     * @return the backing array of the entire KeyValue (all KeyValue fields are
+     *         in a single array)
+     */
+    @Override
+    public byte[] getTagsArray() {
+      return this.kv.getTagsArray();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/362b5dd2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 4e5fad5..032dc5f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1710,6 +1710,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
 
   @Override
   public boolean hasReferences() {
+    List<HStoreFile> reloadedStoreFiles = null;
     // Grab the read lock here, because we need to ensure that: only when the atomic
     // replaceStoreFiles(..) finished, we can get all the complete store file list.
     this.lock.readLock().lock();