HBASE-10420 Replace KV.getBuffer with KV.get{Row|Family|Qualifier|Value|Tags}Array.
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1561410 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d967845f5b
commit
cf9fd55406
|
@ -86,7 +86,7 @@ public class Increment extends Mutation implements Comparable<Row> {
|
|||
List<Cell> list = getCellList(family);
|
||||
//Checking that the row of the kv is the same as the put
|
||||
int res = Bytes.compareTo(this.row, 0, row.length,
|
||||
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength());
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength());
|
||||
if (res != 0) {
|
||||
throw new WrongRowIOException("The row in " + kv.toString() +
|
||||
" doesn't match the original one " + Bytes.toStringBinary(this.row));
|
||||
|
|
|
@ -186,7 +186,7 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
foundColumn = true;
|
||||
if (filterColumnValue(keyValue.getBuffer(),
|
||||
if (filterColumnValue(keyValue.getValueArray(),
|
||||
keyValue.getValueOffset(), keyValue.getValueLength())) {
|
||||
return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
|
||||
}
|
||||
|
|
|
@ -248,13 +248,13 @@ public class TablePermission extends Permission {
|
|||
|
||||
if (family != null &&
|
||||
(Bytes.compareTo(family, 0, family.length,
|
||||
kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength()) != 0)) {
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength()) != 0)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (qualifier != null &&
|
||||
(Bytes.compareTo(qualifier, 0, qualifier.length,
|
||||
kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) != 0)) {
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()) != 0)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -1607,7 +1607,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
if (tagsLength == 0) {
|
||||
return EMPTY_ARRAY_LIST;
|
||||
}
|
||||
return Tag.asList(getBuffer(), getTagsOffset(), tagsLength);
|
||||
return Tag.asList(getTagsArray(), getTagsOffset(), tagsLength);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1650,7 +1650,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
}
|
||||
|
||||
public boolean matchingQualifier(final KeyValue other) {
|
||||
return matchingQualifier(other.getBuffer(), other.getQualifierOffset(),
|
||||
return matchingQualifier(other.getQualifierArray(), other.getQualifierOffset(),
|
||||
other.getQualifierLength());
|
||||
}
|
||||
|
||||
|
@ -1664,7 +1664,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
}
|
||||
|
||||
public boolean matchingRow(KeyValue other) {
|
||||
return matchingRow(other.getBuffer(), other.getRowOffset(),
|
||||
return matchingRow(other.getRowArray(), other.getRowOffset(),
|
||||
other.getRowLength());
|
||||
}
|
||||
|
||||
|
@ -2101,8 +2101,8 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
* @return Result comparing rows.
|
||||
*/
|
||||
public int compareRows(final KeyValue left, final KeyValue right) {
|
||||
return compareRows(left.getBuffer(),left.getRowOffset(), left.getRowLength(),
|
||||
right.getBuffer(), right.getRowOffset(), right.getRowLength());
|
||||
return compareRows(left.getRowArray(),left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2351,8 +2351,8 @@ public class KeyValue implements Cell, HeapSize, Cloneable {
|
|||
private boolean matchingRows(final KeyValue left, final short lrowlength,
|
||||
final KeyValue right, final short rrowlength) {
|
||||
return lrowlength == rrowlength &&
|
||||
matchingRows(left.getBuffer(), left.getRowOffset(), lrowlength,
|
||||
right.getBuffer(), right.getRowOffset(), rrowlength);
|
||||
matchingRows(left.getRowArray(), left.getRowOffset(), lrowlength,
|
||||
right.getRowArray(), right.getRowOffset(), rrowlength);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -253,7 +253,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
|
|||
@Override
|
||||
public void append(final KeyValue kv) throws IOException {
|
||||
append(kv.getMvccVersion(), kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
|
||||
kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
|
||||
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
this.maxMemstoreTS = Math.max(this.maxMemstoreTS, kv.getMvccVersion());
|
||||
}
|
||||
|
||||
|
|
|
@ -87,7 +87,7 @@ public class HFileWriterV3 extends HFileWriterV2 {
|
|||
public void append(final KeyValue kv) throws IOException {
|
||||
// Currently get the complete arrays
|
||||
append(kv.getMvccVersion(), kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
|
||||
kv.getBuffer(), kv.getValueOffset(), kv.getValueLength(), kv.getBuffer(),
|
||||
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(), kv.getTagsArray(),
|
||||
kv.getTagsOffset(), kv.getTagsLength());
|
||||
this.maxMemstoreTS = Math.max(this.maxMemstoreTS, kv.getMvccVersion());
|
||||
}
|
||||
|
|
|
@ -68,7 +68,7 @@ class GetClosestRowBeforeTracker {
|
|||
this.rowoffset = kv.getRowOffset();
|
||||
int l = -1;
|
||||
if (metaregion) {
|
||||
l = KeyValue.getDelimiter(kv.getBuffer(), rowoffset, kv.getRowLength(),
|
||||
l = KeyValue.getDelimiter(kv.getRowArray(), rowoffset, kv.getRowLength(),
|
||||
HConstants.DELIMITER) - this.rowoffset;
|
||||
}
|
||||
this.tablenamePlusDelimiterLength = metaregion? l + 1: -1;
|
||||
|
@ -147,9 +147,9 @@ class GetClosestRowBeforeTracker {
|
|||
continue;
|
||||
}
|
||||
// Check column
|
||||
int ret = Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(),
|
||||
int ret = Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(),
|
||||
d.getBuffer(), d.getQualifierOffset(), d.getQualifierLength());
|
||||
d.getQualifierArray(), d.getQualifierOffset(), d.getQualifierLength());
|
||||
if (ret <= -1) {
|
||||
// This delete is for an earlier column.
|
||||
continue;
|
||||
|
@ -233,8 +233,8 @@ class GetClosestRowBeforeTracker {
|
|||
if (!metaregion) return true;
|
||||
// Compare start of keys row. Compare including delimiter. Saves having
|
||||
// to calculate where tablename ends in the candidate kv.
|
||||
return Bytes.compareTo(this.targetkey.getBuffer(), this.rowoffset,
|
||||
return Bytes.compareTo(this.targetkey.getRowArray(), this.rowoffset,
|
||||
this.tablenamePlusDelimiterLength,
|
||||
kv.getBuffer(), kv.getRowOffset(), this.tablenamePlusDelimiterLength) == 0;
|
||||
kv.getRowArray(), kv.getRowOffset(), this.tablenamePlusDelimiterLength) == 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -3754,7 +3754,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
throws IOException {
|
||||
assert joinedContinuationRow != null;
|
||||
KeyValue kv = populateResult(results, this.joinedHeap, limit,
|
||||
joinedContinuationRow.getBuffer(), joinedContinuationRow.getRowOffset(),
|
||||
joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
|
||||
joinedContinuationRow.getRowLength());
|
||||
if (kv != KV_LIMIT) {
|
||||
// We are done with this row, reset the continuation.
|
||||
|
@ -3834,7 +3834,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
int offset = 0;
|
||||
short length = 0;
|
||||
if (current != null) {
|
||||
currentRow = current.getBuffer();
|
||||
currentRow = current.getRowArray();
|
||||
offset = current.getRowOffset();
|
||||
length = current.getRowLength();
|
||||
}
|
||||
|
@ -3871,7 +3871,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
stopRow = nextKv == null ||
|
||||
isStopRow(nextKv.getBuffer(), nextKv.getRowOffset(), nextKv.getRowLength());
|
||||
isStopRow(nextKv.getRowArray(), nextKv.getRowOffset(), nextKv.getRowLength());
|
||||
// save that the row was empty before filters applied to it.
|
||||
final boolean isEmptyRow = results.isEmpty();
|
||||
|
||||
|
@ -4993,17 +4993,17 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
oldKv.getValueLength() + kv.getValueLength(),
|
||||
oldKv.getTagsLength() + kv.getTagsLength());
|
||||
// copy in the value
|
||||
System.arraycopy(oldKv.getBuffer(), oldKv.getValueOffset(),
|
||||
newKV.getBuffer(), newKV.getValueOffset(),
|
||||
System.arraycopy(oldKv.getValueArray(), oldKv.getValueOffset(),
|
||||
newKV.getValueArray(), newKV.getValueOffset(),
|
||||
oldKv.getValueLength());
|
||||
System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
|
||||
newKV.getBuffer(),
|
||||
System.arraycopy(kv.getValueArray(), kv.getValueOffset(),
|
||||
newKV.getValueArray(),
|
||||
newKV.getValueOffset() + oldKv.getValueLength(),
|
||||
kv.getValueLength());
|
||||
// copy in the tags
|
||||
System.arraycopy(oldKv.getBuffer(), oldKv.getTagsOffset(), newKV.getBuffer(),
|
||||
System.arraycopy(oldKv.getTagsArray(), oldKv.getTagsOffset(), newKV.getTagsArray(),
|
||||
newKV.getTagsOffset(), oldKv.getTagsLength());
|
||||
System.arraycopy(kv.getBuffer(), kv.getTagsOffset(), newKV.getBuffer(),
|
||||
System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getTagsArray(),
|
||||
newKV.getTagsOffset() + oldKv.getTagsLength(), kv.getTagsLength());
|
||||
idx++;
|
||||
} else {
|
||||
|
@ -5012,21 +5012,21 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
kv.getQualifierLength(), now, KeyValue.Type.Put,
|
||||
kv.getValueLength(), kv.getTagsLength());
|
||||
// copy in the value
|
||||
System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
|
||||
newKV.getBuffer(), newKV.getValueOffset(),
|
||||
System.arraycopy(kv.getValueArray(), kv.getValueOffset(),
|
||||
newKV.getValueArray(), newKV.getValueOffset(),
|
||||
kv.getValueLength());
|
||||
// copy in tags
|
||||
System.arraycopy(kv.getBuffer(), kv.getTagsOffset(), newKV.getBuffer(),
|
||||
System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getTagsArray(),
|
||||
newKV.getTagsOffset(), kv.getTagsLength());
|
||||
}
|
||||
// copy in row, family, and qualifier
|
||||
System.arraycopy(kv.getBuffer(), kv.getRowOffset(),
|
||||
newKV.getBuffer(), newKV.getRowOffset(), kv.getRowLength());
|
||||
System.arraycopy(kv.getBuffer(), kv.getFamilyOffset(),
|
||||
newKV.getBuffer(), newKV.getFamilyOffset(),
|
||||
System.arraycopy(kv.getRowArray(), kv.getRowOffset(),
|
||||
newKV.getRowArray(), newKV.getRowOffset(), kv.getRowLength());
|
||||
System.arraycopy(kv.getFamilyArray(), kv.getFamilyOffset(),
|
||||
newKV.getFamilyArray(), newKV.getFamilyOffset(),
|
||||
kv.getFamilyLength());
|
||||
System.arraycopy(kv.getBuffer(), kv.getQualifierOffset(),
|
||||
newKV.getBuffer(), newKV.getQualifierOffset(),
|
||||
System.arraycopy(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
newKV.getQualifierArray(), newKV.getQualifierOffset(),
|
||||
kv.getQualifierLength());
|
||||
|
||||
newKV.setMvccVersion(w.getWriteNumber());
|
||||
|
@ -5200,19 +5200,19 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
int incCellTagsLen = kv.getTagsLength();
|
||||
KeyValue newKV = new KeyValue(row.length, family.getKey().length, q.length, now,
|
||||
KeyValue.Type.Put, val.length, oldCellTagsLen + incCellTagsLen);
|
||||
System.arraycopy(row, 0, newKV.getBuffer(), newKV.getRowOffset(), row.length);
|
||||
System.arraycopy(family.getKey(), 0, newKV.getBuffer(), newKV.getFamilyOffset(),
|
||||
System.arraycopy(row, 0, newKV.getRowArray(), newKV.getRowOffset(), row.length);
|
||||
System.arraycopy(family.getKey(), 0, newKV.getFamilyArray(), newKV.getFamilyOffset(),
|
||||
family.getKey().length);
|
||||
System.arraycopy(q, 0, newKV.getBuffer(), newKV.getQualifierOffset(), q.length);
|
||||
System.arraycopy(q, 0, newKV.getQualifierArray(), newKV.getQualifierOffset(), q.length);
|
||||
// copy in the value
|
||||
System.arraycopy(val, 0, newKV.getBuffer(), newKV.getValueOffset(), val.length);
|
||||
System.arraycopy(val, 0, newKV.getValueArray(), newKV.getValueOffset(), val.length);
|
||||
// copy tags
|
||||
if (oldCellTagsLen > 0) {
|
||||
System.arraycopy(c.getTagsArray(), c.getTagsOffset(), newKV.getBuffer(),
|
||||
System.arraycopy(c.getTagsArray(), c.getTagsOffset(), newKV.getTagsArray(),
|
||||
newKV.getTagsOffset(), oldCellTagsLen);
|
||||
}
|
||||
if (incCellTagsLen > 0) {
|
||||
System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getBuffer(),
|
||||
System.arraycopy(kv.getTagsArray(), kv.getTagsOffset(), newKV.getTagsArray(),
|
||||
newKV.getTagsOffset() + oldCellTagsLen, incCellTagsLen);
|
||||
}
|
||||
newKV.setMvccVersion(w.getWriteNumber());
|
||||
|
|
|
@ -629,16 +629,16 @@ public class HStore implements Store {
|
|||
do {
|
||||
KeyValue kv = scanner.getKeyValue();
|
||||
if (prevKV != null) {
|
||||
if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(),
|
||||
prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(),
|
||||
if (Bytes.compareTo(prevKV.getRowArray(), prevKV.getRowOffset(),
|
||||
prevKV.getRowLength(), kv.getRowArray(), kv.getRowOffset(),
|
||||
kv.getRowLength()) > 0) {
|
||||
throw new InvalidHFileException("Previous row is greater than"
|
||||
+ " current row: path=" + srcPath + " previous="
|
||||
+ Bytes.toStringBinary(prevKV.getKey()) + " current="
|
||||
+ Bytes.toStringBinary(kv.getKey()));
|
||||
}
|
||||
if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(),
|
||||
prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(),
|
||||
if (Bytes.compareTo(prevKV.getFamilyArray(), prevKV.getFamilyOffset(),
|
||||
prevKV.getFamilyLength(), kv.getFamilyArray(), kv.getFamilyOffset(),
|
||||
kv.getFamilyLength()) != 0) {
|
||||
throw new InvalidHFileException("Previous key had different"
|
||||
+ " family compared to current key: path=" + srcPath
|
||||
|
|
|
@ -550,9 +550,9 @@ public class MemStore implements HeapSize {
|
|||
// Get the KeyValues for the row/family/qualifier regardless of timestamp.
|
||||
// For this case we want to clean up any other puts
|
||||
KeyValue firstKv = KeyValue.createFirstOnRow(
|
||||
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
|
||||
Iterator<KeyValue> it = ss.iterator();
|
||||
// versions visible to oldest scanner
|
||||
|
|
|
@ -76,8 +76,8 @@ public class ReversedKeyValueHeap extends KeyValueHeap {
|
|||
KeyValueScanner scanner;
|
||||
while ((scanner = heap.poll()) != null) {
|
||||
KeyValue topKey = scanner.peek();
|
||||
if (comparator.getComparator().compareRows(topKey.getBuffer(),
|
||||
topKey.getRowOffset(), topKey.getRowLength(), seekKey.getBuffer(),
|
||||
if (comparator.getComparator().compareRows(topKey.getRowArray(),
|
||||
topKey.getRowOffset(), topKey.getRowLength(), seekKey.getRowArray(),
|
||||
seekKey.getRowOffset(), seekKey.getRowLength()) < 0) {
|
||||
// Row of Top KeyValue is before Seek row.
|
||||
heap.add(scanner);
|
||||
|
|
|
@ -467,7 +467,7 @@ public class ScanQueryMatcher {
|
|||
|
||||
public boolean moreRowsMayExistAfter(KeyValue kv) {
|
||||
if (this.isReversed) {
|
||||
if (rowComparator.compareRows(kv.getBuffer(), kv.getRowOffset(),
|
||||
if (rowComparator.compareRows(kv.getRowArray(), kv.getRowOffset(),
|
||||
kv.getRowLength(), stopRow, 0, stopRow.length) <= 0) {
|
||||
return false;
|
||||
} else {
|
||||
|
@ -475,7 +475,7 @@ public class ScanQueryMatcher {
|
|||
}
|
||||
}
|
||||
if (!Bytes.equals(stopRow , HConstants.EMPTY_END_ROW) &&
|
||||
rowComparator.compareRows(kv.getBuffer(),kv.getRowOffset(),
|
||||
rowComparator.compareRows(kv.getRowArray(),kv.getRowOffset(),
|
||||
kv.getRowLength(), stopRow, 0, stopRow.length) >= 0) {
|
||||
// KV >= STOPROW
|
||||
// then NO there is nothing left.
|
||||
|
@ -532,20 +532,20 @@ public class ScanQueryMatcher {
|
|||
ColumnCount nextColumn = columns.getColumnHint();
|
||||
if (nextColumn == null) {
|
||||
return KeyValue.createLastOnRow(
|
||||
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
} else {
|
||||
return KeyValue.createFirstOnRow(
|
||||
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
nextColumn.getBuffer(), nextColumn.getOffset(), nextColumn.getLength());
|
||||
}
|
||||
}
|
||||
|
||||
public KeyValue getKeyForNextRow(KeyValue kv) {
|
||||
return KeyValue.createLastOnRow(
|
||||
kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
null, 0, 0,
|
||||
null, 0, 0);
|
||||
}
|
||||
|
|
|
@ -843,7 +843,7 @@ public class StoreFile {
|
|||
|
||||
switch (bloomType) {
|
||||
case ROW:
|
||||
bloomKey = kv.getBuffer();
|
||||
bloomKey = kv.getRowArray();
|
||||
bloomKeyOffset = kv.getRowOffset();
|
||||
bloomKeyLen = kv.getRowLength();
|
||||
break;
|
||||
|
@ -851,8 +851,8 @@ public class StoreFile {
|
|||
// merge(row, qualifier)
|
||||
// TODO: could save one buffer copy in case of compound Bloom
|
||||
// filters when this involves creating a KeyValue
|
||||
bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
|
||||
kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
|
||||
bloomKey = generalBloomFilterWriter.createBloomKey(kv.getRowArray(),
|
||||
kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
|
||||
kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
bloomKeyOffset = 0;
|
||||
bloomKeyLen = bloomKey.length;
|
||||
|
@ -894,7 +894,7 @@ public class StoreFile {
|
|||
newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
|
||||
}
|
||||
if (newKey) {
|
||||
this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
|
||||
this.deleteFamilyBloomFilterWriter.add(kv.getRowArray(),
|
||||
kv.getRowOffset(), kv.getRowLength());
|
||||
this.lastDeleteFamilyKV = kv;
|
||||
}
|
||||
|
|
|
@ -197,8 +197,8 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
hfs.next();
|
||||
cur = hfs.getKeyValue();
|
||||
if (this.stopSkippingKVsIfNextRow
|
||||
&& Bytes.compareTo(cur.getBuffer(), cur.getRowOffset(),
|
||||
cur.getRowLength(), startKV.getBuffer(), startKV.getRowOffset(),
|
||||
&& Bytes.compareTo(cur.getRowArray(), cur.getRowOffset(),
|
||||
cur.getRowLength(), startKV.getRowArray(), startKV.getRowOffset(),
|
||||
startKV.getRowLength()) > 0) {
|
||||
return false;
|
||||
}
|
||||
|
@ -304,14 +304,14 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
if (useBloom) {
|
||||
// check ROWCOL Bloom filter first.
|
||||
if (reader.getBloomFilterType() == BloomType.ROWCOL) {
|
||||
haveToSeek = reader.passesGeneralBloomFilter(kv.getBuffer(),
|
||||
kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
|
||||
haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(),
|
||||
kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
|
||||
kv.getQualifierOffset(), kv.getQualifierLength());
|
||||
} else if (this.matcher != null && !matcher.hasNullColumnInQuery() &&
|
||||
(kv.isDeleteFamily() || kv.isDeleteFamilyVersion())) {
|
||||
// if there is no such delete family kv in the store file,
|
||||
// then no need to seek.
|
||||
haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getBuffer(),
|
||||
haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(),
|
||||
kv.getRowOffset(), kv.getRowLength());
|
||||
}
|
||||
}
|
||||
|
@ -465,8 +465,8 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
public boolean backwardSeek(KeyValue key) throws IOException {
|
||||
seek(key);
|
||||
if (cur == null
|
||||
|| Bytes.compareTo(cur.getBuffer(), cur.getRowOffset(),
|
||||
cur.getRowLength(), key.getBuffer(), key.getRowOffset(),
|
||||
|| Bytes.compareTo(cur.getRowArray(), cur.getRowOffset(),
|
||||
cur.getRowLength(), key.getRowArray(), key.getRowOffset(),
|
||||
key.getRowLength()) > 0) {
|
||||
return seekToPreviousRow(key);
|
||||
}
|
||||
|
|
|
@ -447,7 +447,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
|
||||
// only call setRow if the row changes; avoids confusing the query matcher
|
||||
// if scanning intra-row
|
||||
byte[] row = peeked.getBuffer();
|
||||
byte[] row = peeked.getRowArray();
|
||||
int offset = peeked.getRowOffset();
|
||||
short length = peeked.getRowLength();
|
||||
if (limit < 0 || matcher.row == null || !Bytes.equals(row, offset, length, matcher.row,
|
||||
|
@ -644,7 +644,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
if (kv == null) {
|
||||
kv = lastTopKey;
|
||||
}
|
||||
byte[] row = kv.getBuffer();
|
||||
byte[] row = kv.getRowArray();
|
||||
int offset = kv.getRowOffset();
|
||||
short length = kv.getRowLength();
|
||||
if ((matcher.row == null) || !Bytes.equals(row, offset, length, matcher.row,
|
||||
|
|
|
@ -201,7 +201,7 @@ public class AccessController extends BaseRegionObserver
|
|||
List<Cell> cells = f.getValue();
|
||||
for (Cell cell: cells) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
if (Bytes.equals(kv.getBuffer(), kv.getFamilyOffset(),
|
||||
if (Bytes.equals(kv.getFamilyArray(), kv.getFamilyOffset(),
|
||||
kv.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
|
||||
AccessControlLists.ACL_LIST_FAMILY.length)) {
|
||||
entries.add(kv.getRow());
|
||||
|
@ -610,13 +610,12 @@ public class AccessController extends BaseRegionObserver
|
|||
// Ensure KeyValue so we can do a scatter gather copy. This is only a win if the
|
||||
// incoming cell type is actually KeyValue.
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
byte[] bytes = kv.getBuffer();
|
||||
newCells.add(
|
||||
new KeyValue(bytes, kv.getRowOffset(), kv.getRowLength(),
|
||||
bytes, kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
bytes, kv.getQualifierOffset(), kv.getQualifierLength(),
|
||||
new KeyValue(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(),
|
||||
kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(),
|
||||
kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
|
||||
kv.getTimestamp(), KeyValue.Type.codeToType(kv.getTypeByte()),
|
||||
bytes, kv.getValueOffset(), kv.getValueLength(),
|
||||
kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
|
||||
tags));
|
||||
}
|
||||
// This is supposed to be safe, won't CME
|
||||
|
@ -1349,12 +1348,11 @@ public class AccessController extends BaseRegionObserver
|
|||
// We need to create another KV, unfortunately, because the current new KV
|
||||
// has no space for tags
|
||||
KeyValue newKv = KeyValueUtil.ensureKeyValue(newCell);
|
||||
byte[] bytes = newKv.getBuffer();
|
||||
KeyValue rewriteKv = new KeyValue(bytes, newKv.getRowOffset(), newKv.getRowLength(),
|
||||
bytes, newKv.getFamilyOffset(), newKv.getFamilyLength(),
|
||||
bytes, newKv.getQualifierOffset(), newKv.getQualifierLength(),
|
||||
KeyValue rewriteKv = new KeyValue(newKv.getRowArray(), newKv.getRowOffset(), newKv.getRowLength(),
|
||||
newKv.getFamilyArray(), newKv.getFamilyOffset(), newKv.getFamilyLength(),
|
||||
newKv.getQualifierArray(), newKv.getQualifierOffset(), newKv.getQualifierLength(),
|
||||
newKv.getTimestamp(), KeyValue.Type.codeToType(newKv.getTypeByte()),
|
||||
bytes, newKv.getValueOffset(), newKv.getValueLength(),
|
||||
newKv.getValueArray(), newKv.getValueOffset(), newKv.getValueLength(),
|
||||
tags);
|
||||
// Preserve mvcc data
|
||||
rewriteKv.setMvccVersion(newKv.getMvccVersion());
|
||||
|
|
|
@ -1056,11 +1056,11 @@ public class VisibilityController extends BaseRegionObserver implements MasterOb
|
|||
// We need to create another KV, unfortunately, because the current new KV
|
||||
// has no space for tags
|
||||
KeyValue newKv = KeyValueUtil.ensureKeyValue(newCell);
|
||||
byte[] bytes = newKv.getBuffer();
|
||||
KeyValue rewriteKv = new KeyValue(bytes, newKv.getRowOffset(), newKv.getRowLength(), bytes,
|
||||
newKv.getFamilyOffset(), newKv.getFamilyLength(), bytes, newKv.getQualifierOffset(),
|
||||
newKv.getQualifierLength(), newKv.getTimestamp(), KeyValue.Type.codeToType(newKv
|
||||
.getTypeByte()), bytes, newKv.getValueOffset(), newKv.getValueLength(), tags);
|
||||
KeyValue rewriteKv = new KeyValue(newKv.getRowArray(), newKv.getRowOffset(), newKv.getRowLength(),
|
||||
newKv.getFamilyArray(), newKv.getFamilyOffset(), newKv.getFamilyLength(),
|
||||
newKv.getQualifierArray(), newKv.getQualifierOffset(), newKv.getQualifierLength(),
|
||||
newKv.getTimestamp(), KeyValue.Type.codeToType(newKv.getTypeByte()),
|
||||
newKv.getValueArray(), newKv.getValueOffset(), newKv.getValueLength(), tags);
|
||||
// Preserve mvcc data
|
||||
rewriteKv.setMvccVersion(newKv.getMvccVersion());
|
||||
return rewriteKv;
|
||||
|
|
|
@ -108,7 +108,7 @@ implements WALObserver {
|
|||
if (Arrays.equals(family, changedFamily) &&
|
||||
Arrays.equals(qulifier, changedQualifier)) {
|
||||
LOG.debug("Found the KeyValue from WALEdit which should be changed.");
|
||||
kv.getBuffer()[kv.getValueOffset()] += 1;
|
||||
kv.getValueArray()[kv.getValueOffset()] += 1;
|
||||
}
|
||||
}
|
||||
kvs.add(new KeyValue(row, addedFamily, addedQualifier));
|
||||
|
|
|
@ -227,9 +227,9 @@ public class TestHalfStoreFileReader {
|
|||
|
||||
private KeyValue getLastOnCol(KeyValue curr) {
|
||||
return KeyValue.createLastOnRow(
|
||||
curr.getBuffer(), curr.getRowOffset(), curr.getRowLength(),
|
||||
curr.getBuffer(), curr.getFamilyOffset(), curr.getFamilyLength(),
|
||||
curr.getBuffer(), curr.getQualifierOffset(), curr.getQualifierLength());
|
||||
curr.getRowArray(), curr.getRowOffset(), curr.getRowLength(),
|
||||
curr.getFamilyArray(), curr.getFamilyOffset(), curr.getFamilyLength(),
|
||||
curr.getQualifierArray(), curr.getQualifierOffset(), curr.getQualifierLength());
|
||||
}
|
||||
|
||||
static final int SIZE = 1000;
|
||||
|
|
|
@ -277,7 +277,7 @@ public class TestPrefixTreeEncoding {
|
|||
userDataStream.writeInt(kv.getKeyLength());
|
||||
userDataStream.writeInt(kv.getValueLength());
|
||||
userDataStream.write(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
|
||||
userDataStream.write(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
|
||||
userDataStream.write(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
if (useTags) {
|
||||
userDataStream.writeShort(kv.getTagsLength());
|
||||
userDataStream.write(kv.getBuffer(), kv.getValueOffset() + kv.getValueLength()
|
||||
|
@ -314,7 +314,7 @@ public class TestPrefixTreeEncoding {
|
|||
userDataStream.writeInt(kv.getKeyLength());
|
||||
userDataStream.writeInt(kv.getValueLength());
|
||||
userDataStream.write(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
|
||||
userDataStream.write(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
|
||||
userDataStream.write(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
if (useTags) {
|
||||
userDataStream.writeShort(kv.getTagsLength());
|
||||
userDataStream.write(kv.getBuffer(), kv.getValueOffset() + kv.getValueLength()
|
||||
|
|
|
@ -184,13 +184,13 @@ public class TestHFileBlock {
|
|||
dataOutputStream.writeInt(kv.getKeyLength());
|
||||
dataOutputStream.writeInt(kv.getValueLength());
|
||||
dataOutputStream.write(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
|
||||
dataOutputStream.write(kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
|
||||
dataOutputStream.write(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
|
||||
// Write the additonal tag into the stream
|
||||
// always write the taglength
|
||||
totalSize += kv.getLength();
|
||||
if (useTag) {
|
||||
dataOutputStream.writeShort(kv.getTagsLength());
|
||||
dataOutputStream.write(kv.getBuffer(), kv.getTagsOffset(), kv.getTagsLength());
|
||||
dataOutputStream.write(kv.getTagsArray(), kv.getTagsOffset(), kv.getTagsLength());
|
||||
}
|
||||
if (includesMemstoreTS) {
|
||||
long memstoreTS = randomizer.nextLong();
|
||||
|
|
|
@ -258,7 +258,7 @@ public class TestHFileWriterV3 {
|
|||
assertNotNull(tagValue);
|
||||
KeyValue tkv = keyValues.get(entriesRead);
|
||||
assertEquals(tagValue.length, tkv.getTagsLength());
|
||||
assertTrue(Bytes.compareTo(tagValue, 0, tagValue.length, tkv.getBuffer(),
|
||||
assertTrue(Bytes.compareTo(tagValue, 0, tagValue.length, tkv.getTagsArray(),
|
||||
tkv.getTagsOffset(), tkv.getTagsLength()) == 0);
|
||||
}
|
||||
++entriesRead;
|
||||
|
|
|
@ -303,7 +303,7 @@ public class TestMultiColumnScanner {
|
|||
}
|
||||
|
||||
private static String qualStr(KeyValue kv) {
|
||||
return Bytes.toString(kv.getBuffer(), kv.getQualifierOffset(),
|
||||
return Bytes.toString(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength());
|
||||
}
|
||||
|
||||
|
|
|
@ -120,7 +120,7 @@ public class TestQueryMatcher extends HBaseTestCase {
|
|||
|
||||
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
|
||||
KeyValue k = memstore.get(0);
|
||||
qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
|
||||
qm.setRow(k.getRowArray(), k.getRowOffset(), k.getRowLength());
|
||||
|
||||
for (KeyValue kv : memstore){
|
||||
actual.add(qm.match(kv));
|
||||
|
@ -166,7 +166,7 @@ public class TestQueryMatcher extends HBaseTestCase {
|
|||
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
|
||||
|
||||
KeyValue k = memstore.get(0);
|
||||
qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
|
||||
qm.setRow(k.getRowArray(), k.getRowOffset(), k.getRowLength());
|
||||
|
||||
for(KeyValue kv : memstore) {
|
||||
actual.add(qm.match(kv));
|
||||
|
@ -219,7 +219,7 @@ public class TestQueryMatcher extends HBaseTestCase {
|
|||
};
|
||||
|
||||
KeyValue k = kvs[0];
|
||||
qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
|
||||
qm.setRow(k.getRowArray(), k.getRowOffset(), k.getRowLength());
|
||||
|
||||
List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
|
||||
for (KeyValue kv : kvs) {
|
||||
|
@ -272,7 +272,7 @@ public class TestQueryMatcher extends HBaseTestCase {
|
|||
new KeyValue(row2, fam1, col1, now-10, data)
|
||||
};
|
||||
KeyValue k = kvs[0];
|
||||
qm.setRow(k.getBuffer(), k.getRowOffset(), k.getRowLength());
|
||||
qm.setRow(k.getRowArray(), k.getRowOffset(), k.getRowLength());
|
||||
|
||||
List<ScanQueryMatcher.MatchCode> actual =
|
||||
new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);
|
||||
|
|
|
@ -891,8 +891,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
kv1.getBuffer(), kv1.getKeyOffset(), kv1.getKeyLength(),
|
||||
kv2.getBuffer(), kv2.getKeyOffset(), kv2.getKeyLength()) == 0);
|
||||
assertTrue(Bytes.compareTo(
|
||||
kv1.getBuffer(), kv1.getValueOffset(), kv1.getValueLength(),
|
||||
kv2.getBuffer(), kv2.getValueOffset(), kv2.getValueLength()) == 0);
|
||||
kv1.getValueArray(), kv1.getValueOffset(), kv1.getValueLength(),
|
||||
kv2.getValueArray(), kv2.getValueOffset(), kv2.getValueLength()) == 0);
|
||||
}
|
||||
assertNull(scannerTwo.next());
|
||||
assertEquals(startHit + 6, cs.getHitCount());
|
||||
|
|
Loading…
Reference in New Issue