diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java new file mode 100644 index 0000000..615b0cf --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java @@ -0,0 +1,337 @@ +/** + * 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.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.util.ByteBufferUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ClassSize; + +/** + * This Cell is an implementation of {@link ByteBufferCell} where the data resides in + * off heap/ on heap ByteBuffer + */ +@InterfaceAudience.Private +public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell { + + protected final ByteBuffer buf; + protected final int offset; + protected final int length; + private long seqId = 0; + + private static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE + + (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT + + Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_LONG; + + public ByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId) { + this.buf = buf; + this.offset = offset; + this.length = length; + this.seqId = seqId; + } + + public ByteBufferKeyValue(ByteBuffer buf, int offset, int length) { + this.buf = buf; + this.offset = offset; + this.length = length; + } + + @Override + public byte[] getRowArray() { + return CellUtil.cloneRow(this); + } + + @Override + public int getRowOffset() { + return 0; + } + + @Override + public short getRowLength() { + return getRowLen(); + } + + private short getRowLen() { + return ByteBufferUtils.toShort(this.buf, this.offset + KeyValue.ROW_OFFSET); + } + + @Override + public byte[] getFamilyArray() { + return CellUtil.cloneFamily(this); + } + + @Override + public int getFamilyOffset() { + return 0; + } + + @Override + public byte getFamilyLength() { + return getFamilyLength(getFamilyLengthPosition()); + } + + private int getFamilyLengthPosition() { + return this.offset + KeyValue.ROW_KEY_OFFSET + + getRowLen(); + } + + private byte getFamilyLength(int famLenPos) { + return ByteBufferUtils.toByte(this.buf, famLenPos); + } + + @Override + public byte[] getQualifierArray() { + return CellUtil.cloneQualifier(this); + } + + @Override + public int getQualifierOffset() { + return 0; + } + + @Override + public int getQualifierLength() { + return getQualifierLength(getRowLength(), getFamilyLength()); + } + + private int getQualifierLength(int rlength, int flength) { + return getKeyLen() + - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0); + } + + @Override + public long getTimestamp() { + int offset = getTimestampOffset(getKeyLen()); + return ByteBufferUtils.toLong(this.buf, offset); + } + + private int getKeyLen() { + return ByteBufferUtils.toInt(this.buf, this.offset); + } + + private int getTimestampOffset(int keyLen) { + return this.offset + KeyValue.ROW_OFFSET + keyLen - KeyValue.TIMESTAMP_TYPE_SIZE; + } + + @Override + public byte getTypeByte() { + return ByteBufferUtils.toByte(this.buf, + this.offset + getKeyLen() - 1 + KeyValue.ROW_OFFSET); + } + + @Override + public long getSequenceId() { + return this.seqId; + } + + public void setSequenceId(long seqId) { + this.seqId = seqId; + } + + @Override + public byte[] getValueArray() { + return CellUtil.cloneValue(this); + } + + @Override + public int getValueOffset() { + return 0; + } + + @Override + public int getValueLength() { + return ByteBufferUtils.toInt(this.buf, this.offset + Bytes.SIZEOF_INT); + } + + @Override + public byte[] getTagsArray() { + return CellUtil.cloneTags(this); + } + + @Override + public int getTagsOffset() { + return 0; + } + + @Override + public int getTagsLength() { + int tagsLen = this.length - (getKeyLen() + getValueLength() + + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE); + if (tagsLen > 0) { + // There are some Tag bytes in the byte[]. So reduce 2 bytes which is + // added to denote the tags + // length + tagsLen -= KeyValue.TAGS_LENGTH_SIZE; + } + return tagsLen; + } + + @Override + public ByteBuffer getRowByteBuffer() { + return this.buf; + } + + @Override + public int getRowPosition() { + return this.offset + KeyValue.ROW_KEY_OFFSET; + } + + @Override + public ByteBuffer getFamilyByteBuffer() { + return this.buf; + } + + @Override + public int getFamilyPosition() { + return getFamilyLengthPosition() + Bytes.SIZEOF_BYTE; + } + + @Override + public ByteBuffer getQualifierByteBuffer() { + return this.buf; + } + + @Override + public int getQualifierPosition() { + return getFamilyPosition() + getFamilyLength(); + } + + @Override + public ByteBuffer getValueByteBuffer() { + return this.buf; + } + + @Override + public int getValuePosition() { + return this.offset + KeyValue.ROW_OFFSET + getKeyLen(); + } + + @Override + public ByteBuffer getTagsByteBuffer() { + return this.buf; + } + + @Override + public int getTagsPosition() { + int tagsLen = getTagsLength(); + if (tagsLen == 0) { + return this.offset + this.length; + } + return this.offset + this.length - tagsLen; + } + + @Override + public long heapSize() { + return ClassSize.align(FIXED_OVERHEAD + ClassSize.align(length)); + } + + @Override + public int write(OutputStream out, boolean withTags) throws IOException { + int length = getSerializedSize(withTags); + ByteBufferUtils.copyBufferToStream(out, this.buf, this.offset, length); + return length; + } + + @Override + public int getSerializedSize(boolean withTags) { + if (withTags) { + return this.length; + } + return getKeyLen() + this.getValueLength() + + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; + } + + @Override + public void write(ByteBuffer buf, int offset) { + ByteBufferUtils.copyFromBufferToBuffer(this.buf, buf, this.offset, offset, this.length); + } + + @Override + public String toString() { + return CellUtil.toString(this, true); + } + + @Override + public void setTimestamp(long ts) throws IOException { + ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), Bytes.toBytes(ts), 0, + Bytes.SIZEOF_LONG); + } + + private int getTimestampOffset() { + return this.offset + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE + + getKeyLen() - KeyValue.TIMESTAMP_TYPE_SIZE; + } + + @Override + public void setTimestamp(byte[] ts, int tsOffset) throws IOException { + ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), ts, tsOffset, + Bytes.SIZEOF_LONG); + } + + @Override + public long heapOverhead() { + return FIXED_OVERHEAD; + } + + @Override + public Cell deepClone() { + byte[] copy = new byte[this.length]; + ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length); + KeyValue kv = new KeyValue(copy, 0, copy.length); + kv.setSequenceId(this.getSequenceId()); + return kv; + } + + /** + * Needed doing 'contains' on List. Only compares the key portion, not the value. + */ + @Override + public boolean equals(Object other) { + if (!(other instanceof Cell)) { + return false; + } + return CellUtil.equals(this, (Cell) other); + } + + /** + * In line with {@link #equals(Object)}, only uses the key portion, not the value. + */ + @Override + public int hashCode() { + return calculateHashForKey(this); + } + + private int calculateHashForKey(ByteBufferCell cell) { + int rowHash = ByteBufferUtils.hashCode(cell.getRowByteBuffer(), cell.getRowPosition(), + cell.getRowLength()); + int familyHash = ByteBufferUtils.hashCode(cell.getFamilyByteBuffer(), cell.getFamilyPosition(), + cell.getFamilyLength()); + int qualifierHash = ByteBufferUtils.hashCode(cell.getQualifierByteBuffer(), + cell.getQualifierPosition(), cell.getQualifierLength()); + + int hash = 31 * rowHash + familyHash; + hash = 31 * hash + qualifierHash; + hash = 31 * hash + (int) cell.getTimestamp(); + hash = 31 * hash + cell.getTypeByte(); + return hash; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferTag.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferTag.java new file mode 100644 index 0000000..a212d2f --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferTag.java @@ -0,0 +1,83 @@ +/** + * Copyright The Apache Software Foundation + * + * 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.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.ByteBufferUtils; + +/** + * This is a {@link Tag} implementation in which value is backed by + * {@link java.nio.ByteBuffer} + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class ByteBufferTag implements Tag { + + private ByteBuffer buffer; + private int offset, length; + private byte type; + + public ByteBufferTag(ByteBuffer buffer, int offset, int length) { + this.buffer = buffer; + this.offset = offset; + this.length = length; + this.type = ByteBufferUtils.toByte(buffer, offset + TAG_LENGTH_SIZE); + } + + @Override + public byte getType() { + return this.type; + } + + @Override + public int getValueOffset() { + return this.offset + INFRASTRUCTURE_SIZE; + } + + @Override + public int getValueLength() { + return this.length - INFRASTRUCTURE_SIZE; + } + + @Override + public boolean hasArray() { + return false; + } + + @Override + public byte[] getValueArray() { + throw new UnsupportedOperationException( + "Tag is backed by an off heap buffer. Use getValueByteBuffer()"); + } + + @Override + public ByteBuffer getValueByteBuffer() { + return this.buffer; + } + + @Override + public String toString() { + return "[Tag type : " + this.type + ", value : " + + ByteBufferUtils.toStringBinary(buffer, getValueOffset(), getValueLength()) + "]"; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java index 986a026..1dab10c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java @@ -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()); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java new file mode 100644 index 0000000..3d86e4d --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NoTagsByteBufferKeyValue.java @@ -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; + } +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java deleted file mode 100644 index 1bc9549..0000000 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java +++ /dev/null @@ -1,340 +0,0 @@ -/** - * 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.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; - -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.util.ByteBufferUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ClassSize; - -/** - * This Cell is an implementation of {@link ByteBufferCell} where the data resides in off heap - * memory. - */ -@InterfaceAudience.Private -public class OffheapKeyValue 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; - - public OffheapKeyValue(ByteBuffer buf, int offset, int length, long seqId) { - assert buf.isDirect(); - this.buf = buf; - this.offset = offset; - this.length = length; - this.seqId = seqId; - } - - public OffheapKeyValue(ByteBuffer buf, int offset, int length) { - assert buf.isDirect(); - this.buf = buf; - this.offset = offset; - this.length = length; - } - - @Override - public byte[] getRowArray() { - return CellUtil.cloneRow(this); - } - - @Override - public int getRowOffset() { - return 0; - } - - @Override - public short getRowLength() { - return getRowLen(); - } - - private short getRowLen() { - return ByteBufferUtils.toShort(this.buf, this.offset + KeyValue.ROW_OFFSET); - } - - @Override - public byte[] getFamilyArray() { - return CellUtil.cloneFamily(this); - } - - @Override - public int getFamilyOffset() { - return 0; - } - - @Override - public byte getFamilyLength() { - return getFamilyLength(getFamilyLengthPosition()); - } - - private int getFamilyLengthPosition() { - return this.offset + KeyValue.ROW_KEY_OFFSET - + getRowLen(); - } - - private byte getFamilyLength(int famLenPos) { - return ByteBufferUtils.toByte(this.buf, famLenPos); - } - - @Override - public byte[] getQualifierArray() { - return CellUtil.cloneQualifier(this); - } - - @Override - public int getQualifierOffset() { - return 0; - } - - @Override - public int getQualifierLength() { - return getQualifierLength(getRowLength(), getFamilyLength()); - } - - private int getQualifierLength(int rlength, int flength) { - return getKeyLen() - - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0); - } - - @Override - public long getTimestamp() { - int offset = getTimestampOffset(getKeyLen()); - return ByteBufferUtils.toLong(this.buf, offset); - } - - private int getKeyLen() { - return ByteBufferUtils.toInt(this.buf, this.offset); - } - - private int getTimestampOffset(int keyLen) { - return this.offset + KeyValue.ROW_OFFSET + keyLen - KeyValue.TIMESTAMP_TYPE_SIZE; - } - - @Override - public byte getTypeByte() { - return ByteBufferUtils.toByte(this.buf, - this.offset + getKeyLen() - 1 + KeyValue.ROW_OFFSET); - } - - @Override - public long getSequenceId() { - return this.seqId; - } - - public void setSequenceId(long seqId) { - this.seqId = seqId; - } - - @Override - public byte[] getValueArray() { - return CellUtil.cloneValue(this); - } - - @Override - public int getValueOffset() { - return 0; - } - - @Override - public int getValueLength() { - return ByteBufferUtils.toInt(this.buf, this.offset + Bytes.SIZEOF_INT); - } - - @Override - public byte[] getTagsArray() { - return CellUtil.cloneTags(this); - } - - @Override - public int getTagsOffset() { - return 0; - } - - @Override - public int getTagsLength() { - int tagsLen = this.length - (getKeyLen() + getValueLength() - + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE); - if (tagsLen > 0) { - // There are some Tag bytes in the byte[]. So reduce 2 bytes which is - // added to denote the tags - // length - tagsLen -= KeyValue.TAGS_LENGTH_SIZE; - } - return tagsLen; - } - - @Override - public ByteBuffer getRowByteBuffer() { - return this.buf; - } - - @Override - public int getRowPosition() { - return this.offset + KeyValue.ROW_KEY_OFFSET; - } - - @Override - public ByteBuffer getFamilyByteBuffer() { - return this.buf; - } - - @Override - public int getFamilyPosition() { - return getFamilyLengthPosition() + Bytes.SIZEOF_BYTE; - } - - @Override - public ByteBuffer getQualifierByteBuffer() { - return this.buf; - } - - @Override - public int getQualifierPosition() { - return getFamilyPosition() + getFamilyLength(); - } - - @Override - public ByteBuffer getValueByteBuffer() { - return this.buf; - } - - @Override - public int getValuePosition() { - return this.offset + KeyValue.ROW_OFFSET + getKeyLen(); - } - - @Override - public ByteBuffer getTagsByteBuffer() { - return this.buf; - } - - @Override - public int getTagsPosition() { - int tagsLen = getTagsLength(); - if (tagsLen == 0) { - return this.offset + this.length; - } - return this.offset + this.length - tagsLen; - } - - @Override - public long heapSize() { - return ClassSize.align(FIXED_OVERHEAD + ClassSize.align(length)); - } - - @Override - public int write(OutputStream out, boolean withTags) throws IOException { - int length = getSerializedSize(withTags); - ByteBufferUtils.copyBufferToStream(out, this.buf, this.offset, length); - return length; - } - - @Override - public int getSerializedSize(boolean withTags) { - if (withTags) { - return this.length; - } - return getKeyLen() + this.getValueLength() - + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; - } - - @Override - public void write(ByteBuffer buf, int offset) { - ByteBufferUtils.copyFromBufferToBuffer(this.buf, buf, this.offset, offset, this.length); - } - - @Override - public String toString() { - return CellUtil.toString(this, true); - } - - @Override - public void setTimestamp(long ts) throws IOException { - ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), Bytes.toBytes(ts), 0, - Bytes.SIZEOF_LONG); - } - - private int getTimestampOffset() { - return this.offset + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE - + getKeyLen() - KeyValue.TIMESTAMP_TYPE_SIZE; - } - - @Override - public void setTimestamp(byte[] ts, int tsOffset) throws IOException { - ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), ts, tsOffset, - Bytes.SIZEOF_LONG); - } - - @Override - public long heapOverhead() { - return FIXED_OVERHEAD; - } - - @Override - public Cell deepClone() { - byte[] copy = new byte[this.length]; - ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length); - KeyValue kv = new KeyValue(copy, 0, copy.length); - kv.setSequenceId(this.getSequenceId()); - return kv; - } - - /** - * Needed doing 'contains' on List. Only compares the key portion, not the value. - */ - @Override - public boolean equals(Object other) { - if (!(other instanceof Cell)) { - return false; - } - return CellUtil.equals(this, (Cell) other); - } - - /** - * In line with {@link #equals(Object)}, only uses the key portion, not the value. - */ - @Override - public int hashCode() { - return calculateHashForKey(this); - } - - private int calculateHashForKey(ByteBufferCell cell) { - int rowHash = ByteBufferUtils.hashCode(cell.getRowByteBuffer(), cell.getRowPosition(), - cell.getRowLength()); - int familyHash = ByteBufferUtils.hashCode(cell.getFamilyByteBuffer(), cell.getFamilyPosition(), - cell.getFamilyLength()); - int qualifierHash = ByteBufferUtils.hashCode(cell.getQualifierByteBuffer(), - cell.getQualifierPosition(), cell.getQualifierLength()); - - int hash = 31 * rowHash + familyHash; - hash = 31 * hash + qualifierHash; - hash = 31 * hash + (int) cell.getTimestamp(); - hash = 31 * hash + cell.getTypeByte(); - return hash; - } -} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java deleted file mode 100644 index b3d65bb..0000000 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapTag.java +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Copyright The Apache Software Foundation - * - * 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.classification.InterfaceStability; -import org.apache.hadoop.hbase.util.ByteBufferUtils; - -/** - * This is a {@link Tag} implementation in which value is backed by an off heap - * {@link java.nio.ByteBuffer} - */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public class OffheapTag implements Tag { - - private ByteBuffer buffer; - private int offset, length; - private byte type; - - public OffheapTag(ByteBuffer buffer, int offset, int length) { - this.buffer = buffer; - this.offset = offset; - this.length = length; - this.type = ByteBufferUtils.toByte(buffer, offset + TAG_LENGTH_SIZE); - } - - @Override - public byte getType() { - return this.type; - } - - @Override - public int getValueOffset() { - return this.offset + INFRASTRUCTURE_SIZE; - } - - @Override - public int getValueLength() { - return this.length - INFRASTRUCTURE_SIZE; - } - - @Override - public boolean hasArray() { - return false; - } - - @Override - public byte[] getValueArray() { - throw new UnsupportedOperationException( - "Tag is backed by an off heap buffer. Use getValueByteBuffer()"); - } - - @Override - public ByteBuffer getValueByteBuffer() { - return this.buffer; - } - - @Override - public String toString() { - return "[Tag type : " + this.type + ", value : " - + ByteBufferUtils.toStringBinary(buffer, getValueOffset(), getValueLength()) + "]"; - } -} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java index 642444f..f9668dd 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java @@ -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; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java index 70ad3dd..9cccfe9 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java @@ -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); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java index de7f36c..90fff91 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java @@ -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); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java index d202f48..4e14acb 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java @@ -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() diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java index 0f9c0ce..c73705a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/RedundantKVGenerator.java @@ -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); } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java index dac7a56..81c85e4 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestOffheapKeyValue.java @@ -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(), diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java index 9e8ba8e..5e609ad 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java @@ -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; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index 1bb5274..c92d77d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -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(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java index 898f3bb..94e7219 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java @@ -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 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 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 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 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))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java index 9ec6dc9..a8fe3f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java @@ -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))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java index c094ccb..f943ce4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java @@ -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); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java index ba5bfa3..c4329b8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java @@ -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); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java index 0562fd9..27cede6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java @@ -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.TableName; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; @@ -124,7 +124,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);