HBASE-17644 Always create ByteBufferCells after copying to MSLAB.
This commit is contained in:
parent
3fc2889f7a
commit
7763dd6688
|
@ -26,38 +26,46 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* This Cell is an implementation of {@link ByteBufferCell} where the data resides in off heap
|
||||
* memory.
|
||||
* This Cell is an implementation of {@link ByteBufferCell} where the data resides in
|
||||
* off heap/ on heap ByteBuffer
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell {
|
||||
public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
|
||||
|
||||
protected final ByteBuffer buf;
|
||||
protected final int offset;
|
||||
protected final int length;
|
||||
private long seqId = 0;
|
||||
// TODO : See if famLen can be cached or not?
|
||||
|
||||
private static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
|
||||
+ (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT
|
||||
+ Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_LONG;
|
||||
+ (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_LONG;
|
||||
|
||||
public OffheapKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
assert buf.isDirect();
|
||||
public ByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
this.buf = buf;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
this.seqId = seqId;
|
||||
}
|
||||
|
||||
public OffheapKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
assert buf.isDirect();
|
||||
public ByteBufferKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
this.buf = buf;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ByteBuffer getBuffer() {
|
||||
return this.buf;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int getOffset() {
|
||||
return this.offset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getRowArray() {
|
||||
return CellUtil.cloneRow(this);
|
|
@ -26,18 +26,18 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
|
||||
/**
|
||||
* This is a {@link Tag} implementation in which value is backed by an off heap
|
||||
* This is a {@link Tag} implementation in which value is backed by
|
||||
* {@link java.nio.ByteBuffer}
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class OffheapTag implements Tag {
|
||||
public class ByteBufferTag implements Tag {
|
||||
|
||||
private ByteBuffer buffer;
|
||||
private int offset, length;
|
||||
private byte type;
|
||||
|
||||
public OffheapTag(ByteBuffer buffer, int offset, int length) {
|
||||
public ByteBufferTag(ByteBuffer buffer, int offset, int length) {
|
||||
this.buffer = buffer;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
|
@ -1539,7 +1539,7 @@ public final class CellUtil {
|
|||
public Tag next() {
|
||||
if (hasNext()) {
|
||||
int curTagLen = ByteBufferUtils.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
|
||||
Tag tag = new OffheapTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
|
||||
Tag tag = new ByteBufferTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
|
||||
this.pos += Bytes.SIZEOF_SHORT + curTagLen;
|
||||
return tag;
|
||||
}
|
||||
|
@ -1622,7 +1622,7 @@ public final class CellUtil {
|
|||
ByteBuffer tagsBuffer = ((ByteBufferCell)cell).getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return new OffheapTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE);
|
||||
return new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE);
|
||||
}
|
||||
} else {
|
||||
tagLen = Bytes.readAsInt(cell.getTagsArray(), pos, TAG_LENGTH_SIZE);
|
||||
|
@ -3188,20 +3188,14 @@ public final class CellUtil {
|
|||
// serialization format only.
|
||||
KeyValueUtil.appendTo(cell, buf, offset, true);
|
||||
}
|
||||
if (buf.hasArray()) {
|
||||
KeyValue newKv;
|
||||
if (tagsLen == 0) {
|
||||
// When tagsLen is 0, make a NoTagsKeyValue version of Cell. This is an optimized class
|
||||
// which directly return tagsLen as 0. So we avoid parsing many length components in
|
||||
// reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
|
||||
// call getTagsLength().
|
||||
newKv = new NoTagsKeyValue(buf.array(), buf.arrayOffset() + offset, len);
|
||||
} else {
|
||||
newKv = new KeyValue(buf.array(), buf.arrayOffset() + offset, len);
|
||||
}
|
||||
newKv.setSequenceId(cell.getSequenceId());
|
||||
return newKv;
|
||||
if (tagsLen == 0) {
|
||||
// When tagsLen is 0, make a NoTagsByteBufferKeyValue version. This is an optimized class
|
||||
// which directly return tagsLen as 0. So we avoid parsing many length components in
|
||||
// reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
|
||||
// call getTagsLength().
|
||||
return new NoTagsByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
|
||||
} else {
|
||||
return new ByteBufferKeyValue(buf, offset, len, cell.getSequenceId());
|
||||
}
|
||||
return new OffheapKeyValue(buf, offset, len, cell.getSequenceId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,62 @@
|
|||
/**
|
||||
* 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 java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
|
||||
/**
|
||||
* An extension of the ByteBufferKeyValue where the tags length is always 0
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class NoTagsByteBufferKeyValue extends ByteBufferKeyValue {
|
||||
|
||||
public NoTagsByteBufferKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
super(buf, offset, length);
|
||||
}
|
||||
|
||||
public NoTagsByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
super(buf, offset, length, seqId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getTagsArray() {
|
||||
return HConstants.EMPTY_BYTE_ARRAY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTagsLength() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSerializedSize(boolean withTags) {
|
||||
return this.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell deepClone() {
|
||||
byte[] copy = new byte[this.length];
|
||||
ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length);
|
||||
KeyValue kv = new NoTagsKeyValue(copy, 0, copy.length);
|
||||
kv.setSequenceId(this.getSequenceId());
|
||||
return kv;
|
||||
}
|
||||
}
|
|
@ -95,7 +95,7 @@ public final class TagUtil {
|
|||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen = ByteBufferUtils.readAsInt(b, pos, TAG_LENGTH_SIZE);
|
||||
tags.add(new OffheapTag(b, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
tags.add(new ByteBufferTag(b, pos, tagLen + TAG_LENGTH_SIZE));
|
||||
pos += TAG_LENGTH_SIZE + tagLen;
|
||||
}
|
||||
return tags;
|
||||
|
|
|
@ -25,8 +25,8 @@ import java.nio.ByteBuffer;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.NoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
|
@ -111,7 +111,7 @@ public class KeyValueCodec implements Codec {
|
|||
|
||||
protected Cell createCell(ByteBuffer bb, int pos, int len) {
|
||||
// We know there is not going to be any tags.
|
||||
return new OffheapKeyValue(bb, pos, len, 0);
|
||||
return new NoTagsByteBufferKeyValue(bb, pos, len);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
|
@ -94,7 +94,7 @@ public class KeyValueCodecWithTags implements Codec {
|
|||
|
||||
@Override
|
||||
protected Cell createCell(ByteBuffer bb, int pos, int len) {
|
||||
return new OffheapKeyValue(bb, pos, len);
|
||||
return new ByteBufferKeyValue(bb, pos, len);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -382,7 +382,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
currentBuffer.asSubByteBuffer(startOffset, cellBufSize, tmpPair);
|
||||
ByteBuffer buf = tmpPair.getFirst();
|
||||
if (buf.isDirect()) {
|
||||
ret = new OffheapKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId);
|
||||
ret = new ByteBufferKeyValue(buf, tmpPair.getSecond(), cellBufSize, seqId);
|
||||
} else {
|
||||
if (tagsLength > 0) {
|
||||
ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset()
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
|
@ -370,7 +370,7 @@ public class RedundantKVGenerator {
|
|||
ByteBuffer offheapKVBB = ByteBuffer.allocateDirect(keyValue.getLength());
|
||||
ByteBufferUtils.copyFromArrayToBuffer(offheapKVBB, keyValue.getBuffer(),
|
||||
keyValue.getOffset(), keyValue.getLength());
|
||||
OffheapKeyValue offheapKV =
|
||||
ByteBufferKeyValue offheapKV =
|
||||
new ExtendedOffheapKeyValue(offheapKVBB, 0, keyValue.getLength(), 0);
|
||||
result.add(offheapKV);
|
||||
} else {
|
||||
|
@ -378,7 +378,7 @@ public class RedundantKVGenerator {
|
|||
ByteBuffer offheapKVBB = ByteBuffer.allocateDirect(keyValue.getLength());
|
||||
ByteBufferUtils.copyFromArrayToBuffer(offheapKVBB, keyValue.getBuffer(),
|
||||
keyValue.getOffset(), keyValue.getLength());
|
||||
OffheapKeyValue offheapKV =
|
||||
ByteBufferKeyValue offheapKV =
|
||||
new ExtendedOffheapKeyValue(offheapKVBB, 0, keyValue.getLength(), 0);
|
||||
result.add(offheapKV);
|
||||
}
|
||||
|
@ -389,7 +389,7 @@ public class RedundantKVGenerator {
|
|||
return result;
|
||||
}
|
||||
|
||||
static class ExtendedOffheapKeyValue extends OffheapKeyValue {
|
||||
static class ExtendedOffheapKeyValue extends ByteBufferKeyValue {
|
||||
public ExtendedOffheapKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
|
||||
super(buf, offset, length, seqId);
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({ MiscTests.class, SmallTests.class })
|
||||
public class TestOffheapKeyValue {
|
||||
public class TestByteBufferKeyValue {
|
||||
private static final String QUAL2 = "qual2";
|
||||
private static final String FAM2 = "fam2";
|
||||
private static final String QUAL1 = "qual1";
|
||||
|
@ -56,7 +56,7 @@ public class TestOffheapKeyValue {
|
|||
KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
|
||||
|
@ -99,7 +99,7 @@ public class TestOffheapKeyValue {
|
|||
kvCell = new KeyValue(row1, fam2, qual2, 0l, Type.Put, row1);
|
||||
buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
assertEquals(
|
||||
FAM2,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getFamilyByteBuffer(),
|
||||
|
@ -112,7 +112,7 @@ public class TestOffheapKeyValue {
|
|||
kvCell = new KeyValue(row1, fam1, nullQualifier, 0L, Type.Put, row1);
|
||||
buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
|
||||
|
@ -138,7 +138,7 @@ public class TestOffheapKeyValue {
|
|||
KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1, tags);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
|
|
@ -27,7 +27,7 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
|
@ -166,7 +166,7 @@ public class TestTagCompressionContext {
|
|||
KeyValue kv = new KeyValue(ROW, CF, Q, 1234L, V, tags);
|
||||
ByteBuffer dbb = ByteBuffer.allocateDirect(kv.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(dbb, kv.getBuffer(), 0, kv.getBuffer().length);
|
||||
OffheapKeyValue offheapKV = new OffheapKeyValue(dbb, 0, kv.getBuffer().length, 0);
|
||||
ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, 0, kv.getBuffer().length, 0);
|
||||
return offheapKV;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedKeyValue;
|
||||
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -957,7 +957,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
} else {
|
||||
ByteBuffer buf = blockBuffer.asSubByteBuffer(cellBufSize);
|
||||
if (buf.isDirect()) {
|
||||
ret = new OffheapKeyValue(buf, buf.position(), cellBufSize, seqId);
|
||||
ret = new ByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId);
|
||||
} else {
|
||||
if (currTagsLen > 0) {
|
||||
ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset() + buf.position(),
|
||||
|
|
|
@ -385,23 +385,23 @@ public class SimpleRegionObserver extends BaseRegionObserver {
|
|||
List<Cell> cells = familyMap.get(TestRegionObserverInterface.A);
|
||||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
KeyValue kv = (KeyValue)cells.get(0);
|
||||
assertTrue(Bytes.equals(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(), TestRegionObserverInterface.A, 0,
|
||||
Cell cell = cells.get(0);
|
||||
assertTrue(Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), TestRegionObserverInterface.A, 0,
|
||||
TestRegionObserverInterface.A.length));
|
||||
cells = familyMap.get(TestRegionObserverInterface.B);
|
||||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
kv = (KeyValue)cells.get(0);
|
||||
assertTrue(Bytes.equals(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(), TestRegionObserverInterface.B, 0,
|
||||
cell = cells.get(0);
|
||||
assertTrue(Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), TestRegionObserverInterface.B, 0,
|
||||
TestRegionObserverInterface.B.length));
|
||||
cells = familyMap.get(TestRegionObserverInterface.C);
|
||||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
kv = (KeyValue)cells.get(0);
|
||||
assertTrue(Bytes.equals(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(), TestRegionObserverInterface.C, 0,
|
||||
cell = cells.get(0);
|
||||
assertTrue(Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), TestRegionObserverInterface.C, 0,
|
||||
TestRegionObserverInterface.C.length));
|
||||
}
|
||||
ctPrePut.incrementAndGet();
|
||||
|
@ -422,25 +422,25 @@ public class SimpleRegionObserver extends BaseRegionObserver {
|
|||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
// KeyValue v1 expectation. Cast for now until we go all Cell all the time. TODO
|
||||
KeyValue kv = (KeyValue)cells.get(0);
|
||||
assertTrue(Bytes.equals(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(), TestRegionObserverInterface.A, 0,
|
||||
Cell cell = cells.get(0);
|
||||
assertTrue(Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), TestRegionObserverInterface.A, 0,
|
||||
TestRegionObserverInterface.A.length));
|
||||
cells = familyMap.get(TestRegionObserverInterface.B);
|
||||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
// KeyValue v1 expectation. Cast for now until we go all Cell all the time. TODO
|
||||
kv = (KeyValue)cells.get(0);
|
||||
assertTrue(Bytes.equals(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(), TestRegionObserverInterface.B, 0,
|
||||
cell = cells.get(0);
|
||||
assertTrue(Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), TestRegionObserverInterface.B, 0,
|
||||
TestRegionObserverInterface.B.length));
|
||||
cells = familyMap.get(TestRegionObserverInterface.C);
|
||||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
// KeyValue v1 expectation. Cast for now until we go all Cell all the time. TODO
|
||||
kv = (KeyValue)cells.get(0);
|
||||
assertTrue(Bytes.equals(kv.getQualifierArray(), kv.getQualifierOffset(),
|
||||
kv.getQualifierLength(), TestRegionObserverInterface.C, 0,
|
||||
cell = cells.get(0);
|
||||
assertTrue(Bytes.equals(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength(), TestRegionObserverInterface.C, 0,
|
||||
TestRegionObserverInterface.C.length));
|
||||
}
|
||||
ctPostPut.incrementAndGet();
|
||||
|
|
|
@ -152,7 +152,7 @@ public class TestDependentColumnFilter {
|
|||
int cells = 0;
|
||||
for (boolean done = true; done; i++) {
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
Arrays.sort(results.toArray(new Cell[results.size()]),
|
||||
CellComparator.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
if (results.isEmpty()) break;
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -119,14 +119,14 @@ public class TestScannerFromBucketCache {
|
|||
List<Cell> actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertFalse(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
// do the scan again and verify. This time it should be from the lru cache
|
||||
actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertFalse(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
|
||||
|
@ -157,7 +157,7 @@ public class TestScannerFromBucketCache {
|
|||
List<Cell> actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertFalse(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
// Wait for the bucket cache threads to move the data to offheap
|
||||
|
@ -166,7 +166,7 @@ public class TestScannerFromBucketCache {
|
|||
actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertTrue(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertTrue(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
|
||||
|
@ -198,7 +198,7 @@ public class TestScannerFromBucketCache {
|
|||
List<Cell> actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertFalse(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
// Wait for the bucket cache threads to move the data to offheap
|
||||
|
@ -218,7 +218,7 @@ public class TestScannerFromBucketCache {
|
|||
if (i != 5) {
|
||||
// the last cell fetched will be of type shareable but not offheap because
|
||||
// the MBB is copied to form a single cell
|
||||
assertTrue(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertTrue(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -250,14 +250,14 @@ public class TestScannerFromBucketCache {
|
|||
List<Cell> actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertFalse(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
// do the scan again and verify. This time it should be from the bucket cache in onheap mode
|
||||
actual = performScan(row1, fam1);
|
||||
// Verify result
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
assertFalse(actual.get(i) instanceof OffheapKeyValue);
|
||||
assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
|
||||
assertTrue(CellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
|
||||
}
|
||||
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
|
@ -215,7 +215,7 @@ public class TestSeekTo {
|
|||
|
||||
// seekBefore d, so the scanner points to c
|
||||
assertTrue(scanner.seekBefore(toKV("d", tagUsage)));
|
||||
assertFalse(scanner.getCell() instanceof OffheapKeyValue);
|
||||
assertFalse(scanner.getCell() instanceof ByteBufferKeyValue);
|
||||
assertEquals("c", toRowStr(scanner.getCell()));
|
||||
// reseekTo e and g
|
||||
assertEquals(0, scanner.reseekTo(toKV("c", tagUsage)));
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.CellComparator;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -331,7 +331,7 @@ public class TestProtobufUtil {
|
|||
kv3.getLength());
|
||||
ByteBuffer dbb = ByteBuffer.allocateDirect(arr.length);
|
||||
dbb.put(arr);
|
||||
OffheapKeyValue offheapKV = new OffheapKeyValue(dbb, kv1.getLength(), kv2.getLength());
|
||||
ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, kv1.getLength(), kv2.getLength());
|
||||
CellProtos.Cell cell = org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toCell(offheapKV);
|
||||
Cell newOffheapKV = org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toCell(cell);
|
||||
assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
||||
|
@ -34,6 +35,7 @@ import org.junit.experimental.categories.Category;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
|
@ -77,7 +79,7 @@ public class TestMemStoreChunkPool {
|
|||
Random rand = new Random();
|
||||
MemStoreLAB mslab = new MemStoreLABImpl(conf);
|
||||
int expectedOff = 0;
|
||||
byte[] lastBuffer = null;
|
||||
ByteBuffer lastBuffer = null;
|
||||
final byte[] rk = Bytes.toBytes("r1");
|
||||
final byte[] cf = Bytes.toBytes("f");
|
||||
final byte[] q = Bytes.toBytes("q");
|
||||
|
@ -86,14 +88,14 @@ public class TestMemStoreChunkPool {
|
|||
int valSize = rand.nextInt(1000);
|
||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||
int size = KeyValueUtil.length(kv);
|
||||
KeyValue newKv = (KeyValue) mslab.copyCellInto(kv);
|
||||
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||
if (newKv.getBuffer() != lastBuffer) {
|
||||
expectedOff = 0;
|
||||
lastBuffer = newKv.getBuffer();
|
||||
}
|
||||
assertEquals(expectedOff, newKv.getOffset());
|
||||
assertTrue("Allocation overruns buffer",
|
||||
newKv.getOffset() + size <= newKv.getBuffer().length);
|
||||
newKv.getOffset() + size <= newKv.getBuffer().capacity());
|
||||
expectedOff += size;
|
||||
}
|
||||
// chunks will be put back to pool after close
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import static org.junit.Assert.*;
|
||||
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -28,6 +29,7 @@ import java.util.Random;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -73,7 +75,7 @@ public class TestMemStoreLAB {
|
|||
Random rand = new Random();
|
||||
MemStoreLAB mslab = new MemStoreLABImpl();
|
||||
int expectedOff = 0;
|
||||
byte[] lastBuffer = null;
|
||||
ByteBuffer lastBuffer = null;
|
||||
// 100K iterations by 0-1K alloc -> 50MB expected
|
||||
// should be reasonable for unit test and also cover wraparound
|
||||
// behavior
|
||||
|
@ -81,14 +83,14 @@ public class TestMemStoreLAB {
|
|||
int valSize = rand.nextInt(1000);
|
||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||
int size = KeyValueUtil.length(kv);
|
||||
KeyValue newKv = (KeyValue) mslab.copyCellInto(kv);
|
||||
ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||
if (newKv.getBuffer() != lastBuffer) {
|
||||
expectedOff = 0;
|
||||
lastBuffer = newKv.getBuffer();
|
||||
}
|
||||
assertEquals(expectedOff, newKv.getOffset());
|
||||
assertTrue("Allocation overruns buffer",
|
||||
newKv.getOffset() + size <= newKv.getBuffer().length);
|
||||
newKv.getOffset() + size <= newKv.getBuffer().capacity());
|
||||
expectedOff += size;
|
||||
}
|
||||
}
|
||||
|
@ -127,9 +129,9 @@ public class TestMemStoreLAB {
|
|||
int valSize = r.nextInt(1000);
|
||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
|
||||
int size = KeyValueUtil.length(kv);
|
||||
KeyValue newKv = (KeyValue) mslab.copyCellInto(kv);
|
||||
ByteBufferKeyValue newCell = (ByteBufferKeyValue) mslab.copyCellInto(kv);
|
||||
totalAllocated.addAndGet(size);
|
||||
allocsByThisThread.add(new AllocRecord(newKv.getBuffer(), newKv.getOffset(), size));
|
||||
allocsByThisThread.add(new AllocRecord(newCell.getBuffer(), newCell.getOffset(), size));
|
||||
}
|
||||
};
|
||||
ctx.addThread(t);
|
||||
|
@ -143,7 +145,7 @@ public class TestMemStoreLAB {
|
|||
|
||||
// Partition the allocations by the actual byte[] they point into,
|
||||
// make sure offsets are unique for each chunk
|
||||
Map<byte[], Map<Integer, AllocRecord>> mapsByChunk =
|
||||
Map<ByteBuffer, Map<Integer, AllocRecord>> mapsByChunk =
|
||||
Maps.newHashMap();
|
||||
|
||||
int sizeCounted = 0;
|
||||
|
@ -169,7 +171,7 @@ public class TestMemStoreLAB {
|
|||
for (AllocRecord alloc : allocsInChunk.values()) {
|
||||
assertEquals(expectedOff, alloc.offset);
|
||||
assertTrue("Allocation overruns buffer",
|
||||
alloc.offset + alloc.size <= alloc.alloc.length);
|
||||
alloc.offset + alloc.size <= alloc.alloc.capacity());
|
||||
expectedOff += alloc.size;
|
||||
}
|
||||
}
|
||||
|
@ -251,11 +253,11 @@ public class TestMemStoreLAB {
|
|||
}
|
||||
|
||||
private static class AllocRecord implements Comparable<AllocRecord>{
|
||||
private final byte[] alloc;
|
||||
private final ByteBuffer alloc;
|
||||
private final int offset;
|
||||
private final int size;
|
||||
|
||||
public AllocRecord(byte[] alloc, int offset, int size) {
|
||||
public AllocRecord(ByteBuffer alloc, int offset, int size) {
|
||||
super();
|
||||
this.alloc = alloc;
|
||||
this.offset = offset;
|
||||
|
|
|
@ -220,9 +220,8 @@ public class TestTags {
|
|||
for (Result result : next) {
|
||||
CellScanner cellScanner = result.cellScanner();
|
||||
cellScanner.advance();
|
||||
KeyValue current = (KeyValue) cellScanner.current();
|
||||
assertTrue(current.getValueOffset() + current.getValueLength() == current.getOffset()
|
||||
+ current.getLength());
|
||||
Cell current = cellScanner.current();
|
||||
assertEquals(0, current.getTagsLength());
|
||||
}
|
||||
} finally {
|
||||
if (scanner != null)
|
||||
|
@ -239,9 +238,8 @@ public class TestTags {
|
|||
for (Result result : next) {
|
||||
CellScanner cellScanner = result.cellScanner();
|
||||
cellScanner.advance();
|
||||
KeyValue current = (KeyValue) cellScanner.current();
|
||||
assertTrue(current.getValueOffset() + current.getValueLength() == current.getOffset()
|
||||
+ current.getLength());
|
||||
Cell current = cellScanner.current();
|
||||
assertEquals(0, current.getTagsLength());
|
||||
}
|
||||
} finally {
|
||||
if (scanner != null) {
|
||||
|
@ -325,7 +323,7 @@ public class TestTags {
|
|||
while ((next = scanner.next()) != null) {
|
||||
CellScanner cellScanner = next.cellScanner();
|
||||
cellScanner.advance();
|
||||
KeyValue current = (KeyValue) cellScanner.current();
|
||||
Cell current = cellScanner.current();
|
||||
if (CellUtil.matchingRow(current, row)) {
|
||||
assertEquals(1, TestCoprocessorForTags.tags.size());
|
||||
Tag tag = TestCoprocessorForTags.tags.get(0);
|
||||
|
@ -350,7 +348,7 @@ public class TestTags {
|
|||
while ((next = scanner.next()) != null) {
|
||||
CellScanner cellScanner = next.cellScanner();
|
||||
cellScanner.advance();
|
||||
KeyValue current = (KeyValue) cellScanner.current();
|
||||
Cell current = cellScanner.current();
|
||||
if (CellUtil.matchingRow(current, row)) {
|
||||
assertEquals(1, TestCoprocessorForTags.tags.size());
|
||||
Tag tag = TestCoprocessorForTags.tags.get(0);
|
||||
|
|
|
@ -30,7 +30,7 @@ import java.util.List;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
|
@ -109,7 +109,7 @@ public class TestWALCellCodecWithCompression {
|
|||
return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
|
||||
}
|
||||
|
||||
private OffheapKeyValue createOffheapKV(int noOfTags) {
|
||||
private ByteBufferKeyValue createOffheapKV(int noOfTags) {
|
||||
byte[] row = Bytes.toBytes("myRow");
|
||||
byte[] cf = Bytes.toBytes("myCF");
|
||||
byte[] q = Bytes.toBytes("myQualifier");
|
||||
|
@ -121,6 +121,6 @@ public class TestWALCellCodecWithCompression {
|
|||
KeyValue kv = new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
|
||||
ByteBuffer dbb = ByteBuffer.allocateDirect(kv.getBuffer().length);
|
||||
dbb.put(kv.getBuffer());
|
||||
return new OffheapKeyValue(dbb, 0, kv.getBuffer().length);
|
||||
return new ByteBufferKeyValue(dbb, 0, kv.getBuffer().length);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -298,13 +298,13 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
|
|||
if (len < 0) {
|
||||
// This is a NOT label.
|
||||
len = (short) (-1 * len);
|
||||
String label = Bytes.toString(tag.getValueArray(), offset, len);
|
||||
String label = getTagValuePartAsString(tag, offset, len);
|
||||
if (authLabelsFinal.contains(label)) {
|
||||
includeKV = false;
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
String label = Bytes.toString(tag.getValueArray(), offset, len);
|
||||
String label = getTagValuePartAsString(tag, offset, len);
|
||||
if (!authLabelsFinal.contains(label)) {
|
||||
includeKV = false;
|
||||
break;
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
|
@ -125,7 +125,7 @@ public class TestWALReaderOnSecureWAL {
|
|||
if (offheap) {
|
||||
ByteBuffer bb = ByteBuffer.allocateDirect(kv.getBuffer().length);
|
||||
bb.put(kv.getBuffer());
|
||||
OffheapKeyValue offheapKV = new OffheapKeyValue(bb, 0, kv.getLength());
|
||||
ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(bb, 0, kv.getLength());
|
||||
kvs.add(offheapKV);
|
||||
} else {
|
||||
kvs.add(kv);
|
||||
|
|
Loading…
Reference in New Issue