HBASE-7845 optimize hfile index key
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1460233 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
fc0443135f
commit
5f17a10648
|
@ -47,6 +47,9 @@ public final class HConstants {
|
|||
* Default block size for an HFile.
|
||||
*/
|
||||
public final static int DEFAULT_BLOCKSIZE = 64 * 1024;
|
||||
|
||||
/** Used as a magic return value while optimized index key feature enabled(HBASE-7845) */
|
||||
public final static int INDEX_KEY_MAGIC = -2;
|
||||
/*
|
||||
* Name of directory that holds recovered edits written by the wal log
|
||||
* splitting code, one per region
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.BufferOverflowException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
@ -2484,6 +2485,11 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
|
||||
return Arrays.copyOf(rightKey, rightKey.length);
|
||||
}
|
||||
|
||||
protected int compareRowid(byte[] left, int loffset, int llength,
|
||||
byte[] right, int roffset, int rlength) {
|
||||
return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
|
||||
|
@ -2679,6 +2685,89 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a faked byte array if possible. It aims to:
|
||||
* 1)reduce key length, which expects to reduce HFile index memory footprint
|
||||
* 2)replace TS field with LATEST_TIMESTAMP(to avoid seeking previous block)
|
||||
* see HBASE-7845 for more details
|
||||
* we need to ensure: leftKey < newKey <= rightKey
|
||||
* @param leftKey the previous block's real stop key usually
|
||||
* @param rightKey the current block's real start key usually
|
||||
* @return newKey: the newly generated faked key
|
||||
*/
|
||||
public byte[] getShortMidpointKey(final byte[] leftKey, final byte[] rightKey) {
|
||||
if (rightKey == null) {
|
||||
throw new IllegalArgumentException("rightKey can not be null");
|
||||
}
|
||||
if (leftKey == null) {
|
||||
return Arrays.copyOf(rightKey, rightKey.length);
|
||||
}
|
||||
if (compare(leftKey, rightKey) >= 0) {
|
||||
throw new IllegalArgumentException("Unexpected input, leftKey:" + Bytes.toString(leftKey)
|
||||
+ ", rightKey:" + Bytes.toString(rightKey));
|
||||
}
|
||||
|
||||
short leftRowLength = Bytes.toShort(leftKey, 0);
|
||||
short rightRowLength = Bytes.toShort(rightKey, 0);
|
||||
int leftCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + leftRowLength;
|
||||
int rightCommonLength = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE + rightRowLength;
|
||||
int leftCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + leftCommonLength;
|
||||
int rightCommonLengthWithTSAndType = TIMESTAMP_TYPE_SIZE + rightCommonLength;
|
||||
int leftColumnLength = leftKey.length - leftCommonLengthWithTSAndType;
|
||||
int rightColumnLength = rightKey.length - rightCommonLengthWithTSAndType;
|
||||
// rows are equal
|
||||
if (leftRowLength == rightRowLength && compareRows(leftKey, ROW_LENGTH_SIZE, leftRowLength,
|
||||
rightKey, ROW_LENGTH_SIZE, rightRowLength) == 0) {
|
||||
// Compare family & qualifier together.
|
||||
int comparison = Bytes.compareTo(leftKey, leftCommonLength, leftColumnLength, rightKey,
|
||||
rightCommonLength, rightColumnLength);
|
||||
// same with "row + family + qualifier", return rightKey directly
|
||||
if (comparison == 0) {
|
||||
return Arrays.copyOf(rightKey, rightKey.length);
|
||||
}
|
||||
// "family + qualifier" are different, generate a faked key per rightKey
|
||||
byte[] newKey = Arrays.copyOf(rightKey, rightKey.length);
|
||||
Bytes.putLong(newKey, rightKey.length - TIMESTAMP_TYPE_SIZE, HConstants.LATEST_TIMESTAMP);
|
||||
Bytes.putByte(newKey, rightKey.length - TYPE_SIZE, Type.Maximum.getCode());
|
||||
return newKey;
|
||||
}
|
||||
// rows are different
|
||||
short minLength = leftRowLength < rightRowLength ? leftRowLength : rightRowLength;
|
||||
short diffIdx = 0;
|
||||
while (diffIdx < minLength
|
||||
&& leftKey[ROW_LENGTH_SIZE + diffIdx] == rightKey[ROW_LENGTH_SIZE + diffIdx]) {
|
||||
diffIdx++;
|
||||
}
|
||||
if (diffIdx >= minLength) {
|
||||
// leftKey's row is prefix of rightKey's. we can optimize it in future
|
||||
return Arrays.copyOf(rightKey, rightKey.length);
|
||||
}
|
||||
int diffByte = leftKey[ROW_LENGTH_SIZE + diffIdx];
|
||||
if ((0xff & diffByte) < 0xff && (diffByte + 1) <
|
||||
(rightKey[ROW_LENGTH_SIZE + diffIdx] & 0xff)) {
|
||||
byte[] newRowKey = new byte[diffIdx + 1];
|
||||
System.arraycopy(leftKey, ROW_LENGTH_SIZE, newRowKey, 0, diffIdx);
|
||||
newRowKey[diffIdx] = (byte) (diffByte + 1);
|
||||
int rightFamilyLength = rightKey[rightCommonLength - 1];
|
||||
byte[] family = null;
|
||||
if (rightFamilyLength > 0) {
|
||||
family = new byte[rightFamilyLength];
|
||||
System.arraycopy(rightKey, rightCommonLength, family, 0, rightFamilyLength);
|
||||
}
|
||||
int rightQualifierLength = rightColumnLength - rightFamilyLength;
|
||||
byte[] qualifier = null;
|
||||
if (rightQualifierLength > 0) {
|
||||
qualifier = new byte[rightQualifierLength];
|
||||
System.arraycopy(rightKey, rightCommonLength + rightFamilyLength, qualifier, 0,
|
||||
rightQualifierLength);
|
||||
}
|
||||
return new KeyValue(newRowKey, null, null, HConstants.LATEST_TIMESTAMP,
|
||||
Type.Maximum).getKey();
|
||||
}
|
||||
// the following is optimizable in future
|
||||
return Arrays.copyOf(rightKey, rightKey.length);
|
||||
}
|
||||
|
||||
protected int compareColumns(
|
||||
byte [] left, int loffset, int llength, final int lfamilylength,
|
||||
byte [] right, int roffset, int rlength, final int rfamilylength) {
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
|
||||
|
@ -228,6 +229,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
|||
if (comp < 0) { // already too large, check previous
|
||||
if (previous.isValid()) {
|
||||
moveToPrevious();
|
||||
} else {
|
||||
return HConstants.INDEX_KEY_MAGIC; // using optimized index key
|
||||
}
|
||||
return 1;
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
|
@ -27,6 +28,7 @@ import junit.framework.TestCase;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.MetaComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -325,6 +327,7 @@ public class TestKeyValue extends TestCase {
|
|||
|
||||
private final byte[] family = Bytes.toBytes("family");
|
||||
private final byte[] qualA = Bytes.toBytes("qfA");
|
||||
private final byte[] qualB = Bytes.toBytes("qfB");
|
||||
|
||||
private void assertKVLess(KeyValue.KVComparator c,
|
||||
KeyValue less,
|
||||
|
@ -493,4 +496,69 @@ public class TestKeyValue extends TestCase {
|
|||
assertEquals(HConstants.LATEST_TIMESTAMP, time1);
|
||||
assertEquals(12345L, time2);
|
||||
}
|
||||
|
||||
/**
|
||||
* See HBASE-7845
|
||||
*/
|
||||
public void testGetShortMidpointKey() {
|
||||
final KeyComparator keyComparator = new KeyValue.KeyComparator();
|
||||
//verify that faked shorter rowkey could be generated
|
||||
long ts = 5;
|
||||
KeyValue kv1 = new KeyValue(Bytes.toBytes("the quick brown fox"), family, qualA, ts, Type.Put);
|
||||
KeyValue kv2 = new KeyValue(Bytes.toBytes("the who test text"), family, qualA, ts, Type.Put);
|
||||
byte[] newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compare(newKey, kv2.getKey()) < 0);
|
||||
short newRowLength = Bytes.toShort(newKey, 0);
|
||||
byte[] expectedArray = Bytes.toBytes("the r");
|
||||
Bytes.equals(newKey, KeyValue.ROW_LENGTH_SIZE, newRowLength, expectedArray, 0,
|
||||
expectedArray.length);
|
||||
|
||||
//verify: same with "row + family + qualifier", return rightKey directly
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, 5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, 0, Type.Put);
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compare(newKey, kv2.getKey()) == 0);
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, -5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, -10, Type.Put);
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compare(newKey, kv2.getKey()) == 0);
|
||||
|
||||
// verify: same with row, different with qualifier
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, 5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualB, 5, Type.Put);
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compare(newKey, kv2.getKey()) < 0);
|
||||
KeyValue newKeyValue = KeyValue.createKeyValueFromKey(newKey);
|
||||
assertTrue(Arrays.equals(newKeyValue.getFamily(),family));
|
||||
assertTrue(Arrays.equals(newKeyValue.getQualifier(),qualB));
|
||||
assertTrue(newKeyValue.getTimestamp() == HConstants.LATEST_TIMESTAMP);
|
||||
assertTrue(newKeyValue.getType() == Type.Maximum.getCode());
|
||||
|
||||
//verify root/metaKeyComparator's getShortMidpointKey output
|
||||
final KeyComparator rootKeyComparator = new KeyValue.RootKeyComparator();
|
||||
final KeyComparator metaKeyComparator = new KeyValue.MetaKeyComparator();
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase123"), family, qualA, 5, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbase234"), family, qualA, 0, Type.Put);
|
||||
newKey = rootKeyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(rootKeyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(rootKeyComparator.compare(newKey, kv2.getKey()) == 0);
|
||||
newKey = metaKeyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(metaKeyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(metaKeyComparator.compare(newKey, kv2.getKey()) == 0);
|
||||
|
||||
//verify common fix scenario
|
||||
kv1 = new KeyValue(Bytes.toBytes("ilovehbase"), family, qualA, ts, Type.Put);
|
||||
kv2 = new KeyValue(Bytes.toBytes("ilovehbaseandhdfs"), family, qualA, ts, Type.Put);
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), kv2.getKey()) < 0);
|
||||
newKey = keyComparator.getShortMidpointKey(kv1.getKey(), kv2.getKey());
|
||||
assertTrue(keyComparator.compare(kv1.getKey(), newKey) < 0);
|
||||
assertTrue(keyComparator.compare(newKey, kv2.getKey()) == 0);
|
||||
}
|
||||
}
|
|
@ -316,9 +316,10 @@ public class HFileBlockIndex {
|
|||
if (rootCount == 0)
|
||||
throw new IOException("HFile empty");
|
||||
|
||||
byte[] midKey = this.midKey.get();
|
||||
if (midKey != null)
|
||||
return midKey;
|
||||
byte[] targetMidKey = this.midKey.get();
|
||||
if (targetMidKey != null) {
|
||||
return targetMidKey;
|
||||
}
|
||||
|
||||
if (midLeafBlockOffset >= 0) {
|
||||
if (cachingBlockReader == null) {
|
||||
|
@ -339,14 +340,14 @@ public class HFileBlockIndex {
|
|||
int keyOffset = b.arrayOffset() +
|
||||
Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset +
|
||||
SECONDARY_INDEX_ENTRY_OVERHEAD;
|
||||
midKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen);
|
||||
targetMidKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen);
|
||||
} else {
|
||||
// The middle of the root-level index.
|
||||
midKey = blockKeys[rootCount / 2];
|
||||
targetMidKey = blockKeys[rootCount / 2];
|
||||
}
|
||||
|
||||
this.midKey.set(midKey);
|
||||
return midKey;
|
||||
this.midKey.set(targetMidKey);
|
||||
return targetMidKey;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -80,7 +80,10 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
/** Maximum minor version supported by this HFile format */
|
||||
// We went to version 2 when we moved to pb'ing fileinfo and the trailer on
|
||||
// the file. This version can read Writables version 1.
|
||||
static final int MAX_MINOR_VERSION = 2;
|
||||
static final int MAX_MINOR_VERSION = 3;
|
||||
|
||||
/** Minor versions starting with this number have faked index key */
|
||||
static final int MINOR_VERSION_WITH_FAKED_KEY = 3;
|
||||
|
||||
/**
|
||||
* Opens a HFile. You must load the index before you can use it by calling
|
||||
|
@ -458,7 +461,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
* doing the seek. If this is false, we are assuming we never go
|
||||
* back, otherwise the result is undefined.
|
||||
* @return -1 if the key is earlier than the first key of the file,
|
||||
* 0 if we are at the given key, and 1 if we are past the given key
|
||||
* 0 if we are at the given key, 1 if we are past the given key
|
||||
* -2 if the key is earlier than the first key of the file while
|
||||
* using a faked index key
|
||||
* @throws IOException
|
||||
*/
|
||||
protected int seekTo(byte[] key, int offset, int length, boolean rewind)
|
||||
|
@ -811,7 +816,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
* @param key the key to find
|
||||
* @param seekBefore find the key before the given key in case of exact
|
||||
* match.
|
||||
* @return 0 in case of an exact key match, 1 in case of an inexact match
|
||||
* @return 0 in case of an exact key match, 1 in case of an inexact match,
|
||||
* -2 in case of an inexact match and furthermore, the input key less
|
||||
* than the first key of current block(e.g. using a faked index key)
|
||||
*/
|
||||
private int blockSeek(byte[] key, int offset, int length,
|
||||
boolean seekBefore) {
|
||||
|
@ -866,12 +873,14 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
currMemstoreTSLen = memstoreTSLen;
|
||||
}
|
||||
return 0; // indicate exact match
|
||||
}
|
||||
|
||||
if (comp < 0) {
|
||||
} else if (comp < 0) {
|
||||
if (lastKeyValueSize > 0)
|
||||
blockBuffer.position(blockBuffer.position() - lastKeyValueSize);
|
||||
readKeyValueLen();
|
||||
if (lastKeyValueSize == -1 && blockBuffer.position() == 0
|
||||
&& this.reader.trailer.getMinorVersion() >= MINOR_VERSION_WITH_FAKED_KEY) {
|
||||
return HConstants.INDEX_KEY_MAGIC;
|
||||
}
|
||||
return 1;
|
||||
}
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
|
||||
|
@ -77,6 +78,9 @@ public class HFileWriterV2 extends AbstractHFileWriter {
|
|||
/** The offset of the last data block or 0 if the file is empty. */
|
||||
private long lastDataBlockOffset;
|
||||
|
||||
/** The last(stop) Key of the previous data block. */
|
||||
private byte[] lastKeyOfPreviousBlock = null;
|
||||
|
||||
/** Additional data items to be written to the "load-on-open" section. */
|
||||
private List<BlockWritable> additionalLoadOnOpenData =
|
||||
new ArrayList<BlockWritable>();
|
||||
|
@ -162,24 +166,37 @@ public class HFileWriterV2 extends AbstractHFileWriter {
|
|||
return;
|
||||
|
||||
long startTimeNs = System.nanoTime();
|
||||
|
||||
// Update the first data block offset for scanning.
|
||||
if (firstDataBlockOffset == -1) {
|
||||
firstDataBlockOffset = outputStream.getPos();
|
||||
}
|
||||
|
||||
// Update the last data block offset
|
||||
lastDataBlockOffset = outputStream.getPos();
|
||||
|
||||
fsBlockWriter.writeHeaderAndData(outputStream);
|
||||
|
||||
int onDiskSize = fsBlockWriter.getOnDiskSizeWithHeader();
|
||||
dataBlockIndexWriter.addEntry(firstKeyInBlock, lastDataBlockOffset,
|
||||
onDiskSize);
|
||||
// Generate a shorter faked key into index block. For example, consider a block boundary
|
||||
// between the keys "the quick brown fox" and "the who test text". We can use "the r" as the
|
||||
// key for the index block entry since it is > all entries in the previous block and <= all
|
||||
// entries in subsequent blocks.
|
||||
if (comparator instanceof KeyComparator) {
|
||||
byte[] fakeKey = ((KeyComparator) comparator).getShortMidpointKey(
|
||||
lastKeyOfPreviousBlock, firstKeyInBlock);
|
||||
if (comparator.compare(fakeKey, firstKeyInBlock) > 0) {
|
||||
throw new IOException("Unexpected getShortMidpointKey result, fakeKey:" + fakeKey
|
||||
+ ", firstKeyInBlock:" + firstKeyInBlock);
|
||||
}
|
||||
if (lastKeyOfPreviousBlock != null && comparator.compare(lastKeyOfPreviousBlock,
|
||||
fakeKey) >= 0) {
|
||||
throw new IOException("Unexpected getShortMidpointKey result, lastKeyOfPreviousBlock:" +
|
||||
Bytes.toString(lastKeyOfPreviousBlock) + ", fakeKey:" +
|
||||
Bytes.toString(fakeKey));
|
||||
}
|
||||
dataBlockIndexWriter.addEntry(fakeKey, lastDataBlockOffset,onDiskSize);
|
||||
} else {
|
||||
dataBlockIndexWriter.addEntry(firstKeyInBlock, lastDataBlockOffset,onDiskSize);
|
||||
}
|
||||
totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
|
||||
|
||||
HFile.offerWriteLatency(System.nanoTime() - startTimeNs);
|
||||
|
||||
if (cacheConf.shouldCacheDataOnWrite()) {
|
||||
doCacheOnWrite(lastDataBlockOffset);
|
||||
}
|
||||
|
@ -230,6 +247,10 @@ public class HFileWriterV2 extends AbstractHFileWriter {
|
|||
// This is where the next block begins.
|
||||
fsBlockWriter.startWriting(BlockType.DATA);
|
||||
firstKeyInBlock = null;
|
||||
if (lastKeyLength > 0) {
|
||||
lastKeyOfPreviousBlock = new byte[lastKeyLength];
|
||||
System.arraycopy(lastKeyBuffer, lastKeyOffset, lastKeyOfPreviousBlock, 0, lastKeyLength);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -464,4 +485,4 @@ public class HFileWriterV2 extends AbstractHFileWriter {
|
|||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
|
@ -220,6 +221,10 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
throws IOException {
|
||||
int result = s.seekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
|
||||
if(result < 0) {
|
||||
if (result == HConstants.INDEX_KEY_MAGIC) {
|
||||
// using faked key
|
||||
return true;
|
||||
}
|
||||
// Passed KV is smaller than first KV in file, work from start of file
|
||||
return s.seekTo();
|
||||
} else if(result > 0) {
|
||||
|
@ -236,6 +241,10 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
//This function is similar to seekAtOrAfter function
|
||||
int result = s.reseekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
|
||||
if (result <= 0) {
|
||||
if (result == HConstants.INDEX_KEY_MAGIC) {
|
||||
// using faked key
|
||||
return true;
|
||||
}
|
||||
// If up to now scanner is not seeked yet, this means passed KV is smaller
|
||||
// than first KV in file, and it is the first time we seek on this file.
|
||||
// So we also need to work from the start of file.
|
||||
|
@ -243,11 +252,10 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
return s.seekTo();
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
// passed KV is larger than current KV in file, if there is a next
|
||||
// it is after, if not then this scanner is done.
|
||||
return s.next();
|
||||
}
|
||||
// passed KV is larger than current KV in file, if there is a next
|
||||
// it is after, if not then this scanner is done.
|
||||
return s.next();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -102,19 +102,16 @@ public class TestHFileWriterV2 {
|
|||
.create();
|
||||
|
||||
Random rand = new Random(9713312); // Just a fixed seed.
|
||||
|
||||
List<byte[]> keys = new ArrayList<byte[]>();
|
||||
List<byte[]> values = new ArrayList<byte[]>();
|
||||
List<KeyValue> keyValues = new ArrayList<KeyValue>(entryCount);
|
||||
|
||||
for (int i = 0; i < entryCount; ++i) {
|
||||
byte[] keyBytes = randomOrderedKey(rand, i);
|
||||
|
||||
// A random-length random value.
|
||||
byte[] valueBytes = randomValue(rand);
|
||||
writer.append(keyBytes, valueBytes);
|
||||
|
||||
keys.add(keyBytes);
|
||||
values.add(valueBytes);
|
||||
KeyValue keyValue = new KeyValue(keyBytes, null, null, valueBytes);
|
||||
writer.append(keyValue);
|
||||
keyValues.add(keyValue);
|
||||
}
|
||||
|
||||
// Add in an arbitrary order. They will be sorted lexicographically by
|
||||
|
@ -207,8 +204,8 @@ public class TestHFileWriterV2 {
|
|||
}
|
||||
|
||||
// A brute-force check to see that all keys and values are correct.
|
||||
assertTrue(Bytes.compareTo(key, keys.get(entriesRead)) == 0);
|
||||
assertTrue(Bytes.compareTo(value, values.get(entriesRead)) == 0);
|
||||
assertTrue(Bytes.compareTo(key, keyValues.get(entriesRead).getKey()) == 0);
|
||||
assertTrue(Bytes.compareTo(value, keyValues.get(entriesRead).getValue()) == 0);
|
||||
|
||||
++entriesRead;
|
||||
}
|
||||
|
|
|
@ -251,13 +251,13 @@ public class TestBlocksRead extends HBaseTestCase {
|
|||
// The first 2 seeks is to find out col2. [HBASE-4443]
|
||||
// One additional seek for col3
|
||||
// So 3 seeks are needed.
|
||||
kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 3);
|
||||
kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 2);
|
||||
assertEquals(2, kvs.length);
|
||||
verifyData(kvs[0], "row", "col2", 2);
|
||||
verifyData(kvs[1], "row", "col3", 3);
|
||||
|
||||
// Expected block reads: 2. [HBASE-4443]
|
||||
kvs = getData(FAMILY, "row", Arrays.asList("col5"), 2);
|
||||
// Expected block reads: 1. [HBASE-4443]&[HBASE-7845]
|
||||
kvs = getData(FAMILY, "row", Arrays.asList("col5"), 1);
|
||||
assertEquals(1, kvs.length);
|
||||
verifyData(kvs[0], "row", "col5", 5);
|
||||
} finally {
|
||||
|
@ -334,7 +334,7 @@ public class TestBlocksRead extends HBaseTestCase {
|
|||
// For NONE Bloom filter: Expected blocks read: 3.
|
||||
kvs = getData(FAMILY, "row", "col1", 2, 3, 3);
|
||||
assertEquals(0, kvs.length);
|
||||
kvs = getData(FAMILY, "row", "col2", 3, 4, 4);
|
||||
kvs = getData(FAMILY, "row", "col2", 2, 3, 3);
|
||||
assertEquals(0, kvs.length);
|
||||
kvs = getData(FAMILY, "row", "col3", 2);
|
||||
assertEquals(0, kvs.length);
|
||||
|
|
Loading…
Reference in New Issue