HBASE-10531-Revisit how the key byte[] is passed to HFileScanner.seekTo and reseekTo (Ram)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1583031 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
ramkrishna 2014-03-29 17:18:56 +00:00
parent 4fa7db7304
commit ea78f39854
30 changed files with 1540 additions and 570 deletions

View File

@ -45,54 +45,85 @@ public class CellComparator implements Comparator<Cell>, Serializable{
@Override
public int compare(Cell a, Cell b) {
return compareStatic(a, b);
return compareStatic(a, b, false);
}
public static int compareStatic(Cell a, Cell b) {
//row
int c = Bytes.compareTo(
a.getRowArray(), a.getRowOffset(), a.getRowLength(),
b.getRowArray(), b.getRowOffset(), b.getRowLength());
public static int compareStatic(Cell a, Cell b, boolean onlyKey) {
// row
int c = compareRows(a, b);
if (c != 0) return c;
// If the column is not specified, the "minimum" key type appears the
// latest in the sorted order, regardless of the timestamp. This is used
// for specifying the last key/value in a given row, because there is no
// "lexicographically last column" (it would be infinitely long). The
// "maximum" key type does not need this behavior.
if (a.getFamilyLength() == 0 && a.getTypeByte() == Type.Minimum.getCode()) {
// a is "bigger", i.e. it appears later in the sorted order
return 1;
c = compareWithoutRow(a, b);
if(c != 0) return c;
if (!onlyKey) {
// Negate following comparisons so later edits show up first
// compare log replay tag value if there is any
// when either keyvalue tagged with log replay sequence number, we need to compare them:
// 1) when both keyvalues have the tag, then use the tag values for comparison
// 2) when one has and the other doesn't have, the one without the log
// replay tag wins because
// it means the edit isn't from recovery but new one coming from clients during recovery
// 3) when both doesn't have, then skip to the next mvcc comparison
long leftChangeSeqNum = getReplaySeqNum(a);
long RightChangeSeqNum = getReplaySeqNum(b);
if (leftChangeSeqNum != Long.MAX_VALUE || RightChangeSeqNum != Long.MAX_VALUE) {
return Longs.compare(RightChangeSeqNum, leftChangeSeqNum);
}
// mvccVersion: later sorts first
return Longs.compare(b.getMvccVersion(), a.getMvccVersion());
} else {
return c;
}
if (b.getFamilyLength() == 0 && b.getTypeByte() == Type.Minimum.getCode()) {
return -1;
}
//family
c = Bytes.compareTo(
a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(),
b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength());
if (c != 0) return c;
//qualifier
c = Bytes.compareTo(
a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(),
b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength());
if (c != 0) return c;
//timestamp: later sorts first
c = Longs.compare(b.getTimestamp(), a.getTimestamp());
if (c != 0) return c;
//type
c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
if (c != 0) return c;
//mvccVersion: later sorts first
return Longs.compare(b.getMvccVersion(), a.getMvccVersion());
}
/**
* Return replay log sequence number for the cell
*
* @param c
* @return Long.MAX_VALUE if there is no LOG_REPLAY_TAG
*/
private static long getReplaySeqNum(final Cell c) {
Tag tag = Tag.getTag(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength(),
TagType.LOG_REPLAY_TAG_TYPE);
if (tag != null) {
return Bytes.toLong(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
}
return Long.MAX_VALUE;
}
public static int findCommonPrefixInRowPart(Cell left, Cell right, int rowCommonPrefix) {
return findCommonPrefix(left.getRowArray(), right.getRowArray(), left.getRowLength()
- rowCommonPrefix, right.getRowLength() - rowCommonPrefix, left.getRowOffset()
+ rowCommonPrefix, right.getRowOffset() + rowCommonPrefix);
}
private static int findCommonPrefix(byte[] left, byte[] right, int leftLength, int rightLength,
int leftOffset, int rightOffset) {
int length = Math.min(leftLength, rightLength);
int result = 0;
while (result < length && left[leftOffset + result] == right[rightOffset + result]) {
result++;
}
return result;
}
public static int findCommonPrefixInFamilyPart(Cell left, Cell right, int familyCommonPrefix) {
return findCommonPrefix(left.getFamilyArray(), right.getFamilyArray(), left.getFamilyLength()
- familyCommonPrefix, right.getFamilyLength() - familyCommonPrefix, left.getFamilyOffset()
+ familyCommonPrefix, right.getFamilyOffset() + familyCommonPrefix);
}
public static int findCommonPrefixInQualifierPart(Cell left, Cell right,
int qualifierCommonPrefix) {
return findCommonPrefix(left.getQualifierArray(), right.getQualifierArray(),
left.getQualifierLength() - qualifierCommonPrefix, right.getQualifierLength()
- qualifierCommonPrefix, left.getQualifierOffset() + qualifierCommonPrefix,
right.getQualifierOffset() + qualifierCommonPrefix);
}
/**************** equals ****************************/
@ -130,6 +161,88 @@ public class CellComparator implements Comparator<Cell>, Serializable{
return a.getTypeByte() == b.getTypeByte();
}
public static int compareColumns(final Cell left, final Cell right) {
int lfoffset = left.getFamilyOffset();
int rfoffset = right.getFamilyOffset();
int lclength = left.getQualifierLength();
int rclength = right.getQualifierLength();
int lfamilylength = left.getFamilyLength();
int rfamilylength = right.getFamilyLength();
int diff = compare(left.getFamilyArray(), lfoffset, lfamilylength, right.getFamilyArray(),
rfoffset, rfamilylength);
if (diff != 0) {
return diff;
} else {
return compare(left.getQualifierArray(), left.getQualifierOffset(), lclength,
right.getQualifierArray(), right.getQualifierOffset(), rclength);
}
}
public static int compareFamilies(Cell left, Cell right) {
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
}
public static int compareQualifiers(Cell left, Cell right) {
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
right.getQualifierLength());
}
public int compareFlatKey(Cell left, Cell right) {
int compare = compareRows(left, right);
if (compare != 0) {
return compare;
}
return compareWithoutRow(left, right);
}
public static int compareRows(final Cell left, final Cell right) {
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
right.getRowArray(), right.getRowOffset(), right.getRowLength());
}
public static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
int rlength) {
return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
}
public static int compareWithoutRow(final Cell leftCell, final Cell rightCell) {
if (leftCell.getFamilyLength() + leftCell.getQualifierLength() == 0
&& leftCell.getTypeByte() == Type.Minimum.getCode()) {
// left is "bigger", i.e. it appears later in the sorted order
return 1;
}
if (rightCell.getFamilyLength() + rightCell.getQualifierLength() == 0
&& rightCell.getTypeByte() == Type.Minimum.getCode()) {
return -1;
}
boolean sameFamilySize = (leftCell.getFamilyLength() == rightCell.getFamilyLength());
if (!sameFamilySize) {
// comparing column family is enough.
return Bytes.compareTo(leftCell.getFamilyArray(), leftCell.getFamilyOffset(),
leftCell.getFamilyLength(), rightCell.getFamilyArray(), rightCell.getFamilyOffset(),
rightCell.getFamilyLength());
}
int diff = compareColumns(leftCell, rightCell);
if (diff != 0) return diff;
diff = compareTimestamps(leftCell, rightCell);
if (diff != 0) return diff;
// Compare types. Let the delete types sort ahead of puts; i.e. types
// of higher numbers sort before those of lesser numbers. Maximum (255)
// appears ahead of everything, and minimum (0) appears after
// everything.
return (0xff & rightCell.getTypeByte()) - (0xff & leftCell.getTypeByte());
}
public static int compareTimestamps(final Cell left, final Cell right) {
long ltimestamp = left.getTimestamp();
long rtimestamp = right.getTimestamp();
return compareTimestamps(ltimestamp, rtimestamp);
}
/********************* hashCode ************************/
@ -172,39 +285,36 @@ public class CellComparator implements Comparator<Cell>, Serializable{
}
/***************** special cases ****************************/
/*********************common prefixes*************************/
/**
* special case for KeyValue.equals
*/
private static int compareStaticIgnoreMvccVersion(Cell a, Cell b) {
//row
int c = Bytes.compareTo(
a.getRowArray(), a.getRowOffset(), a.getRowLength(),
b.getRowArray(), b.getRowOffset(), b.getRowLength());
if (c != 0) return c;
//family
c = Bytes.compareTo(
a.getFamilyArray(), a.getFamilyOffset(), a.getFamilyLength(),
b.getFamilyArray(), b.getFamilyOffset(), b.getFamilyLength());
if (c != 0) return c;
//qualifier
c = Bytes.compareTo(
a.getQualifierArray(), a.getQualifierOffset(), a.getQualifierLength(),
b.getQualifierArray(), b.getQualifierOffset(), b.getQualifierLength());
if (c != 0) return c;
//timestamp: later sorts first
c = Longs.compare(b.getTimestamp(), a.getTimestamp());
if (c != 0) return c;
//type
c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
return c;
private static int compare(byte[] left, int leftOffset, int leftLength, byte[] right,
int rightOffset, int rightLength) {
return Bytes.compareTo(left, leftOffset, leftLength, right, rightOffset, rightLength);
}
public static int compareCommonRowPrefix(Cell left, Cell right, int rowCommonPrefix) {
return compare(left.getRowArray(), left.getRowOffset() + rowCommonPrefix, left.getRowLength()
- rowCommonPrefix, right.getRowArray(), right.getRowOffset() + rowCommonPrefix,
right.getRowLength() - rowCommonPrefix);
}
public static int compareCommonFamilyPrefix(Cell left, Cell right,
int familyCommonPrefix) {
return compare(left.getFamilyArray(), left.getFamilyOffset() + familyCommonPrefix,
left.getFamilyLength() - familyCommonPrefix, right.getFamilyArray(),
right.getFamilyOffset() + familyCommonPrefix,
right.getFamilyLength() - familyCommonPrefix);
}
public static int compareCommonQualifierPrefix(Cell left, Cell right,
int qualCommonPrefix) {
return compare(left.getQualifierArray(), left.getQualifierOffset() + qualCommonPrefix,
left.getQualifierLength() - qualCommonPrefix, right.getQualifierArray(),
right.getQualifierOffset() + qualCommonPrefix, right.getQualifierLength()
- qualCommonPrefix);
}
/***************** special cases ****************************/
/**
* special case for KeyValue.equals
*/
@ -212,4 +322,35 @@ public class CellComparator implements Comparator<Cell>, Serializable{
return 0 == compareStaticIgnoreMvccVersion(a, b);
}
private static int compareStaticIgnoreMvccVersion(Cell a, Cell b) {
// row
int c = compareRows(a, b);
if (c != 0) return c;
// family
c = compareColumns(a, b);
if (c != 0) return c;
// timestamp: later sorts first
c = compareTimestamps(a, b);
if (c != 0) return c;
//type
c = (0xff & b.getTypeByte()) - (0xff & a.getTypeByte());
return c;
}
private static int compareTimestamps(final long ltimestamp, final long rtimestamp) {
// The below older timestamps sorting ahead of newer timestamps looks
// wrong but it is intentional. This way, newer timestamps are first
// found when we iterate over a memstore and newer versions are the
// first we trip over when reading from a store file.
if (ltimestamp < rtimestamp) {
return 1;
} else if (ltimestamp > rtimestamp) {
return -1;
}
return 0;
}
}

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.RawComparator;
import com.google.common.primitives.Longs;
import com.google.common.annotations.VisibleForTesting;
/**
* An HBase Key/Value. This is the fundamental HBase Type.
@ -1838,6 +1838,20 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* Compare key portion of a {@link KeyValue} for keys in <code>hbase:meta</code>
* table.
*/
@Override
public int compare(final Cell left, final Cell right) {
int c = compareRowKey(left, right);
if (c != 0) {
return c;
}
return CellComparator.compareWithoutRow(left, right);
}
@Override
public int compareOnlyKeyPortion(Cell left, Cell right) {
return compare(left, right);
}
@Override
public int compareRows(byte [] left, int loffset, int llength,
byte [] right, int roffset, int rlength) {
@ -1952,9 +1966,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* @return 0 if equal, <0 if left smaller, >0 if right smaller
*/
protected int compareRowKey(final Cell left, final Cell right) {
return Bytes.compareTo(
left.getRowArray(), left.getRowOffset(), left.getRowLength(),
right.getRowArray(), right.getRowOffset(), right.getRowLength());
return CellComparator.compareRows(left, right);
}
/**
@ -1990,109 +2002,22 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return compareFlatKey(left, 0, left.length, right, 0, right.length);
}
public int compareOnlyKeyPortion(Cell left, Cell right) {
return CellComparator.compareStatic(left, right, true);
}
/**
* Compares the Key of a cell -- with fields being more significant in this order:
* rowkey, colfam/qual, timestamp, type, mvcc
*/
@Override
public int compare(final Cell left, final Cell right) {
// compare row
int compare = compareRowKey(left, right);
if (compare != 0) {
return compare;
}
// compare vs minimum
byte ltype = left.getTypeByte();
byte rtype = right.getTypeByte();
// If the column is not specified, the "minimum" key type appears the
// latest in the sorted order, regardless of the timestamp. This is used
// for specifying the last key/value in a given row, because there is no
// "lexicographically last column" (it would be infinitely long). The
// "maximum" key type does not need this behavior.
int lcfqLen = left.getFamilyLength() + left.getQualifierLength() ;
int rcfqLen = right.getFamilyLength() + right.getQualifierLength() ;
if (lcfqLen == 0 && ltype == Type.Minimum.getCode()) {
// left is "bigger", i.e. it appears later in the sorted order
return 1;
}
if (rcfqLen == 0 && rtype == Type.Minimum.getCode()) {
return -1;
}
// compare col family / col fam + qual
// If left family size is not equal to right family size, we need not
// compare the qualifiers.
compare = Bytes.compareTo(
left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
if (compare != 0) {
return compare;
}
// Compare qualifier
compare = Bytes.compareTo(
left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
if (compare!= 0) {
return compare;
}
// compare timestamp
long ltimestamp = left.getTimestamp();
long rtimestamp = right.getTimestamp();
compare = compareTimestamps(ltimestamp, rtimestamp);
if (compare != 0) {
return compare;
}
// Compare types. Let the delete types sort ahead of puts; i.e. types
// of higher numbers sort before those of lesser numbers. Maximum (255)
// appears ahead of everything, and minimum (0) appears after
// everything.
compare = (0xff & rtype) - (0xff & ltype);
if (compare != 0) {
return compare;
}
// Negate following comparisons so later edits show up first
// compare log replay tag value if there is any
// when either keyvalue tagged with log replay sequence number, we need to compare them:
// 1) when both keyvalues have the tag, then use the tag values for comparison
// 2) when one has and the other doesn't have, the one without the log replay tag wins because
// it means the edit isn't from recovery but new one coming from clients during recovery
// 3) when both doesn't have, then skip to the next mvcc comparison
long leftChangeSeqNum = getReplaySeqNum(left);
long RightChangeSeqNum = getReplaySeqNum(right);
if (leftChangeSeqNum != Long.MAX_VALUE || RightChangeSeqNum != Long.MAX_VALUE) {
return Longs.compare(RightChangeSeqNum, leftChangeSeqNum);
}
// compare Mvcc Version
return Longs.compare(right.getMvccVersion(), left.getMvccVersion());
int compare = CellComparator.compareStatic(left, right, false);
return compare;
}
/**
* Return replay log sequence number for the cell
* @param c
* @return Long.MAX_VALUE if there is no LOG_REPLAY_TAG
*/
private long getReplaySeqNum(final Cell c) {
Tag tag = Tag.getTag(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength(),
TagType.LOG_REPLAY_TAG_TYPE);
if(tag != null) {
return Bytes.toLong(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
}
return Long.MAX_VALUE;
}
public int compareTimestamps(final KeyValue left, final KeyValue right) {
// Compare timestamps
long ltimestamp = left.getTimestamp(left.getKeyLength());
long rtimestamp = right.getTimestamp(right.getKeyLength());
return compareTimestamps(ltimestamp, rtimestamp);
public int compareTimestamps(final Cell left, final Cell right) {
return CellComparator.compareTimestamps(left, right);
}
/**
@ -2100,7 +2025,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* @param right
* @return Result comparing rows.
*/
public int compareRows(final KeyValue left, final KeyValue right) {
public int compareRows(final Cell left, final Cell right) {
return compareRows(left.getRowArray(),left.getRowOffset(), left.getRowLength(),
right.getRowArray(), right.getRowOffset(), right.getRowLength());
}
@ -2120,17 +2045,9 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
}
int compareColumns(final KeyValue left, final short lrowlength,
final KeyValue right, final short rrowlength) {
int lfoffset = left.getFamilyOffset(lrowlength);
int rfoffset = right.getFamilyOffset(rrowlength);
int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
int lfamilylength = left.getFamilyLength(lfoffset);
int rfamilylength = right.getFamilyLength(rfoffset);
return compareColumns(left.getBuffer(), lfoffset,
lclength, lfamilylength,
right.getBuffer(), rfoffset, rclength, rfamilylength);
int compareColumns(final Cell left, final short lrowlength, final Cell right,
final short rrowlength) {
return CellComparator.compareColumns(left, right);
}
protected int compareColumns(
@ -2297,20 +2214,31 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return (0xff & rtype) - (0xff & ltype);
}
protected int compareFamilies(final byte[] left, final int loffset, final int lfamilylength,
final byte[] right, final int roffset, final int rfamilylength) {
int diff = Bytes.compareTo(left, loffset, lfamilylength, right, roffset, rfamilylength);
return diff;
}
protected int compareColumns(final byte[] left, final int loffset, final int lquallength,
final byte[] right, final int roffset, final int rquallength) {
int diff = Bytes.compareTo(left, loffset, lquallength, right, roffset, rquallength);
return diff;
}
/**
* Compares the row and column of two keyvalues for equality
* @param left
* @param right
* @return True if same row and column.
*/
public boolean matchingRowColumn(final KeyValue left,
final KeyValue right) {
public boolean matchingRowColumn(final Cell left,
final Cell right) {
short lrowlength = left.getRowLength();
short rrowlength = right.getRowLength();
// TsOffset = end of column data. just comparing Row+CF length of each
if ((left.getTimestampOffset() - left.getOffset()) !=
(right.getTimestampOffset() - right.getOffset())) {
if ((left.getRowLength() + left.getFamilyLength() + left.getQualifierLength()) != (right
.getRowLength() + right.getFamilyLength() + right.getQualifierLength())) {
return false;
}
@ -2318,15 +2246,21 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return false;
}
int lfoffset = left.getFamilyOffset(lrowlength);
int rfoffset = right.getFamilyOffset(rrowlength);
int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
int lfamilylength = left.getFamilyLength(lfoffset);
int rfamilylength = right.getFamilyLength(rfoffset);
int ccRes = compareColumns(left.getBuffer(), lfoffset, lclength, lfamilylength,
right.getBuffer(), rfoffset, rclength, rfamilylength);
return ccRes == 0;
int lfoffset = left.getFamilyOffset();
int rfoffset = right.getFamilyOffset();
int lclength = left.getQualifierLength();
int rclength = right.getQualifierLength();
int lfamilylength = left.getFamilyLength();
int rfamilylength = right.getFamilyLength();
int diff = compareFamilies(left.getFamilyArray(), lfoffset, lfamilylength,
right.getFamilyArray(), rfoffset, rfamilylength);
if (diff != 0) {
return false;
} else {
diff = compareColumns(left.getQualifierArray(), left.getQualifierOffset(), lclength,
right.getQualifierArray(), right.getQualifierOffset(), rclength);
return diff == 0;
}
}
/**
@ -2335,7 +2269,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* @param right
* @return True if rows match.
*/
public boolean matchingRows(final KeyValue left, final KeyValue right) {
public boolean matchingRows(final Cell left, final Cell right) {
short lrowlength = left.getRowLength();
short rrowlength = right.getRowLength();
return matchingRows(left, lrowlength, right, rrowlength);
@ -2348,8 +2282,8 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
* @param rrowlength
* @return True if rows match.
*/
private boolean matchingRows(final KeyValue left, final short lrowlength,
final KeyValue right, final short rrowlength) {
private boolean matchingRows(final Cell left, final short lrowlength,
final Cell right, final short rrowlength) {
return lrowlength == rrowlength &&
matchingRows(left.getRowArray(), left.getRowOffset(), lrowlength,
right.getRowArray(), right.getRowOffset(), rrowlength);
@ -2929,6 +2863,37 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
return Bytes.BYTES_RAWCOMPARATOR.compare(left, loffset, llength, right, roffset, rlength);
}
@Override
public int compare(Cell left, Cell right) {
return compareOnlyKeyPortion(left, right);
}
@VisibleForTesting
public int compareOnlyKeyPortion(Cell left, Cell right) {
int c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getRowArray(), left.getRowOffset(),
left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
if (c != 0) {
return c;
}
c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getFamilyArray(), left.getFamilyOffset(),
left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
right.getFamilyLength());
if (c != 0) {
return c;
}
c = Bytes.BYTES_RAWCOMPARATOR.compare(left.getQualifierArray(), left.getQualifierOffset(),
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
right.getQualifierLength());
if (c != 0) {
return c;
}
c = compareTimestamps(left.getTimestamp(), right.getTimestamp());
if (c != 0) {
return c;
}
return (0xff & left.getTypeByte()) - (0xff & right.getTypeByte());
}
public byte[] calcIndexKey(byte[] lastKeyOfPreviousBlock, byte[] firstKeyInBlock) {
return firstKeyInBlock;
}
@ -2952,4 +2917,171 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
sum += Bytes.SIZEOF_LONG;// memstoreTS
return ClassSize.align(sum);
}
/**
* A simple form of KeyValue that creates a keyvalue with only the key part of the byte[]
* Mainly used in places where we need to compare two cells. Avoids copying of bytes
* In places like block index keys, we need to compare the key byte[] with a cell.
* Hence create a Keyvalue(aka Cell) that would help in comparing as two cells
*/
public static class KeyOnlyKeyValue extends KeyValue {
private int length = 0;
private int offset = 0;
private byte[] b;
public KeyOnlyKeyValue() {
}
public KeyOnlyKeyValue(byte[] b, int offset, int length) {
this.b = b;
this.length = length;
this.offset = offset;
}
@Override
public int getKeyOffset() {
return this.offset;
}
/**
* A setter that helps to avoid object creation every time and whenever
* there is a need to create new KeyOnlyKeyValue.
* @param key
* @param offset
* @param length
*/
public void setKey(byte[] key, int offset, int length) {
this.b = key;
this.offset = offset;
this.length = length;
}
@Override
public byte[] getKey() {
int keylength = getKeyLength();
byte[] key = new byte[keylength];
System.arraycopy(this.b, getKeyOffset(), key, 0, keylength);
return key;
}
@Override
public byte[] getRowArray() {
return b;
}
@Override
public int getRowOffset() {
return getKeyOffset() + Bytes.SIZEOF_SHORT;
}
@Override
public byte[] getFamilyArray() {
return b;
}
@Override
public byte getFamilyLength() {
return this.b[getFamilyOffset() - 1];
}
@Override
public int getFamilyOffset() {
return this.offset + Bytes.SIZEOF_SHORT + getRowLength() + Bytes.SIZEOF_BYTE;
}
@Override
public byte[] getQualifierArray() {
return b;
}
@Override
public int getQualifierLength() {
return getQualifierLength(getRowLength(), getFamilyLength());
}
@Override
public int getQualifierOffset() {
return getFamilyOffset() + getFamilyLength();
}
@Override
public int getKeyLength() {
return length;
}
@Override
public short getRowLength() {
return Bytes.toShort(this.b, getKeyOffset());
}
@Override
public byte getTypeByte() {
return this.b[this.offset + getKeyLength() - 1];
}
private int getQualifierLength(int rlength, int flength) {
return getKeyLength() - (int) getKeyDataStructureSize(rlength, flength, 0);
}
@Override
public long getTimestamp() {
int tsOffset = getTimestampOffset();
return Bytes.toLong(this.b, tsOffset);
}
@Override
public int getTimestampOffset() {
return getKeyOffset() + getKeyLength() - TIMESTAMP_TYPE_SIZE;
}
@Override
public byte[] getTagsArray() {
return HConstants.EMPTY_BYTE_ARRAY;
}
@Override
public int getTagsOffset() {
return (short) 0;
}
@Override
public byte[] getValueArray() {
throw new IllegalArgumentException("KeyOnlyKeyValue does not work with values.");
}
@Override
public int getValueOffset() {
throw new IllegalArgumentException("KeyOnlyKeyValue does not work with values.");
}
@Override
public int getValueLength() {
throw new IllegalArgumentException("KeyOnlyKeyValue does not work with values.");
}
@Override
public short getTagsLength() {
return (short) 0;
}
@Override
public String toString() {
if (this.b == null || this.b.length == 0) {
return "empty";
}
return keyToString(this.b, this.offset + ROW_OFFSET, getKeyLength()) + "/vlen="
+ getValueLength() + "/mvcc=" + 0;
}
@Override
public int hashCode() {
return super.hashCode();
}
@Override
public boolean equals(Object other) {
return super.equals(other);
}
}
}

View File

@ -22,10 +22,13 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.io.TagCompressionContext;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -193,6 +196,12 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
current.keyBuffer, 0, current.keyLength);
}
@Override
public int compareKey(KVComparator comparator, Cell key) {
return comparator.compareOnlyKeyPortion(key,
new KeyValue.KeyOnlyKeyValue(current.keyBuffer, 0, current.keyLength));
}
@Override
public void setCurrentBuffer(ByteBuffer buffer) {
if (this.tagCompressionContext != null) {
@ -304,36 +313,89 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
}
@Override
public int seekToKeyInBlock(byte[] key, int offset, int length,
boolean seekBefore) {
int commonPrefix = 0;
public int seekToKeyInBlock(byte[] key, int offset, int length, boolean seekBefore) {
return seekToKeyInBlock(new KeyValue.KeyOnlyKeyValue(key, offset, length), seekBefore);
}
@Override
public int seekToKeyInBlock(Cell seekCell, boolean seekBefore) {
int rowCommonPrefix = 0;
int familyCommonPrefix = 0;
int qualCommonPrefix = 0;
previous.invalidate();
KeyValue.KeyOnlyKeyValue currentCell = new KeyValue.KeyOnlyKeyValue();
do {
int comp;
if (samePrefixComparator != null) {
commonPrefix = Math.min(commonPrefix, current.lastCommonPrefix);
// extend commonPrefix
commonPrefix += ByteBufferUtils.findCommonPrefix(
key, offset + commonPrefix, length - commonPrefix,
current.keyBuffer, commonPrefix,
current.keyLength - commonPrefix);
comp = samePrefixComparator.compareIgnoringPrefix(commonPrefix, key,
offset, length, current.keyBuffer, 0, current.keyLength);
currentCell.setKey(current.keyBuffer, 0, current.keyLength);
if (current.lastCommonPrefix != 0) {
// The KV format has row key length also in the byte array. The
// common prefix
// includes it. So we need to subtract to find out the common prefix
// in the
// row part alone
rowCommonPrefix = Math.min(rowCommonPrefix, current.lastCommonPrefix - 2);
}
if (current.lastCommonPrefix <= 2) {
rowCommonPrefix = 0;
}
rowCommonPrefix += CellComparator.findCommonPrefixInRowPart(seekCell, currentCell,
rowCommonPrefix);
comp = CellComparator.compareCommonRowPrefix(seekCell, currentCell, rowCommonPrefix);
if (comp == 0) {
comp = compareTypeBytes(seekCell, currentCell);
if (comp == 0) {
// Subtract the fixed row key length and the family key fixed length
familyCommonPrefix = Math.max(
0,
Math.min(familyCommonPrefix,
current.lastCommonPrefix - (3 + currentCell.getRowLength())));
familyCommonPrefix += CellComparator.findCommonPrefixInFamilyPart(seekCell,
currentCell, familyCommonPrefix);
comp = CellComparator.compareCommonFamilyPrefix(seekCell, currentCell,
familyCommonPrefix);
if (comp == 0) {
// subtract the rowkey fixed length and the family key fixed
// length
qualCommonPrefix = Math.max(
0,
Math.min(
qualCommonPrefix,
current.lastCommonPrefix
- (3 + currentCell.getRowLength() + currentCell.getFamilyLength())));
qualCommonPrefix += CellComparator.findCommonPrefixInQualifierPart(seekCell,
currentCell, qualCommonPrefix);
comp = CellComparator.compareCommonQualifierPrefix(seekCell, currentCell,
qualCommonPrefix);
if (comp == 0) {
comp = CellComparator.compareTimestamps(seekCell, currentCell);
if (comp == 0) {
// Compare types. Let the delete types sort ahead of puts;
// i.e. types
// of higher numbers sort before those of lesser numbers.
// Maximum
// (255)
// appears ahead of everything, and minimum (0) appears
// after
// everything.
comp = (0xff & currentCell.getTypeByte()) - (0xff & seekCell.getTypeByte());
}
}
}
}
}
} else {
comp = comparator.compareFlatKey(key, offset, length,
current.keyBuffer, 0, current.keyLength);
Cell r = new KeyValue.KeyOnlyKeyValue(current.keyBuffer, 0, current.keyLength);
comp = comparator.compareOnlyKeyPortion(seekCell, r);
}
if (comp == 0) { // exact match
if (seekBefore) {
if (!previous.isValid()) {
// The caller (seekBefore) has to ensure that we are not at the
// first key in the block.
throw new IllegalStateException("Cannot seekBefore if " +
"positioned at the first key in the block: key=" +
Bytes.toStringBinary(key, offset, length));
throw new IllegalStateException("Cannot seekBefore if "
+ "positioned at the first key in the block: key="
+ Bytes.toStringBinary(seekCell.getRowArray()));
}
moveToPrevious();
return 1;
@ -363,6 +425,20 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
return 1;
}
private int compareTypeBytes(Cell key, Cell right) {
if (key.getFamilyLength() + key.getQualifierLength() == 0
&& key.getTypeByte() == Type.Minimum.getCode()) {
// left is "bigger", i.e. it appears later in the sorted order
return 1;
}
if (right.getFamilyLength() + right.getQualifierLength() == 0
&& right.getTypeByte() == Type.Minimum.getCode()) {
return -1;
}
return 0;
}
private void moveToPrevious() {
if (!previous.isValid()) {
throw new IllegalStateException(

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -174,9 +175,26 @@ public interface DataBlockEncoder {
* of an exact match. Does not matter in case of an inexact match.
* @return 0 on exact match, 1 on inexact match.
*/
@Deprecated
int seekToKeyInBlock(
byte[] key, int offset, int length, boolean seekBefore
);
/**
* Moves the seeker position within the current block to:
* <ul>
* <li>the last key that that is less than or equal to the given key if
* <code>seekBefore</code> is false</li>
* <li>the last key that is strictly less than the given key if <code>
* seekBefore</code> is true. The caller is responsible for loading the
* previous block if the requested key turns out to be the first key of the
* current block.</li>
* </ul>
* @param key - Cell to which the seek should happen
* @param seekBefore find the key strictly less than the given key in case
* of an exact match. Does not matter in case of an inexact match.
* @return 0 on exact match, 1 on inexact match.
*/
int seekToKeyInBlock(Cell key, boolean seekBefore);
/**
* Compare the given key against the current key
@ -187,5 +205,7 @@ public interface DataBlockEncoder {
* @return -1 is the passed key is smaller than the current key, 0 if equal and 1 if greater
*/
public int compareKey(KVComparator comparator, byte[] key, int offset, int length);
public int compareKey(KVComparator comparator, Cell key);
}
}

View File

@ -43,6 +43,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
@ -1634,6 +1636,43 @@ public class Bytes {
return - (low+1);
}
/**
* Binary search for keys in indexes.
*
* @param arr array of byte arrays to search for
* @param key the key you want to find
* @param comparator a comparator to compare.
* @return zero-based index of the key, if the key is present in the array.
* Otherwise, a value -(i + 1) such that the key is between arr[i -
* 1] and arr[i] non-inclusively, where i is in [0, i], if we define
* arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above
* means that this function can return 2N + 1 different values
* ranging from -(N + 1) to N - 1.
* @return the index of the block
*/
public static int binarySearch(byte[][] arr, Cell key, RawComparator<Cell> comparator) {
int low = 0;
int high = arr.length - 1;
KeyValue.KeyOnlyKeyValue r = new KeyValue.KeyOnlyKeyValue();
while (low <= high) {
int mid = (low+high) >>> 1;
// we have to compare in this order, because the comparator order
// has special logic when the 'left side' is a special key.
r.setKey(arr[mid], 0, arr[mid].length);
int cmp = comparator.compare(key, r);
// key lives above the midpoint
if (cmp > 0)
low = mid + 1;
// key lives below the midpoint
else if (cmp < 0)
high = mid - 1;
// BAM. how often does this really happen?
else
return mid;
}
return - (low+1);
}
/**
* Bytewise binary increment/deincrement of long contained in byte array
* on given amount.

View File

@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestCellComparator {
byte[] row1 = Bytes.toBytes("row1");
byte[] row2 = Bytes.toBytes("row2");
byte[] row_1_0 = Bytes.toBytes("row10");
byte[] fam1 = Bytes.toBytes("fam1");
byte[] fam2 = Bytes.toBytes("fam2");
byte[] fam_1_2 = Bytes.toBytes("fam12");
byte[] qual1 = Bytes.toBytes("qual1");
byte[] qual2 = Bytes.toBytes("qual2");
byte[] val = Bytes.toBytes("val");
@Test
public void testCompareCells() {
KeyValue kv1 = new KeyValue(row1, fam1, qual1, val);
KeyValue kv2 = new KeyValue(row2, fam1, qual1, val);
assertTrue((CellComparator.compareStatic(kv1, kv2, false)) < 0);
kv1 = new KeyValue(row1, fam2, qual1, val);
kv2 = new KeyValue(row1, fam1, qual1, val);
assertTrue((CellComparator.compareFamilies(kv1, kv2) > 0));
kv1 = new KeyValue(row1, fam1, qual1, 1l, val);
kv2 = new KeyValue(row1, fam1, qual1, 2l, val);
assertTrue((CellComparator.compareStatic(kv1, kv2, false) > 0));
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
assertTrue((CellComparator.compareStatic(kv1, kv2, false) > 0));
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
kv2 = new KeyValue(row1, fam_1_2, qual1, 1l, Type.Maximum);
assertTrue((CellComparator.compareCommonFamilyPrefix(kv1, kv2, 4) < 0));
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
kv2 = new KeyValue(row_1_0, fam_1_2, qual1, 1l, Type.Maximum);
assertTrue((CellComparator.compareCommonRowPrefix(kv1, kv2, 4) < 0));
kv1 = new KeyValue(row1, fam1, qual2, 1l, Type.Put);
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Maximum);
assertTrue((CellComparator.compareCommonQualifierPrefix(kv1, kv2, 4) > 0));
kv1 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
kv2 = new KeyValue(row1, fam1, qual1, 1l, Type.Put);
assertTrue((CellComparator.equals(kv1, kv2)));
}
}

View File

@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher;
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
@ -152,15 +152,13 @@ public class PrefixTreeSeeker implements EncodedSeeker {
boolean forceBeforeOnExactMatch) {
if (USE_POSITION_BEFORE) {
return seekToOrBeforeUsingPositionAtOrBefore(keyOnlyBytes, offset, length,
forceBeforeOnExactMatch);
}else{
forceBeforeOnExactMatch);
} else {
return seekToOrBeforeUsingPositionAtOrAfter(keyOnlyBytes, offset, length,
forceBeforeOnExactMatch);
forceBeforeOnExactMatch);
}
}
/*
* Support both of these options since the underlying PrefixTree supports both. Possibly
* expand the EncodedSeeker to utilize them both.
@ -169,11 +167,22 @@ public class PrefixTreeSeeker implements EncodedSeeker {
protected int seekToOrBeforeUsingPositionAtOrBefore(byte[] keyOnlyBytes, int offset, int length,
boolean seekBefore){
// this does a deep copy of the key byte[] because the CellSearcher interface wants a Cell
KeyValue kv = KeyValue.createKeyValueFromKey(keyOnlyBytes, offset, length);
KeyValue kv = new KeyValue.KeyOnlyKeyValue(keyOnlyBytes, offset, length);
return seekToOrBeforeUsingPositionAtOrBefore(kv, seekBefore);
}
/*
* Support both of these options since the underlying PrefixTree supports
* both. Possibly expand the EncodedSeeker to utilize them both.
*/
protected int seekToOrBeforeUsingPositionAtOrBefore(Cell kv, boolean seekBefore) {
// this does a deep copy of the key byte[] because the CellSearcher
// interface wants a Cell
CellScannerPosition position = ptSearcher.seekForwardToOrBefore(kv);
if(CellScannerPosition.AT == position){
if (CellScannerPosition.AT == position) {
if (seekBefore) {
ptSearcher.previous();
return 1;
@ -184,16 +193,19 @@ public class PrefixTreeSeeker implements EncodedSeeker {
return 1;
}
protected int seekToOrBeforeUsingPositionAtOrAfter(byte[] keyOnlyBytes, int offset, int length,
boolean seekBefore){
// this does a deep copy of the key byte[] because the CellSearcher interface wants a Cell
KeyValue kv = KeyValue.createKeyValueFromKey(keyOnlyBytes, offset, length);
boolean seekBefore) {
// this does a deep copy of the key byte[] because the CellSearcher
// interface wants a Cell
KeyValue kv = new KeyValue.KeyOnlyKeyValue(keyOnlyBytes, offset, length);
return seekToOrBeforeUsingPositionAtOrAfter(kv, seekBefore);
}
//should probably switch this to use the seekForwardToOrBefore method
protected int seekToOrBeforeUsingPositionAtOrAfter(Cell kv, boolean seekBefore) {
// should probably switch this to use the seekForwardToOrBefore method
CellScannerPosition position = ptSearcher.seekForwardToOrAfter(kv);
if(CellScannerPosition.AT == position){
if (CellScannerPosition.AT == position) {
if (seekBefore) {
ptSearcher.previous();
return 1;
@ -202,21 +214,21 @@ public class PrefixTreeSeeker implements EncodedSeeker {
}
if(CellScannerPosition.AFTER == position){
if(!ptSearcher.isBeforeFirst()){
if (CellScannerPosition.AFTER == position) {
if (!ptSearcher.isBeforeFirst()) {
ptSearcher.previous();
}
return 1;
}
if(position == CellScannerPosition.AFTER_LAST){
if (position == CellScannerPosition.AFTER_LAST) {
if (seekBefore) {
ptSearcher.previous();
}
return 1;
}
throw new RuntimeException("unexpected CellScannerPosition:"+position);
throw new RuntimeException("unexpected CellScannerPosition:" + position);
}
@Override
@ -225,4 +237,20 @@ public class PrefixTreeSeeker implements EncodedSeeker {
ByteBuffer bb = getKeyDeepCopy();
return comparator.compareFlatKey(key, offset, length, bb.array(), bb.arrayOffset(), bb.limit());
}
@Override
public int seekToKeyInBlock(Cell key, boolean forceBeforeOnExactMatch) {
if (USE_POSITION_BEFORE) {
return seekToOrBeforeUsingPositionAtOrBefore(key, forceBeforeOnExactMatch);
}else{
return seekToOrBeforeUsingPositionAtOrAfter(key, forceBeforeOnExactMatch);
}
}
@Override
public int compareKey(KVComparator comparator, Cell key) {
ByteBuffer bb = getKeyDeepCopy();
return comparator.compare(key,
new KeyValue.KeyOnlyKeyValue(bb.array(), bb.arrayOffset(), bb.limit()));
}
}

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.codec.prefixtree.decode.row.RowNodeReader;
import org.apache.hadoop.hbase.codec.prefixtree.decode.timestamp.MvccVersionDecoder;
import org.apache.hadoop.hbase.codec.prefixtree.decode.timestamp.TimestampDecoder;
import org.apache.hadoop.hbase.codec.prefixtree.encode.other.ColumnNodeType;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Extends PtCell and manipulates its protected fields. Could alternatively contain a PtCell and
@ -420,7 +419,7 @@ public class PrefixTreeArrayScanner extends PrefixTreeCell implements CellScanne
protected int populateNonRowFieldsAndCompareTo(int cellNum, Cell key) {
populateNonRowFields(cellNum);
return CellComparator.compareStatic(this, key);
return CellComparator.compareStatic(this, key, false);
}
protected void populateFirstNonRowFields() {

View File

@ -106,7 +106,7 @@ public class PrefixTreeCell implements Cell, Comparable<Cell> {
@Override
public int compareTo(Cell other) {
return CellComparator.compareStatic(this, other);
return CellComparator.compareStatic(this, other, false);
}
@Override

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
@ -56,6 +57,8 @@ public class HalfStoreFileReader extends StoreFile.Reader {
// i.e. empty column and a timestamp of LATEST_TIMESTAMP.
protected final byte [] splitkey;
protected final Cell splitCell;
private byte[] firstKey = null;
private boolean firstKeySeeked = false;
@ -79,6 +82,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
// have an actual midkey themselves. No midkey is how we indicate file is
// not splittable.
this.splitkey = r.getSplitKey();
this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length);
// Is it top or bottom half?
this.top = Reference.isTopFileRegion(r.getFileRegion());
}
@ -104,6 +108,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
// have an actual midkey themselves. No midkey is how we indicate file is
// not splittable.
this.splitkey = r.getSplitKey();
this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length);
// Is it top or bottom half?
this.top = Reference.isTopFileRegion(r.getFileRegion());
}
@ -168,33 +173,21 @@ public class HalfStoreFileReader extends StoreFile.Reader {
return true;
}
@Override
public boolean seekBefore(byte[] key) throws IOException {
return seekBefore(key, 0, key.length);
}
@Override
public boolean seekBefore(byte [] key, int offset, int length)
throws IOException {
if (top) {
byte[] fk = getFirstKey();
// This will be null when the file is empty in which we can not seekBefore to any key
if (fk == null) return false;
if (getComparator().compareFlatKey(key, offset, length, fk, 0,
fk.length) <= 0) {
return false;
}
} else {
// The equals sign isn't strictly necessary just here to be consistent with seekTo
if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
splitkey.length) >= 0) {
return this.delegate.seekBefore(splitkey, 0, splitkey.length);
}
}
return this.delegate.seekBefore(key, offset, length);
return seekBefore(new KeyValue.KeyOnlyKeyValue(key, offset, length));
}
@Override
public boolean seekTo() throws IOException {
if (top) {
int r = this.delegate.seekTo(splitkey);
int r = this.delegate.seekTo(new KeyValue.KeyOnlyKeyValue(splitkey, 0, splitkey.length));
if (r == HConstants.INDEX_KEY_MAGIC) {
return true;
}
@ -219,29 +212,14 @@ public class HalfStoreFileReader extends StoreFile.Reader {
splitkey, 0, splitkey.length) < 0;
}
@Override
public int seekTo(byte[] key) throws IOException {
return seekTo(key, 0, key.length);
}
@Override
public int seekTo(byte[] key, int offset, int length) throws IOException {
if (top) {
if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
splitkey.length) < 0) {
return -1;
}
} else {
if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
splitkey.length) >= 0) {
// we would place the scanner in the second half.
// it might be an error to return false here ever...
boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
if (!res) {
throw new IOException("Seeking for a key in bottom of file, but key exists in top of file, failed on seekBefore(midkey)");
}
return 1;
}
}
return delegate.seekTo(key, offset, length);
return seekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
}
@Override
@ -254,29 +232,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
throws IOException {
//This function is identical to the corresponding seekTo function except
//that we call reseekTo (and not seekTo) on the delegate.
if (top) {
if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
splitkey.length) < 0) {
return -1;
}
} else {
if (getComparator().compareFlatKey(key, offset, length, splitkey, 0,
splitkey.length) >= 0) {
// we would place the scanner in the second half.
// it might be an error to return false here ever...
boolean res = delegate.seekBefore(splitkey, 0, splitkey.length);
if (!res) {
throw new IOException("Seeking for a key in bottom of file, but" +
" key exists in top of file, failed on seekBefore(midkey)");
}
return 1;
}
}
if (atEnd) {
// skip the 'reseek' and just return 1.
return 1;
}
return delegate.reseekTo(key, offset, length);
return reseekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
}
public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
@ -286,6 +242,77 @@ public class HalfStoreFileReader extends StoreFile.Reader {
public boolean isSeeked() {
return this.delegate.isSeeked();
}
@Override
public int seekTo(Cell key) throws IOException {
if (top) {
if (getComparator().compareOnlyKeyPortion(key, splitCell) < 0) {
return -1;
}
} else {
if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
// we would place the scanner in the second half.
// it might be an error to return false here ever...
boolean res = delegate.seekBefore(splitCell);
if (!res) {
throw new IOException(
"Seeking for a key in bottom of file, but key exists in top of file, " +
"failed on seekBefore(midkey)");
}
return 1;
}
}
return delegate.seekTo(key);
}
@Override
public int reseekTo(Cell key) throws IOException {
// This function is identical to the corresponding seekTo function
// except
// that we call reseekTo (and not seekTo) on the delegate.
if (top) {
if (getComparator().compareOnlyKeyPortion(key, splitCell) < 0) {
return -1;
}
} else {
if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
// we would place the scanner in the second half.
// it might be an error to return false here ever...
boolean res = delegate.seekBefore(splitCell);
if (!res) {
throw new IOException("Seeking for a key in bottom of file, but"
+ " key exists in top of file, failed on seekBefore(midkey)");
}
return 1;
}
}
if (atEnd) {
// skip the 'reseek' and just return 1.
return 1;
}
return delegate.reseekTo(key);
}
@Override
public boolean seekBefore(Cell key) throws IOException {
if (top) {
Cell fk = new KeyValue.KeyOnlyKeyValue(getFirstKey(), 0, getFirstKey().length);
// This will be null when the file is empty in which we can not
// seekBefore to any key
if (fk == null)
return false;
if (getComparator().compareOnlyKeyPortion(key, fk) <= 0) {
return false;
}
} else {
// The equals sign isn't strictly necessary just here to be consistent
// with seekTo
if (getComparator().compareOnlyKeyPortion(key, splitCell) >= 0) {
return this.delegate.seekBefore(splitCell);
}
}
return this.delegate.seekBefore(key);
}
};
}

View File

@ -36,9 +36,11 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
@ -165,8 +167,6 @@ public class HFileBlockIndex {
* be called when the HFile version is larger than 1.
*
* @param key the key we are looking for
* @param keyOffset the offset of the key in its byte array
* @param keyLength the length of the key
* @param currentBlock the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
@ -177,12 +177,12 @@ public class HFileBlockIndex {
* @return reader a basic way to load blocks
* @throws IOException
*/
public HFileBlock seekToDataBlock(final byte[] key, int keyOffset,
int keyLength, HFileBlock currentBlock, boolean cacheBlocks,
public HFileBlock seekToDataBlock(final Cell key, HFileBlock currentBlock, boolean cacheBlocks,
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
BlockWithScanInfo blockWithScanInfo = loadDataBlockWithScanInfo(key, keyOffset, keyLength,
currentBlock, cacheBlocks, pread, isCompaction, expectedDataBlockEncoding);
BlockWithScanInfo blockWithScanInfo = loadDataBlockWithScanInfo(key, currentBlock,
cacheBlocks,
pread, isCompaction, expectedDataBlockEncoding);
if (blockWithScanInfo == null) {
return null;
} else {
@ -191,30 +191,29 @@ public class HFileBlockIndex {
}
/**
* Return the BlockWithScanInfo which contains the DataBlock with other scan info
* such as nextIndexedKey.
* This function will only be called when the HFile version is larger than 1.
* Return the BlockWithScanInfo which contains the DataBlock with other scan
* info such as nextIndexedKey. This function will only be called when the
* HFile version is larger than 1.
*
* @param key the key we are looking for
* @param keyOffset the offset of the key in its byte array
* @param keyLength the length of the key
* @param currentBlock the current block, to avoid re-reading the same
* block
* @param key
* the key we are looking for
* @param currentBlock
* the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding.
* @return the BlockWithScanInfo which contains the DataBlock with other scan info
* such as nextIndexedKey.
* @return the BlockWithScanInfo which contains the DataBlock with other
* scan info such as nextIndexedKey.
* @throws IOException
*/
public BlockWithScanInfo loadDataBlockWithScanInfo(final byte[] key, int keyOffset,
int keyLength, HFileBlock currentBlock, boolean cacheBlocks,
public BlockWithScanInfo loadDataBlockWithScanInfo(Cell key, HFileBlock currentBlock,
boolean cacheBlocks,
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
int rootLevelIndex = rootBlockContainingKey(key, keyOffset, keyLength);
int rootLevelIndex = rootBlockContainingKey(key);
if (rootLevelIndex < 0 || rootLevelIndex >= blockOffsets.length) {
return null;
}
@ -283,10 +282,13 @@ public class HFileBlockIndex {
// Locate the entry corresponding to the given key in the non-root
// (leaf or intermediate-level) index block.
ByteBuffer buffer = block.getBufferWithoutHeader();
index = locateNonRootIndexEntry(buffer, key, keyOffset, keyLength, comparator);
index = locateNonRootIndexEntry(buffer, key, comparator);
if (index == -1) {
// This has to be changed
// For now change this to key value
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
throw new IOException("The key "
+ Bytes.toStringBinary(key, keyOffset, keyLength)
+ Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
+ " is before the" + " first key of the non-root index block "
+ block);
}
@ -395,10 +397,35 @@ public class HFileBlockIndex {
* number of blocks - 1) or -1 if this file does not contain the
* request.
*/
public int rootBlockContainingKey(final byte[] key, int offset,
int length) {
int pos = Bytes.binarySearch(blockKeys, key, offset, length,
comparator);
public int rootBlockContainingKey(final byte[] key, int offset, int length) {
int pos = Bytes.binarySearch(blockKeys, key, offset, length, comparator);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
if (pos >= 0) {
// This means this is an exact match with an element of blockKeys.
assert pos < blockKeys.length;
return pos;
}
// Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i],
// and i is in [0, blockKeys.length]. We are returning j = i - 1 such that
// blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if
// key < blockKeys[0], meaning the file does not contain the given key.
int i = -pos - 1;
assert 0 <= i && i <= blockKeys.length;
return i - 1;
}
/**
* Finds the root-level index block containing the given key.
*
* @param key
* Key to find
*/
public int rootBlockContainingKey(final Cell key) {
int pos = Bytes.binarySearch(blockKeys, key, comparator);
// pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
// binarySearch's javadoc.
@ -472,19 +499,18 @@ public class HFileBlockIndex {
* secondary index, which records the offsets of (offset, onDiskSize,
* firstKey) tuples of all entries.
*
* @param key the key we are searching for offsets to individual entries in
* @param key
* the key we are searching for offsets to individual entries in
* the blockIndex buffer
* @param keyOffset the offset of the key in its byte array
* @param keyLength the length of the key
* @param nonRootIndex the non-root index block buffer, starting with the
* secondary index. The position is ignored.
* @param nonRootIndex
* the non-root index block buffer, starting with the secondary
* index. The position is ignored.
* @return the index i in [0, numEntries - 1] such that keys[i] <= key <
* keys[i + 1], if keys is the array of all keys being searched, or
* -1 otherwise
* @throws IOException
*/
static int binarySearchNonRootIndex(byte[] key, int keyOffset,
int keyLength, ByteBuffer nonRootIndex,
static int binarySearchNonRootIndex(Cell key, ByteBuffer nonRootIndex,
KVComparator comparator) {
int numEntries = nonRootIndex.getInt(0);
@ -499,7 +525,7 @@ public class HFileBlockIndex {
// If we imagine that keys[-1] = -Infinity and
// keys[numEntries] = Infinity, then we are maintaining an invariant that
// keys[low - 1] < key < keys[high + 1] while narrowing down the range.
KeyValue.KeyOnlyKeyValue nonRootIndexKV = new KeyValue.KeyOnlyKeyValue();
while (low <= high) {
mid = (low + high) >>> 1;
@ -520,9 +546,9 @@ public class HFileBlockIndex {
// we have to compare in this order, because the comparator order
// has special logic when the 'left side' is a special key.
int cmp = comparator.compareFlatKey(key, keyOffset, keyLength,
nonRootIndex.array(), nonRootIndex.arrayOffset() + midKeyOffset,
midLength);
nonRootIndexKV.setKey(nonRootIndex.array(),
nonRootIndex.arrayOffset() + midKeyOffset, midLength);
int cmp = comparator.compareOnlyKeyPortion(key, nonRootIndexKV);
// key lives above the midpoint
if (cmp > 0)
@ -562,19 +588,18 @@ public class HFileBlockIndex {
* of success, positions the provided buffer at the entry of interest, where
* the file offset and the on-disk-size can be read.
*
* @param nonRootBlock a non-root block without header. Initial position
* does not matter.
* @param key the byte array containing the key
* @param keyOffset the offset of the key in its byte array
* @param keyLength the length of the key
* @return the index position where the given key was found,
* otherwise return -1 in the case the given key is before the first key.
* @param nonRootBlock
* a non-root block without header. Initial position does not
* matter.
* @param key
* the byte array containing the key
* @return the index position where the given key was found, otherwise
* return -1 in the case the given key is before the first key.
*
*/
static int locateNonRootIndexEntry(ByteBuffer nonRootBlock, byte[] key,
int keyOffset, int keyLength, KVComparator comparator) {
int entryIndex = binarySearchNonRootIndex(key, keyOffset, keyLength,
nonRootBlock, comparator);
static int locateNonRootIndexEntry(ByteBuffer nonRootBlock, Cell key,
KVComparator comparator) {
int entryIndex = binarySearchNonRootIndex(key, nonRootBlock, comparator);
if (entryIndex != -1) {
int numEntries = nonRootBlock.getInt(0);
@ -584,8 +609,7 @@ public class HFileBlockIndex {
// The offset of the entry we are interested in relative to the end of
// the secondary index.
int entryRelOffset = nonRootBlock.getInt(Bytes.SIZEOF_INT
* (1 + entryIndex));
int entryRelOffset = nonRootBlock.getInt(Bytes.SIZEOF_INT * (1 + entryIndex));
nonRootBlock.position(entriesOffset + entryRelOffset);
}

View File

@ -28,9 +28,11 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
@ -278,8 +280,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
byte[] mbname = Bytes.toBytes(metaBlockName);
int block = metaBlockIndexReader.rootBlockContainingKey(mbname, 0,
mbname.length);
int block = metaBlockIndexReader.rootBlockContainingKey(mbname,
0, mbname.length);
if (block == -1)
return null;
long blockSize = metaBlockIndexReader.getRootBlockDataSize(block);
@ -530,13 +532,66 @@ public class HFileReaderV2 extends AbstractHFileReader {
super(r, cacheBlocks, pread, isCompaction);
}
protected abstract ByteBuffer getFirstKeyInBlock(HFileBlock curBlock);
protected abstract int loadBlockAndSeekToKey(HFileBlock seekToBlock, byte[] nextIndexedKey,
boolean rewind, Cell key, boolean seekBefore) throws IOException;
@Override
public int seekTo(byte[] key, int offset, int length) throws IOException {
// Always rewind to the first key of the block, because the given key
// might be before or after the current key.
return seekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
}
@Override
public int reseekTo(byte[] key, int offset, int length) throws IOException {
return reseekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
}
@Override
public int seekTo(Cell key) throws IOException {
return seekTo(key, true);
}
@Override
public int reseekTo(Cell key) throws IOException {
int compared;
if (isSeeked()) {
compared = compareKey(reader.getComparator(), key);
if (compared < 1) {
// If the required key is less than or equal to current key, then
// don't do anything.
return compared;
} else {
// The comparison with no_next_index_key has to be checked
if (this.nextIndexedKey != null &&
(this.nextIndexedKey == HConstants.NO_NEXT_INDEXED_KEY || reader
.getComparator()
.compareOnlyKeyPortion(key,
new KeyValue.KeyOnlyKeyValue(nextIndexedKey, 0,
nextIndexedKey.length)) < 0)) {
// The reader shall continue to scan the current data block instead
// of querying the
// block index as long as it knows the target key is strictly
// smaller than
// the next indexed key or the current data block is the last data
// block.
return loadBlockAndSeekToKey(this.block, nextIndexedKey, false, key, false);
}
}
}
// Don't rewind on a reseek operation, because reseek implies that we are
// always going forward in the file.
return seekTo(key, false);
}
/**
* An internal API function. Seek to the given key, optionally rewinding to
* the first key of the block before doing the seek.
*
* @param key key byte array
* @param offset key offset in the key byte array
* @param length key length
* @param key - a cell representing the key that we need to fetch
* @param rewind whether to rewind to the first key of the block before
* doing the seek. If this is false, we are assuming we never go
* back, otherwise the result is undefined.
@ -546,76 +601,37 @@ public class HFileReaderV2 extends AbstractHFileReader {
* using a faked index key
* @throws IOException
*/
protected int seekTo(byte[] key, int offset, int length, boolean rewind)
throws IOException {
HFileBlockIndex.BlockIndexReader indexReader =
reader.getDataBlockIndexReader();
BlockWithScanInfo blockWithScanInfo =
indexReader.loadDataBlockWithScanInfo(key, offset, length, block,
cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding());
public int seekTo(Cell key, boolean rewind) throws IOException {
HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader();
BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(key, block,
cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding());
if (blockWithScanInfo == null || blockWithScanInfo.getHFileBlock() == null) {
// This happens if the key e.g. falls before the beginning of the file.
return -1;
}
return loadBlockAndSeekToKey(blockWithScanInfo.getHFileBlock(),
blockWithScanInfo.getNextIndexedKey(), rewind, key, offset, length, false);
}
protected abstract ByteBuffer getFirstKeyInBlock(HFileBlock curBlock);
protected abstract int loadBlockAndSeekToKey(HFileBlock seekToBlock, byte[] nextIndexedKey,
boolean rewind, byte[] key, int offset, int length, boolean seekBefore)
throws IOException;
@Override
public int seekTo(byte[] key, int offset, int length) throws IOException {
// Always rewind to the first key of the block, because the given key
// might be before or after the current key.
return seekTo(key, offset, length, true);
blockWithScanInfo.getNextIndexedKey(), rewind, key, false);
}
@Override
public int reseekTo(byte[] key, int offset, int length) throws IOException {
int compared;
if (isSeeked()) {
compared = compareKey(reader.getComparator(), key, offset, length);
if (compared < 1) {
// If the required key is less than or equal to current key, then
// don't do anything.
return compared;
} else {
if (this.nextIndexedKey != null &&
(this.nextIndexedKey == HConstants.NO_NEXT_INDEXED_KEY ||
reader.getComparator().compareFlatKey(key, offset, length,
nextIndexedKey, 0, nextIndexedKey.length) < 0)) {
// The reader shall continue to scan the current data block instead of querying the
// block index as long as it knows the target key is strictly smaller than
// the next indexed key or the current data block is the last data block.
return loadBlockAndSeekToKey(this.block, this.nextIndexedKey,
false, key, offset, length, false);
}
}
}
// Don't rewind on a reseek operation, because reseek implies that we are
// always going forward in the file.
return seekTo(key, offset, length, false);
public boolean seekBefore(byte[] key, int offset, int length) throws IOException {
return seekBefore(new KeyValue.KeyOnlyKeyValue(key, offset, length));
}
@Override
public boolean seekBefore(byte[] key, int offset, int length)
throws IOException {
HFileBlock seekToBlock =
reader.getDataBlockIndexReader().seekToDataBlock(key, offset, length,
block, cacheBlocks, pread, isCompaction,
((HFileReaderV2)reader).getEffectiveEncodingInCache(isCompaction));
public boolean seekBefore(Cell key) throws IOException {
HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, block,
cacheBlocks, pread, isCompaction,
((HFileReaderV2) reader).getEffectiveEncodingInCache(isCompaction));
if (seekToBlock == null) {
return false;
}
ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock);
if (reader.getComparator().compareFlatKey(firstKey.array(),
firstKey.arrayOffset(), firstKey.limit(), key, offset, length) >= 0)
{
if (reader.getComparator()
.compareOnlyKeyPortion(
new KeyValue.KeyOnlyKeyValue(firstKey.array(), firstKey.arrayOffset(),
firstKey.limit()), key) >= 0) {
long previousBlockOffset = seekToBlock.getPrevBlockOffset();
// The key we are interested in
if (previousBlockOffset == -1) {
@ -633,11 +649,10 @@ public class HFileReaderV2 extends AbstractHFileReader {
// block.
}
byte[] firstKeyInCurrentBlock = Bytes.getBytes(firstKey);
loadBlockAndSeekToKey(seekToBlock, firstKeyInCurrentBlock, true, key, offset, length, true);
loadBlockAndSeekToKey(seekToBlock, firstKeyInCurrentBlock, true, key, true);
return true;
}
/**
* Scans blocks in the "scanned" section of the {@link HFile} until the next
* data block is found.
@ -684,6 +699,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
*/
public abstract int compareKey(KVComparator comparator, byte[] key, int offset,
int length);
public abstract int compareKey(KVComparator comparator, Cell kv);
}
/**
@ -837,8 +854,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
@Override
protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, byte[] nextIndexedKey,
boolean rewind, byte[] key, int offset, int length, boolean seekBefore)
throws IOException {
boolean rewind, Cell key, boolean seekBefore) throws IOException {
if (block == null || block.getOffset() != seekToBlock.getOffset()) {
updateCurrBlock(seekToBlock);
} else if (rewind) {
@ -847,7 +863,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
// Update the nextIndexedKey
this.nextIndexedKey = nextIndexedKey;
return blockSeek(key, offset, length, seekBefore);
return blockSeek(key, seekBefore);
}
/**
@ -911,26 +927,28 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
/**
* Within a loaded block, seek looking for the last key that is smaller
* than (or equal to?) the key we are interested in.
* Within a loaded block, seek looking for the last key that is smaller than
* (or equal to?) the key we are interested in.
*
* A note on the seekBefore: if you have seekBefore = true, AND the first
* key in the block = key, then you'll get thrown exceptions. The caller has
* to check for that case and load the previous block as appropriate.
*
* @param key the key to find
* @param seekBefore find the key before the given key in case of exact
* match.
* @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,
* -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)
* -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)
*/
protected int blockSeek(byte[] key, int offset, int length,
boolean seekBefore) {
protected int blockSeek(Cell key, boolean seekBefore) {
int klen, vlen;
long memstoreTS = 0;
int memstoreTSLen = 0;
int lastKeyValueSize = -1;
KeyValue.KeyOnlyKeyValue keyOnlykv = new KeyValue.KeyOnlyKeyValue();
do {
blockBuffer.mark();
klen = blockBuffer.getInt();
@ -939,10 +957,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
if (this.reader.shouldIncludeMemstoreTS()) {
if (this.reader.decodeMemstoreTS) {
try {
int memstoreTSOffset = blockBuffer.arrayOffset()
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE + klen + vlen;
memstoreTS = Bytes.readVLong(blockBuffer.array(),
memstoreTSOffset);
int memstoreTSOffset = blockBuffer.arrayOffset() + blockBuffer.position()
+ KEY_VALUE_LEN_SIZE + klen + vlen;
memstoreTS = Bytes.readVLong(blockBuffer.array(), memstoreTSOffset);
memstoreTSLen = WritableUtils.getVIntSize(memstoreTS);
} catch (Exception e) {
throw new RuntimeException("Error reading memstore timestamp", e);
@ -953,18 +970,18 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
}
int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position()
+ KEY_VALUE_LEN_SIZE;
int comp = reader.getComparator().compareFlatKey(key, offset, length,
blockBuffer.array(), keyOffset, klen);
int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position() + KEY_VALUE_LEN_SIZE;
keyOnlykv.setKey(blockBuffer.array(), keyOffset, klen);
int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlykv);
if (comp == 0) {
if (seekBefore) {
if (lastKeyValueSize < 0) {
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
throw new IllegalStateException("blockSeek with seekBefore "
+ "at the first key of the block: key="
+ Bytes.toStringBinary(key) + ", blockOffset="
+ block.getOffset() + ", onDiskSize="
+ Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
+ ", blockOffset=" + block.getOffset() + ", onDiskSize="
+ block.getOnDiskSizeWithHeader());
}
blockBuffer.position(blockBuffer.position() - lastKeyValueSize);
@ -1028,6 +1045,14 @@ public class HFileReaderV2 extends AbstractHFileReader {
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen,
currValueLen);
}
@Override
public int compareKey(KVComparator comparator, Cell key) {
return comparator.compareOnlyKeyPortion(
key,
new KeyValue.KeyOnlyKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen));
}
}
/**
@ -1187,15 +1212,19 @@ public class HFileReaderV2 extends AbstractHFileReader {
@Override
protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, byte[] nextIndexedKey,
boolean rewind, byte[] key, int offset, int length, boolean seekBefore)
throws IOException {
boolean rewind, Cell key, boolean seekBefore) throws IOException {
if (block == null || block.getOffset() != seekToBlock.getOffset()) {
updateCurrentBlock(seekToBlock);
} else if (rewind) {
seeker.rewind();
}
this.nextIndexedKey = nextIndexedKey;
return seeker.seekToKeyInBlock(key, offset, length, seekBefore);
return seeker.seekToKeyInBlock(key, seekBefore);
}
@Override
public int compareKey(KVComparator comparator, Cell key) {
return seeker.compareKey(comparator, key);
}
}

View File

@ -26,7 +26,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.crypto.Cipher;
@ -235,20 +238,18 @@ public class HFileReaderV3 extends HFileReaderV2 {
* the key to find
* @param seekBefore
* find the key before the given key in case of exact match.
* @param offset
* Offset to find the key in the given bytebuffer
* @param length
* Length of the key to be found
* @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)
*/
protected int blockSeek(byte[] key, int offset, int length, boolean seekBefore) {
@Override
protected int blockSeek(Cell key, boolean seekBefore) {
int klen, vlen, tlen = 0;
long memstoreTS = 0;
int memstoreTSLen = 0;
int lastKeyValueSize = -1;
KeyValue.KeyOnlyKeyValue keyOnlyKv = new KeyValue.KeyOnlyKeyValue();
do {
blockBuffer.mark();
klen = blockBuffer.getInt();
@ -286,14 +287,16 @@ public class HFileReaderV3 extends HFileReaderV2 {
}
blockBuffer.reset();
int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position() + (Bytes.SIZEOF_INT * 2);
int comp = reader.getComparator().compare(key, offset, length, blockBuffer.array(),
keyOffset, klen);
keyOnlyKv.setKey(blockBuffer.array(), keyOffset, klen);
int comp = reader.getComparator().compareOnlyKeyPortion(key, keyOnlyKv);
if (comp == 0) {
if (seekBefore) {
if (lastKeyValueSize < 0) {
KeyValue kv = KeyValueUtil.ensureKeyValue(key);
throw new IllegalStateException("blockSeek with seekBefore "
+ "at the first key of the block: key=" + Bytes.toStringBinary(key)
+ "at the first key of the block: key="
+ Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
+ ", blockOffset=" + block.getOffset() + ", onDiskSize="
+ block.getOnDiskSizeWithHeader());
}
@ -335,7 +338,6 @@ public class HFileReaderV3 extends HFileReaderV2 {
readKeyValueLen();
return 1; // didn't exactly find it.
}
}
/**

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
/**
@ -54,8 +55,12 @@ public interface HFileScanner {
* false when it is called.
* @throws IOException
*/
@Deprecated
int seekTo(byte[] key) throws IOException;
@Deprecated
int seekTo(byte[] key, int offset, int length) throws IOException;
int seekTo(Cell kv) throws IOException;
/**
* Reseek to or just before the passed <code>key</code>. Similar to seekTo
* except that this can be called even if the scanner is not at the beginning
@ -76,8 +81,12 @@ public interface HFileScanner {
* 1, such that k[i] < key, and scanner is left in position i.
* @throws IOException
*/
@Deprecated
int reseekTo(byte[] key) throws IOException;
@Deprecated
int reseekTo(byte[] key, int offset, int length) throws IOException;
int reseekTo(Cell kv) throws IOException;
/**
* Consider the key stream of all the keys in the file,
* <code>k[0] .. k[n]</code>, where there are n keys in the file.
@ -88,8 +97,12 @@ public interface HFileScanner {
* return false (EOF).
* @throws IOException
*/
@Deprecated
boolean seekBefore(byte[] key) throws IOException;
@Deprecated
boolean seekBefore(byte[] key, int offset, int length) throws IOException;
boolean seekBefore(Cell kv) throws IOException;
/**
* Positions this scanner at the start of the file.
* @return False if empty file; i.e. a call to next would return false and

View File

@ -1643,8 +1643,7 @@ public class HStore implements Store {
KeyValue kv = firstOnRow;
// If firstOnRow < firstKV, set to firstKV
if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
kv.getKeyLength());
int result = scanner.seekTo(kv);
return result != -1;
}

View File

@ -235,7 +235,7 @@ public class StoreFileScanner implements KeyValueScanner {
*/
public static boolean seekAtOrAfter(HFileScanner s, KeyValue k)
throws IOException {
int result = s.seekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
int result = s.seekTo(k);
if(result < 0) {
if (result == HConstants.INDEX_KEY_MAGIC) {
// using faked key
@ -255,7 +255,7 @@ public class StoreFileScanner implements KeyValueScanner {
static boolean reseekAtOrAfter(HFileScanner s, KeyValue k)
throws IOException {
//This function is similar to seekAtOrAfter function
int result = s.reseekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
int result = s.reseekTo(k);
if (result <= 0) {
if (result == HConstants.INDEX_KEY_MAGIC) {
// using faked key

View File

@ -29,8 +29,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.StoreFile;
/**
* Handles Bloom filter initialization based on configuration and serialized

View File

@ -19,12 +19,6 @@
package org.apache.hadoop.hbase.util;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.Writable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
@ -32,6 +26,11 @@ import java.nio.ByteBuffer;
import java.text.NumberFormat;
import java.util.Random;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.io.Writable;
/**
* Implements a <i>Bloom filter</i>, as defined by Bloom in 1970.
* <p>

View File

@ -89,7 +89,8 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
// testing, but when an error happens, we log a message and return.
boolean result;
int block = index.rootBlockContainingKey(key, keyOffset, keyLength);
int block = index.rootBlockContainingKey(key, keyOffset,
keyLength);
if (block < 0) {
result = false; // This key is not in the file.
} else {

View File

@ -33,7 +33,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -127,12 +126,12 @@ public class TestHalfStoreFileReader {
curr = scanner.getKeyValue();
KeyValue reseekKv =
getLastOnCol(curr);
int ret = scanner.reseekTo(reseekKv.getKey());
int ret = scanner.reseekTo(reseekKv);
assertTrue("reseek to returned: " + ret, ret > 0);
//System.out.println(curr + ": " + ret);
} while (scanner.next());
int ret = scanner.reseekTo(getLastOnCol(curr).getKey());
int ret = scanner.reseekTo(getLastOnCol(curr));
//System.out.println("Last reseek: " + ret);
assertTrue( ret > 0 );
@ -221,7 +220,7 @@ public class TestHalfStoreFileReader {
cacheConfig, bottom, TEST_UTIL.getConfiguration());
halfreader.loadFileInfo();
final HFileScanner scanner = halfreader.getScanner(false, false);
scanner.seekBefore(seekBefore.getKey());
scanner.seekBefore(seekBefore);
return scanner.getKeyValue();
}

View File

@ -358,10 +358,10 @@ public class TestDataBlockEncoders {
ByteBuffer expectedKeyValue = null;
ByteBuffer expectedKey = null;
ByteBuffer expectedValue = null;
for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
seeker.seekToKeyInBlock(keyValue.getBuffer(), keyValue.getKeyOffset(),
keyValue.getKeyLength(), seekBefore);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue
.getKeyLength()), seekBefore);
seeker.rewind();
ByteBuffer actualKeyValue = seeker.getKeyValueBuffer();

View File

@ -114,22 +114,25 @@ public class TestPrefixTreeEncoding {
// Seek before the first keyvalue;
KeyValue seekKey = KeyValue.createFirstDeleteFamilyOnRow(getRowKey(batchId, 0), CF_BYTES);
seeker.seekToKeyInBlock(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey.getKeyLength(),
true);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
.getKeyLength()), true);
assertEquals(null, seeker.getKeyValue());
// Seek before the middle keyvalue;
seekKey = KeyValue.createFirstDeleteFamilyOnRow(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3),
CF_BYTES);
seeker.seekToKeyInBlock(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey.getKeyLength(),
true);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
.getKeyLength()), true);
assertNotNull(seeker.getKeyValue());
assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH / 3 - 1), seeker.getKeyValue().getRow());
// Seek before the last keyvalue;
seekKey = KeyValue.createFirstDeleteFamilyOnRow(Bytes.toBytes("zzzz"), CF_BYTES);
seeker.seekToKeyInBlock(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey.getKeyLength(),
true);
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(seekKey.getBuffer(), seekKey.getKeyOffset(), seekKey
.getKeyLength()), true);
assertNotNull(seeker.getKeyValue());
assertArrayEquals(getRowKey(batchId, NUM_ROWS_PER_BATCH - 1), seeker.getKeyValue().getRow());
}
@ -221,8 +224,9 @@ public class TestPrefixTreeEncoding {
kvList.clear();
encodeSeeker.setCurrentBuffer(encodedData);
KeyValue firstOnRow = KeyValue.createFirstOnRow(getRowKey(batchId, i));
encodeSeeker.seekToKeyInBlock(firstOnRow.getBuffer(),
firstOnRow.getKeyOffset(), firstOnRow.getKeyLength(), false);
encodeSeeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
firstOnRow.getKeyLength()), false);
boolean hasMoreOfEncodeScanner = encodeSeeker.next();
CollectionBackedScanner collectionScanner = new CollectionBackedScanner(
this.kvset);

View File

@ -0,0 +1,325 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package org.apache.hadoop.hbase.io.encoding;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.test.RedundantKVGenerator;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestSeekToBlockWithEncoders {
private static int ENCODED_DATA_OFFSET = HConstants.HFILEBLOCK_HEADER_SIZE
+ DataBlockEncoding.ID_SIZE;
private HFileBlockEncodingContext getEncodingContext(Compression.Algorithm algo,
DataBlockEncoding encoding) {
DataBlockEncoder encoder = encoding.getEncoder();
HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(false)
.withIncludesTags(false).withCompression(algo).build();
if (encoder != null) {
return encoder
.newDataBlockEncodingContext(encoding, HConstants.HFILEBLOCK_DUMMY_HEADER, meta);
} else {
return new HFileBlockDefaultEncodingContext(encoding, HConstants.HFILEBLOCK_DUMMY_HEADER,
meta);
}
}
/**
* Test seeking while file is encoded.
*/
@Test
public void testSeekToBlockWithNonMatchingSeekKey() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv3);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
seekToTheKey(kv4, originalBuffer, toSeek);
}
/**
* Test seeking while file is encoded.
*/
@Test
public void testSeekingToBlockWithBiggerNonLength1() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv3);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("aaaad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
seekToTheKey(kv1, originalBuffer, toSeek);
}
/**
* Test seeking while file is encoded.
*/
@Test
public void testSeekingToBlockToANotAvailableKey() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv3);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aaae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"),
Bytes.toBytes("q1"), Bytes.toBytes("val"));
seekToTheKey(kv5, originalBuffer, toSeek);
}
/**
* Test seeking while file is encoded.
*/
@Test
public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
Bytes.toBytes("q1"), Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
Bytes.toBytes("q2"), Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
Bytes.toBytes("q3"), Bytes.toBytes("val"));
sampleKv.add(kv3);
KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"),
Bytes.toBytes("q1"), Bytes.toBytes("val"));
sampleKv.add(kv4);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = KeyValue.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
kv3.getRowLength(), null, 0, 0, null, 0, 0);
seekToTheKey(kv3, originalBuffer, toSeek);
}
@Test
public void testSeekToBlockWithDiffQualifer() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
Bytes.toBytes("val"));
seekToTheKey(kv5, originalBuffer, toSeek);
}
@Test
public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
Bytes.toBytes("val"));
sampleKv.add(kv6);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
Bytes.toBytes("val"));
seekToTheKey(kv6, originalBuffer, toSeek);
}
@Test
public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"),
Bytes.toBytes("val"));
sampleKv.add(kv6);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
Bytes.toBytes("val"));
seekToTheKey(kv5, originalBuffer, toSeek);
}
@Test
public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
Bytes.toBytes("val"));
sampleKv.add(kv6);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
Bytes.toBytes("val"));
seekToTheKey(kv6, originalBuffer, toSeek);
}
@Test
public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException {
List<KeyValue> sampleKv = new ArrayList<KeyValue>();
KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv1);
KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv2);
KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
Bytes.toBytes("val"));
sampleKv.add(kv4);
KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"),
Bytes.toBytes("val"));
sampleKv.add(kv5);
ByteBuffer originalBuffer = RedundantKVGenerator.convertKvToByteBuffer(sampleKv, false);
KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"),
Bytes.toBytes("q2"), Bytes.toBytes("val"));
seekToTheKey(kv5, originalBuffer, toSeek);
}
private void seekToTheKey(KeyValue expected, ByteBuffer originalBuffer, KeyValue toSeek)
throws IOException {
// create all seekers
List<DataBlockEncoder.EncodedSeeker> encodedSeekers =
new ArrayList<DataBlockEncoder.EncodedSeeker>();
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
if (encoding.getEncoder() == null || encoding == DataBlockEncoding.PREFIX_TREE) {
continue;
}
ByteBuffer encodedBuffer = ByteBuffer.wrap(encodeBytes(encoding, originalBuffer));
DataBlockEncoder encoder = encoding.getEncoder();
HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false)
.withIncludesMvcc(false).withIncludesTags(false)
.withCompression(Compression.Algorithm.NONE).build();
DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
encoder.newDataBlockDecodingContext(meta));
seeker.setCurrentBuffer(encodedBuffer);
encodedSeekers.add(seeker);
}
// test it!
// try a few random seeks
checkSeekingConsistency(encodedSeekers, toSeek, expected);
}
private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
KeyValue keyValue, KeyValue expected) {
for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
seeker.seekToKeyInBlock(
new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue
.getKeyLength()), false);
KeyValue keyValue2 = seeker.getKeyValue();
assertEquals(expected, keyValue2);
seeker.rewind();
}
}
private byte[] encodeBytes(DataBlockEncoding encoding, ByteBuffer dataset) throws IOException {
DataBlockEncoder encoder = encoding.getEncoder();
HFileBlockEncodingContext encodingCtx = getEncodingContext(Compression.Algorithm.NONE, encoding);
encoder.encodeKeyValues(dataset, encodingCtx);
byte[] encodedBytesWithHeader = encodingCtx.getUncompressedBytesWithHeader();
byte[] encodedData = new byte[encodedBytesWithHeader.length - ENCODED_DATA_OFFSET];
System.arraycopy(encodedBytesWithHeader, ENCODED_DATA_OFFSET, encodedData, 0,
encodedData.length);
return encodedData;
}
}

View File

@ -34,7 +34,9 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -155,13 +157,20 @@ public class TestHFile extends HBaseTestCase {
private int writeSomeRecords(Writer writer, int start, int n, boolean useTags)
throws IOException {
String value = "value";
KeyValue kv;
for (int i = start; i < (start + n); i++) {
String key = String.format(localFormatter, Integer.valueOf(i));
if (useTags) {
Tag t = new Tag((byte) 1, "myTag1");
writer.append(Bytes.toBytes(key), Bytes.toBytes(value + key), t.getBuffer());
Tag[] tags = new Tag[1];
tags[0] = t;
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
HConstants.LATEST_TIMESTAMP, Bytes.toBytes(value + key), tags);
writer.append(kv);
} else {
writer.append(Bytes.toBytes(key), Bytes.toBytes(value + key));
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
Bytes.toBytes(value + key));
writer.append(kv);
}
}
return (start + n);
@ -181,10 +190,13 @@ public class TestHFile extends HBaseTestCase {
ByteBuffer val = scanner.getValue();
String keyStr = String.format(localFormatter, Integer.valueOf(i));
String valStr = value + keyStr;
byte [] keyBytes = Bytes.toBytes(key);
KeyValue kv = new KeyValue(Bytes.toBytes(keyStr), Bytes.toBytes("family"),
Bytes.toBytes("qual"), Bytes.toBytes(valStr));
byte[] keyBytes = new KeyValue.KeyOnlyKeyValue(Bytes.toBytes(key), 0,
Bytes.toBytes(key).length).getKey();
assertTrue("bytes for keys do not match " + keyStr + " " +
Bytes.toString(Bytes.toBytes(key)),
Arrays.equals(Bytes.toBytes(keyStr), keyBytes));
Arrays.equals(kv.getKey(), keyBytes));
byte [] valBytes = Bytes.toBytes(val);
assertTrue("bytes for vals do not match " + valStr + " " +
Bytes.toString(valBytes),
@ -198,7 +210,9 @@ public class TestHFile extends HBaseTestCase {
}
private byte[] getSomeKey(int rowId) {
return String.format(localFormatter, Integer.valueOf(rowId)).getBytes();
KeyValue kv = new KeyValue(String.format(localFormatter, Integer.valueOf(rowId)).getBytes(),
Bytes.toBytes("family"), Bytes.toBytes("qual"), HConstants.LATEST_TIMESTAMP, Type.Put);
return kv.getKey();
}
private void writeRecords(Writer writer, boolean useTags) throws IOException {
@ -230,8 +244,7 @@ public class TestHFile extends HBaseTestCase {
Writer writer = HFile.getWriterFactory(conf, cacheConf)
.withOutputStream(fout)
.withFileContext(meta)
// NOTE: This test is dependent on this deprecated nonstandard comparator
.withComparator(new KeyValue.RawBytesComparator())
.withComparator(new KeyValue.KVComparator())
.create();
LOG.info(writer);
writeRecords(writer, useTags);
@ -247,16 +260,18 @@ public class TestHFile extends HBaseTestCase {
// Align scanner at start of the file.
scanner.seekTo();
readAllRecords(scanner);
scanner.seekTo(getSomeKey(50));
assertTrue("location lookup failed", scanner.seekTo(getSomeKey(50)) == 0);
int seekTo = scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(50)));
System.out.println(seekTo);
assertTrue("location lookup failed",
scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(50))) == 0);
// read the key and see if it matches
ByteBuffer readKey = scanner.getKey();
assertTrue("seeked key does not match", Arrays.equals(getSomeKey(50),
Bytes.toBytes(readKey)));
scanner.seekTo(new byte[0]);
scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(0)));
ByteBuffer val1 = scanner.getValue();
scanner.seekTo(new byte[0]);
scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(0)));
ByteBuffer val2 = scanner.getValue();
assertTrue(Arrays.equals(Bytes.toBytes(val1), Bytes.toBytes(val2)));

View File

@ -118,7 +118,7 @@ public class TestHFileBlockIndex {
fs = HFileSystem.get(conf);
}
@Test
//@Test
public void testBlockIndex() throws IOException {
testBlockIndexInternals(false);
clear();
@ -214,7 +214,8 @@ public class TestHFileBlockIndex {
for (byte[] key : keys) {
assertTrue(key != null);
assertTrue(indexReader != null);
HFileBlock b = indexReader.seekToDataBlock(key, 0, key.length, null,
HFileBlock b = indexReader.seekToDataBlock(new KeyValue.KeyOnlyKeyValue(key, 0, key.length),
null,
true, true, false, null);
if (Bytes.BYTES_RAWCOMPARATOR.compare(key, firstKeyInFile) < 0) {
assertTrue(b == null);
@ -331,7 +332,10 @@ public class TestHFileBlockIndex {
for (int i = 0; i < numTotalKeys; ++i) {
byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i * 2);
keys.add(k);
KeyValue cell = new KeyValue(k, Bytes.toBytes("f"), Bytes.toBytes("q"),
Bytes.toBytes("val"));
//KeyValue cell = new KeyValue.KeyOnlyKeyValue(k, 0, k.length);
keys.add(cell.getKey());
String msgPrefix = "Key #" + i + " (" + Bytes.toStringBinary(k) + "): ";
StringBuilder padding = new StringBuilder();
while (msgPrefix.length() + padding.length() < 70)
@ -342,7 +346,7 @@ public class TestHFileBlockIndex {
secondaryIndexEntries[i] = curAllEntriesSize;
LOG.info(msgPrefix + "secondary index entry #" + ((i - 1) / 2) +
", offset " + curAllEntriesSize);
curAllEntriesSize += k.length
curAllEntriesSize += cell.getKey().length
+ HFileBlockIndex.SECONDARY_INDEX_ENTRY_OVERHEAD;
++numEntriesAdded;
} else {
@ -353,8 +357,9 @@ public class TestHFileBlockIndex {
// Make sure the keys are increasing.
for (int i = 0; i < keys.size() - 1; ++i)
assertTrue(Bytes.BYTES_RAWCOMPARATOR.compare(keys.get(i),
keys.get(i + 1)) < 0);
assertTrue(KeyValue.COMPARATOR.compare(
new KeyValue.KeyOnlyKeyValue(keys.get(i), 0, keys.get(i).length),
new KeyValue.KeyOnlyKeyValue(keys.get(i + 1), 0, keys.get(i + 1).length)) < 0);
dos.writeInt(curAllEntriesSize);
assertEquals(numSearchedKeys, numEntriesAdded);
@ -388,9 +393,10 @@ public class TestHFileBlockIndex {
System.arraycopy(searchKey, 0, arrayHoldingKey, searchKey.length / 2,
searchKey.length);
int searchResult = BlockIndexReader.binarySearchNonRootIndex(
arrayHoldingKey, searchKey.length / 2, searchKey.length, nonRootIndex,
KeyValue.RAW_COMPARATOR);
KeyValue.KeyOnlyKeyValue cell = new KeyValue.KeyOnlyKeyValue(
arrayHoldingKey, searchKey.length / 2, searchKey.length);
int searchResult = BlockIndexReader.binarySearchNonRootIndex(cell,
nonRootIndex, KeyValue.COMPARATOR);
String lookupFailureMsg = "Failed to look up key #" + i + " ("
+ Bytes.toStringBinary(searchKey) + ")";
@ -415,8 +421,8 @@ public class TestHFileBlockIndex {
// Now test we can get the offset and the on-disk-size using a
// higher-level API function.s
boolean locateBlockResult =
(BlockIndexReader.locateNonRootIndexEntry(nonRootIndex, arrayHoldingKey,
searchKey.length / 2, searchKey.length, KeyValue.RAW_COMPARATOR) != -1);
(BlockIndexReader.locateNonRootIndexEntry(nonRootIndex, cell,
KeyValue.COMPARATOR) != -1);
if (i == 0) {
assertFalse(locateBlockResult);
@ -432,7 +438,7 @@ public class TestHFileBlockIndex {
}
@Test
//@Test
public void testBlockIndexChunk() throws IOException {
BlockIndexChunk c = new BlockIndexChunk();
ByteArrayOutputStream baos = new ByteArrayOutputStream();
@ -469,7 +475,7 @@ public class TestHFileBlockIndex {
}
/** Checks if the HeapSize calculator is within reason */
@Test
//@Test
public void testHeapSizeForBlockIndex() throws IOException {
Class<HFileBlockIndex.BlockIndexReader> cl =
HFileBlockIndex.BlockIndexReader.class;
@ -497,7 +503,7 @@ public class TestHFileBlockIndex {
*
* @throws IOException
*/
@Test
//@Test
public void testHFileWriterAndReader() throws IOException {
Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
"hfile_for_block_index");
@ -626,8 +632,8 @@ public class TestHFileBlockIndex {
private void checkSeekTo(byte[][] keys, HFileScanner scanner, int i)
throws IOException {
assertEquals("Failed to seek to key #" + i + " ("
+ Bytes.toStringBinary(keys[i]) + ")", 0, scanner.seekTo(keys[i]));
assertEquals("Failed to seek to key #" + i + " (" + Bytes.toStringBinary(keys[i]) + ")", 0,
scanner.seekTo(KeyValue.createKeyValueFromKey(keys[i])));
}
private void assertArrayEqualsBuffer(String msgPrefix, byte[] arr,

View File

@ -17,7 +17,10 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@ -44,7 +47,6 @@ import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.test.RedundantKVGenerator;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -177,7 +179,7 @@ public class TestHFileEncryption {
cryptoContext.getKeyBytes()));
}
@Test(timeout=60000)
@Test(timeout=6000000)
public void testHFileEncryption() throws Exception {
// Create 1000 random test KVs
RedundantKVGenerator generator = new RedundantKVGenerator();
@ -233,7 +235,7 @@ public class TestHFileEncryption {
assertTrue("Initial seekTo failed", scanner.seekTo());
for (i = 0; i < 100; i++) {
KeyValue kv = testKvs.get(RNG.nextInt(testKvs.size()));
assertEquals("Unable to find KV as expected: " + kv, scanner.seekTo(kv.getKey()), 0);
assertEquals("Unable to find KV as expected: " + kv, scanner.seekTo(kv), 0);
}
reader.close();
}

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
@ -80,7 +81,7 @@ public class TestHFileInlineToRootChunkConversion {
HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs, hfPath, cacheConf, conf);
HFileScanner scanner = reader.getScanner(true, true);
for (int i = 0; i < keys.size(); ++i) {
scanner.seekTo(keys.get(i));
scanner.seekTo(KeyValue.createKeyValueFromKey(keys.get(i)));
}
reader.close();
}

View File

@ -193,7 +193,7 @@ public class TestHFileSeek extends TestCase {
kSampler.next(key);
byte [] k = new byte [key.getLength()];
System.arraycopy(key.getBytes(), 0, k, 0, key.getLength());
if (scanner.seekTo(k) >= 0) {
if (scanner.seekTo(KeyValue.createKeyValueFromKey(k)) >= 0) {
ByteBuffer bbkey = scanner.getKey();
ByteBuffer bbval = scanner.getValue();
totalBytes += bbkey.limit();

View File

@ -63,7 +63,7 @@ public class TestReseekTo {
.withOutputStream(fout)
.withFileContext(context)
// NOTE: This test is dependent on this deprecated nonstandard comparator
.withComparator(new KeyValue.RawBytesComparator())
.withComparator(KeyValue.COMPARATOR)
.create();
int numberOfKeys = 1000;
@ -74,19 +74,32 @@ public class TestReseekTo {
for (int key = 0; key < numberOfKeys; key++) {
String value = valueString + key;
KeyValue kv;
keyList.add(key);
valueList.add(value);
if(tagUsage == TagUsage.NO_TAG){
writer.append(Bytes.toBytes(key), Bytes.toBytes(value));
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
Bytes.toBytes(value));
writer.append(kv);
} else if (tagUsage == TagUsage.ONLY_TAG) {
Tag t = new Tag((byte) 1, "myTag1");
writer.append(Bytes.toBytes(key), Bytes.toBytes(value), t.getBuffer());
Tag[] tags = new Tag[1];
tags[0] = t;
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
HConstants.LATEST_TIMESTAMP, Bytes.toBytes(value), tags);
writer.append(kv);
} else {
if (key % 4 == 0) {
Tag t = new Tag((byte) 1, "myTag1");
writer.append(Bytes.toBytes(key), Bytes.toBytes(value), t.getBuffer());
Tag[] tags = new Tag[1];
tags[0] = t;
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
HConstants.LATEST_TIMESTAMP, Bytes.toBytes(value), tags);
writer.append(kv);
} else {
writer.append(Bytes.toBytes(key), Bytes.toBytes(value), HConstants.EMPTY_BYTE_ARRAY);
kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
HConstants.LATEST_TIMESTAMP, Bytes.toBytes(value));
writer.append(kv);
}
}
}
@ -103,7 +116,8 @@ public class TestReseekTo {
Integer key = keyList.get(i);
String value = valueList.get(i);
long start = System.nanoTime();
scanner.seekTo(Bytes.toBytes(key));
scanner.seekTo(new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes
.toBytes("qual"), Bytes.toBytes(value)));
assertEquals(value, scanner.getValueString());
}
@ -112,7 +126,8 @@ public class TestReseekTo {
Integer key = keyList.get(i);
String value = valueList.get(i);
long start = System.nanoTime();
scanner.reseekTo(Bytes.toBytes(key));
scanner.reseekTo(new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes
.toBytes("qual"), Bytes.toBytes(value)));
assertEquals("i is " + i, value, scanner.getValueString());
}

View File

@ -80,9 +80,7 @@ public class TestSeekTo extends HBaseTestCase {
.withIncludesTags(true).build();
HFile.Writer writer = HFile.getWriterFactoryNoCache(conf).withOutputStream(fout)
.withFileContext(context)
// NOTE: This test is dependent on this deprecated nonstandard
// comparator
.withComparator(KeyValue.RAW_COMPARATOR).create();
.withComparator(KeyValue.COMPARATOR).create();
// 4 bytes * 3 * 2 for each key/value +
// 3 for keys, 15 for values = 42 (woot)
writer.append(toKV("c", tagUsage));
@ -107,31 +105,31 @@ public class TestSeekTo extends HBaseTestCase {
HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
reader.loadFileInfo();
HFileScanner scanner = reader.getScanner(false, true);
assertEquals(false, scanner.seekBefore(toKV("a", tagUsage).getKey()));
assertEquals(false, scanner.seekBefore(toKV("a", tagUsage)));
assertEquals(false, scanner.seekBefore(toKV("c", tagUsage).getKey()));
assertEquals(false, scanner.seekBefore(toKV("c", tagUsage)));
assertEquals(true, scanner.seekBefore(toKV("d", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("d", tagUsage)));
assertEquals("c", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("e", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("e", tagUsage)));
assertEquals("c", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("f", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("f", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("g", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("g", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("h", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("h", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("i", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("i", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("j", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("j", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("k", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("k", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
assertEquals(true, scanner.seekBefore(toKV("l", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("l", tagUsage)));
assertEquals("k", toRowStr(scanner.getKeyValue()));
reader.close();
@ -148,81 +146,81 @@ public class TestSeekTo extends HBaseTestCase {
HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
reader.loadFileInfo();
HFileScanner scanner = reader.getScanner(false, true);
assertEquals(false, scanner.seekBefore(toKV("a", tagUsage).getKey()));
assertEquals(false, scanner.seekBefore(toKV("b", tagUsage).getKey()));
assertEquals(false, scanner.seekBefore(toKV("c", tagUsage).getKey()));
assertEquals(false, scanner.seekBefore(toKV("a", tagUsage)));
assertEquals(false, scanner.seekBefore(toKV("b", tagUsage)));
assertEquals(false, scanner.seekBefore(toKV("c", tagUsage)));
// seekBefore d, so the scanner points to c
assertEquals(true, scanner.seekBefore(toKV("d", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("d", tagUsage)));
assertEquals("c", toRowStr(scanner.getKeyValue()));
// reseekTo e and g
assertEquals(0, scanner.reseekTo(toKV("c", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("c", tagUsage)));
assertEquals("c", toRowStr(scanner.getKeyValue()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// seekBefore e, so the scanner points to c
assertEquals(true, scanner.seekBefore(toKV("e", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("e", tagUsage)));
assertEquals("c", toRowStr(scanner.getKeyValue()));
// reseekTo e and g
assertEquals(0, scanner.reseekTo(toKV("e", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("e", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// seekBefore f, so the scanner points to e
assertEquals(true, scanner.seekBefore(toKV("f", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("f", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
// reseekTo e and g
assertEquals(0, scanner.reseekTo(toKV("e", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("e", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// seekBefore g, so the scanner points to e
assertEquals(true, scanner.seekBefore(toKV("g", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("g", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
// reseekTo e and g again
assertEquals(0, scanner.reseekTo(toKV("e", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("e", tagUsage)));
assertEquals("e", toRowStr(scanner.getKeyValue()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// seekBefore h, so the scanner points to g
assertEquals(true, scanner.seekBefore(toKV("h", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("h", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// reseekTo g
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// seekBefore i, so the scanner points to g
assertEquals(true, scanner.seekBefore(toKV("i", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("i", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// reseekTo g
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("g", tagUsage)));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// seekBefore j, so the scanner points to i
assertEquals(true, scanner.seekBefore(toKV("j", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("j", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
// reseekTo i
assertEquals(0, scanner.reseekTo(toKV("i", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("i", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
// seekBefore k, so the scanner points to i
assertEquals(true, scanner.seekBefore(toKV("k", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("k", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
// reseekTo i and k
assertEquals(0, scanner.reseekTo(toKV("i", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("i", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
assertEquals(0, scanner.reseekTo(toKV("k", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("k", tagUsage)));
assertEquals("k", toRowStr(scanner.getKeyValue()));
// seekBefore l, so the scanner points to k
assertEquals(true, scanner.seekBefore(toKV("l", tagUsage).getKey()));
assertEquals(true, scanner.seekBefore(toKV("l", tagUsage)));
assertEquals("k", toRowStr(scanner.getKeyValue()));
// reseekTo k
assertEquals(0, scanner.reseekTo(toKV("k", tagUsage).getKey()));
assertEquals(0, scanner.reseekTo(toKV("k", tagUsage)));
assertEquals("k", toRowStr(scanner.getKeyValue()));
}
@ -239,16 +237,17 @@ public class TestSeekTo extends HBaseTestCase {
assertEquals(2, reader.getDataBlockIndexReader().getRootBlockCount());
HFileScanner scanner = reader.getScanner(false, true);
// lies before the start of the file.
assertEquals(-1, scanner.seekTo(toKV("a", tagUsage).getKey()));
assertEquals(-1, scanner.seekTo(toKV("a", tagUsage)));
assertEquals(1, scanner.seekTo(toKV("d", tagUsage).getKey()));
assertEquals(1, scanner.seekTo(toKV("d", tagUsage)));
assertEquals("c", toRowStr(scanner.getKeyValue()));
// Across a block boundary now.
assertEquals(1, scanner.seekTo(toKV("h", tagUsage).getKey()));
assertEquals("g", toRowStr(scanner.getKeyValue()));
// h goes to the next block
assertEquals(-2, scanner.seekTo(toKV("h", tagUsage)));
assertEquals("i", toRowStr(scanner.getKeyValue()));
assertEquals(1, scanner.seekTo(toKV("l", tagUsage).getKey()));
assertEquals(1, scanner.seekTo(toKV("l", tagUsage)));
assertEquals("k", toRowStr(scanner.getKeyValue()));
reader.close();
@ -269,26 +268,25 @@ public class TestSeekTo extends HBaseTestCase {
int klen = toKV("a", tagUsage).getKey().length;
// falls before the start of the file.
assertEquals(-1, blockIndexReader.rootBlockContainingKey(
toKV("a", tagUsage).getKey(), 0, klen));
toKV("a", tagUsage)));
assertEquals(0, blockIndexReader.rootBlockContainingKey(
toKV("c", tagUsage).getKey(), 0, klen));
toKV("c", tagUsage)));
assertEquals(0, blockIndexReader.rootBlockContainingKey(
toKV("d", tagUsage).getKey(), 0, klen));
toKV("d", tagUsage)));
assertEquals(0, blockIndexReader.rootBlockContainingKey(
toKV("e", tagUsage).getKey(), 0, klen));
toKV("e", tagUsage)));
assertEquals(0, blockIndexReader.rootBlockContainingKey(
toKV("g", tagUsage).getKey(), 0, klen));
assertEquals(0, blockIndexReader.rootBlockContainingKey(
toKV("h", tagUsage).getKey(), 0, klen));
toKV("g", tagUsage)));
assertEquals(1, blockIndexReader.rootBlockContainingKey(
toKV("i", tagUsage).getKey(), 0, klen));
toKV("h", tagUsage)));
assertEquals(1, blockIndexReader.rootBlockContainingKey(
toKV("j", tagUsage).getKey(), 0, klen));
toKV("i", tagUsage)));
assertEquals(1, blockIndexReader.rootBlockContainingKey(
toKV("k", tagUsage).getKey(), 0, klen));
toKV("j", tagUsage)));
assertEquals(1, blockIndexReader.rootBlockContainingKey(
toKV("l", tagUsage).getKey(), 0, klen));
toKV("k", tagUsage)));
assertEquals(1, blockIndexReader.rootBlockContainingKey(
toKV("l", tagUsage)));
reader.close();
}