HBASE-21401 Sanity check when constructing the KeyValue

This commit is contained in:
huzheng 2018-10-27 16:57:01 +08:00
parent ad819380c7
commit 12786f80c1
9 changed files with 293 additions and 189 deletions

View File

@ -251,6 +251,15 @@ 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. * Cannot rely on enum ordinals . They change if item is removed or moved.
* Do our own codes. * Do our own codes.
@ -331,7 +340,8 @@ public class KeyValue implements ExtendedCell, Cloneable {
* @param offset offset to start of the KeyValue * @param offset offset to start of the KeyValue
* @param length length of the KeyValue * @param length length of the KeyValue
*/ */
public KeyValue(final byte [] bytes, final int offset, final int length) { public KeyValue(final byte[] bytes, final int offset, final int length) {
KeyValueUtil.checkKeyValueBytes(bytes, offset, length, true);
this.bytes = bytes; this.bytes = bytes;
this.offset = offset; this.offset = offset;
this.length = length; this.length = length;

View File

@ -518,17 +518,145 @@ public class KeyValueUtil {
return (long) length + Bytes.SIZEOF_INT; 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;
}
static void checkKeyValueBytes(byte[] buf, int offset, int length, boolean withTags) {
int pos = offset, endOffset = offset + length;
// check the key
if (pos + Bytes.SIZEOF_INT > endOffset) {
throw new IllegalArgumentException(
"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 IllegalArgumentException(
"Invalid key length in KeyValue. keyLength=" + keyLen + bytesToHex(buf, offset, length));
}
// check the value
if (pos + Bytes.SIZEOF_INT > endOffset) {
throw new IllegalArgumentException("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 IllegalArgumentException("Invalid value length in KeyValue, valueLength=" + valLen
+ bytesToHex(buf, offset, length));
}
// check the row
if (pos + Bytes.SIZEOF_SHORT > endOffset) {
throw new IllegalArgumentException(
"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 IllegalArgumentException(
"Invalid row length in KeyValue, rowLength=" + rowLen + bytesToHex(buf, offset, length));
}
pos += rowLen;
// check the family
if (pos + Bytes.SIZEOF_BYTE > endOffset) {
throw new IllegalArgumentException("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 IllegalArgumentException("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 IllegalArgumentException("Invalid qualifier length in KeyValue, qualifierLen="
+ qualifierLen + bytesToHex(buf, offset, length));
}
pos += qualifierLen;
// check the timestamp
if (pos + Bytes.SIZEOF_LONG > endOffset) {
throw new IllegalArgumentException(
"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 IllegalArgumentException(
"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 IllegalArgumentException(
"Overflow when reading type at position=" + pos + bytesToHex(buf, offset, length));
}
byte type = buf[pos];
if (!Type.isValidType(type)) {
throw new IllegalArgumentException(
"Invalid type in KeyValue, type=" + type + bytesToHex(buf, offset, length));
}
pos += Bytes.SIZEOF_BYTE;
// check the value
if (pos + valLen > endOffset) {
throw new IllegalArgumentException(
"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 IllegalArgumentException("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 IllegalArgumentException("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 IllegalArgumentException("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 IllegalArgumentException(
"Invalid tag length at position=" + (pos - Tag.TAG_LENGTH_SIZE) + ", tagLength="
+ tagLen + bytesToHex(buf, offset, length));
}
pos += tagLen;
}
}
if (pos != endOffset) {
throw new IllegalArgumentException("Some redundant bytes in KeyValue's buffer, startOffset="
+ pos + ", endOffset=" + endOffset + bytesToHex(buf, offset, length));
}
}
/** /**
* Create a KeyValue reading from the raw InputStream. Named * Create a KeyValue reading from the raw InputStream. Named
* <code>iscreate</code> so doesn't clash with {@link #create(DataInput)} * <code>createKeyValueFromInputStream</code> so doesn't clash with {@link #create(DataInput)}
* * @param in inputStream to read.
* @param in
* @param withTags whether the keyvalue should include tags are not * @param withTags whether the keyvalue should include tags are not
* @return Created KeyValue OR if we find a length of zero, we will return * @return Created KeyValue OR if we find a length of zero, we will return null which can be
* null which can be useful marking a stream as done. * useful marking a stream as done.
* @throws IOException * @throws IOException
*/ */
public static KeyValue iscreate(final InputStream in, boolean withTags) throws IOException { public static KeyValue createKeyValueFromInputStream(InputStream in, boolean withTags)
throws IOException {
byte[] intBytes = new byte[Bytes.SIZEOF_INT]; byte[] intBytes = new byte[Bytes.SIZEOF_INT];
int bytesRead = 0; int bytesRead = 0;
while (bytesRead < intBytes.length) { while (bytesRead < intBytes.length) {
@ -541,14 +669,10 @@ public class KeyValueUtil {
} }
bytesRead += n; bytesRead += n;
} }
// TODO: perhaps some sanity check is needed here.
byte[] bytes = new byte[Bytes.toInt(intBytes)]; byte[] bytes = new byte[Bytes.toInt(intBytes)];
IOUtils.readFully(in, bytes, 0, bytes.length); IOUtils.readFully(in, bytes, 0, bytes.length);
if (withTags) { return withTags ? new KeyValue(bytes, 0, bytes.length)
return new KeyValue(bytes, 0, bytes.length); : new NoTagsKeyValue(bytes, 0, bytes.length);
} else {
return new NoTagsKeyValue(bytes, 0, bytes.length);
}
} }
/** /**

View File

@ -72,7 +72,7 @@ public class KeyValueCodec implements Codec {
@Override @Override
protected Cell parseCell() throws IOException { protected Cell parseCell() throws IOException {
// No tags here // No tags here
return KeyValueUtil.iscreate(in, false); return KeyValueUtil.createKeyValueFromInputStream(in, false);
} }
} }
@ -114,7 +114,6 @@ public class KeyValueCodec implements Codec {
// We know there is not going to be any tags. // We know there is not going to be any tags.
return new NoTagsByteBufferKeyValue(bb, pos, len); return new NoTagsByteBufferKeyValue(bb, pos, len);
} }
} }
/** /**

View File

@ -78,7 +78,7 @@ public class KeyValueCodecWithTags implements Codec {
@Override @Override
protected Cell parseCell() throws IOException { protected Cell parseCell() throws IOException {
// create KeyValue with tags // create KeyValue with tags
return KeyValueUtil.iscreate(in, true); return KeyValueUtil.createKeyValueFromInputStream(in, true);
} }
} }

View File

@ -18,11 +18,13 @@
*/ */
package org.apache.hadoop.hbase; package org.apache.hadoop.hbase;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
@ -590,181 +592,152 @@ public class TestKeyValue {
@Test @Test
public void testKeyValueSerialization() throws Exception { public void testKeyValueSerialization() throws Exception {
KeyValue kvA1 = new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"), KeyValue[] keyValues = new KeyValue[] {
Bytes.toBytes("1")); new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
KeyValue kvA2 = new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"), Bytes.toBytes("1")),
Bytes.toBytes("2")); new KeyValue(Bytes.toBytes("key"), Bytes.toBytes("cf"), Bytes.toBytes("qualA"),
MockKeyValue mkvA1 = new MockKeyValue(kvA1); Bytes.toBytes("2")),
MockKeyValue mkvA2 = new MockKeyValue(kvA2); 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")) };
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
DataOutputStream os = new DataOutputStream(byteArrayOutputStream); for (KeyValue kv : keyValues) {
ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(mkvA1, true)); DataOutputStream os = new DataOutputStream(byteArrayOutputStream);
KeyValueUtil.oswrite(mkvA1, os, true); ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(kv, true));
ByteBufferUtils.putInt(os, KeyValueUtil.getSerializedSize(mkvA2, true)); KeyValueUtil.oswrite(kv, os, true);
KeyValueUtil.oswrite(mkvA2, os, true); }
DataInputStream is = new DataInputStream(new ByteArrayInputStream( DataInputStream is =
byteArrayOutputStream.toByteArray())); new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
KeyValue deSerKV1 = KeyValueUtil.iscreate(is, true); for (int i = 0; i < keyValues.length; i++) {
assertTrue(kvA1.equals(deSerKV1)); LOG.info("Case#" + i + ": deserialize the kv: " + keyValues[i]);
KeyValue deSerKV2 = KeyValueUtil.iscreate(is, true); KeyValue destKv = KeyValueUtil.createKeyValueFromInputStream(is, true);
assertTrue(kvA2.equals(deSerKV2)); assertEquals(keyValues[i], destKv);
assertArrayEquals(CellUtil.cloneValue(keyValues[i]), CellUtil.cloneValue(destKv));
assertArrayEquals(PrivateCellUtil.cloneTags(keyValues[i]), PrivateCellUtil.cloneTags(destKv));
}
} }
private static class MockKeyValue implements Cell { private static class FailureCase {
private final KeyValue kv; byte[] buf;
int offset;
int length;
boolean withTags;
String expectedMessage;
public MockKeyValue(KeyValue kv) { public FailureCase(byte[] buf, int offset, int length, boolean withTags,
this.kv = kv; String expectedMessage) {
this.buf = buf;
this.offset = offset;
this.length = length;
this.withTags = withTags;
this.expectedMessage = expectedMessage;
} }
/**
* This returns the offset where the tag actually starts.
*/
@Override @Override
public int getTagsOffset() { public String toString() {
return this.kv.getTagsOffset(); return "FailureCaseDetails: [buf=" + Bytes.toStringBinary(buf, offset, length) + ", offset="
+ offset + ", " + "length=" + length + ", expectedMessage=" + expectedMessage
+ ", withtags=" + withTags + "]";
} }
/** public String getExpectedMessage() {
* used to achieve atomic operations in the memstore. return this.expectedMessage + KeyValueUtil.bytesToHex(buf, offset, length);
*/ }
@Override }
public long getSequenceId() {
return this.kv.getSequenceId(); @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",
"Invalid tags length in KeyValue at position=26", };
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]);
}
for (int i = 0; i < withTagsInputs.length; i++) {
cases[inputs.length + i] =
new FailureCase(withTagsInputs[i], 0, withTagsInputs[i].length, true, withTagsOutputs[i]);
} }
/** for (int i = 0; i < cases.length; i++) {
* This returns the total length of the tag bytes FailureCase c = cases[i];
*/ ByteArrayOutputStream baos = new ByteArrayOutputStream();
@Override DataOutputStream os = new DataOutputStream(baos);
public int getTagsLength() { ByteBufferUtils.putInt(os, c.length);
return this.kv.getTagsLength(); os.write(c.buf, c.offset, c.length);
} try {
KeyValueUtil.createKeyValueFromInputStream(
/** new DataInputStream(new ByteArrayInputStream(baos.toByteArray())), c.withTags);
* if (c.expectedMessage != null) {
* @return Timestamp fail("Should fail when parse key value from an invalid bytes for case#" + i + ". " + c);
*/ }
@Override } catch (IllegalArgumentException e) {
public long getTimestamp() { assertEquals("Case#" + i + " failed," + c, c.getExpectedMessage(), e.getMessage());
return this.kv.getTimestamp(); }
}
/**
* @return KeyValue.TYPE byte representation
*/
@Override
public byte getTypeByte() {
return this.kv.getTypeByte();
}
/**
* @return the backing array of the entire KeyValue (all KeyValue fields are
* in a single array)
*/
@Override
public byte[] getValueArray() {
return this.kv.getValueArray();
}
/**
* @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();
} }
} }
} }

View File

@ -1710,7 +1710,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@Override @Override
public boolean hasReferences() { public boolean hasReferences() {
List<HStoreFile> reloadedStoreFiles = null;
// Grab the read lock here, because we need to ensure that: only when the atomic // 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. // replaceStoreFiles(..) finished, we can get all the complete store file list.
this.lock.readLock().lock(); this.lock.readLock().lock();

View File

@ -332,7 +332,7 @@ public class TestDataBlockEncoders {
byte[] family = new byte[0]; byte[] family = new byte[0];
byte[] qualifier = new byte[0]; byte[] qualifier = new byte[0];
byte[] value = new byte[0]; byte[] value = new byte[0];
KeyValue expectedKV = new KeyValue(row, family, qualifier, -1L, Type.Put, value); KeyValue expectedKV = new KeyValue(row, family, qualifier, 1L, Type.Put, value);
kvList.add(expectedKV); kvList.add(expectedKV);
DataBlockEncoding encoding = DataBlockEncoding.ROW_INDEX_V1; DataBlockEncoding encoding = DataBlockEncoding.ROW_INDEX_V1;
DataBlockEncoder encoder = encoding.getEncoder(); DataBlockEncoder encoder = encoding.getEncoder();

View File

@ -392,11 +392,11 @@ public class TestCacheOnWrite {
tags[0] = t; tags[0] = t;
kv = kv =
new KeyValue(row, 0, row.length, cf, 0, cf.length, qualifier, 0, qualifier.length, new KeyValue(row, 0, row.length, cf, 0, cf.length, qualifier, 0, qualifier.length,
rand.nextLong(), generateKeyType(rand), value, 0, value.length, tagList); Math.abs(rand.nextLong()), generateKeyType(rand), value, 0, value.length, tagList);
} else { } else {
kv = kv =
new KeyValue(row, 0, row.length, cf, 0, cf.length, qualifier, 0, qualifier.length, new KeyValue(row, 0, row.length, cf, 0, cf.length, qualifier, 0, qualifier.length,
rand.nextLong(), generateKeyType(rand), value, 0, value.length); Math.abs(rand.nextLong()), generateKeyType(rand), value, 0, value.length);
} }
sfw.append(kv); sfw.append(kv);
} }

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader; import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer; import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.IOTests;
@ -200,15 +199,15 @@ public class TestHFileSeek extends TestCase {
timer.start(); timer.start();
for (int i = 0; i < options.seekCount; ++i) { for (int i = 0; i < options.seekCount; ++i) {
kSampler.next(key); kSampler.next(key);
byte [] k = new byte [key.getLength()]; byte[] k = new byte[key.getLength()];
System.arraycopy(key.getBytes(), 0, k, 0, key.getLength()); System.arraycopy(key.getBytes(), 0, k, 0, key.getLength());
if (scanner.seekTo(KeyValueUtil.createKeyValueFromKey(k)) >= 0) { KeyValue kv = new KeyValue(k, CF, QUAL);
if (scanner.seekTo(kv) >= 0) {
ByteBuffer bbkey = ByteBuffer.wrap(((KeyValue) scanner.getKey()).getKey()); ByteBuffer bbkey = ByteBuffer.wrap(((KeyValue) scanner.getKey()).getKey());
ByteBuffer bbval = scanner.getValue(); ByteBuffer bbval = scanner.getValue();
totalBytes += bbkey.limit(); totalBytes += bbkey.limit();
totalBytes += bbval.limit(); totalBytes += bbval.limit();
} } else {
else {
++miss; ++miss;
} }
} }