.../java/org/apache/hadoop/hbase/CellUtil.java | 68 ++++++++++++++-------- .../src/main/java/org/apache/hadoop/hbase/Key.java | 55 +++++++++++++++++ .../java/org/apache/hadoop/hbase/KeyValue.java | 13 ++++- .../org/apache/hadoop/hbase/OffheapKeyValue.java | 23 +++++++- .../io/encoding/BufferedDataBlockEncoder.java | 45 +++++++++++++- 5 files changed, 177 insertions(+), 27 deletions(-) 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 d9f0679..7d4bee1 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 @@ -998,29 +998,39 @@ public final class CellUtil { * @throws IOException */ public static void writeFlatKey(Cell cell, DataOutputStream out) throws IOException { - short rowLen = cell.getRowLength(); - byte fLen = cell.getFamilyLength(); - int qLen = cell.getQualifierLength(); - // Using just one if/else loop instead of every time checking before writing every - // component of cell - if (cell instanceof ByteBufferedCell) { - out.writeShort(rowLen); - ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(), - ((ByteBufferedCell) cell).getRowPosition(), rowLen); - out.writeByte(fLen); - ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getFamilyByteBuffer(), - ((ByteBufferedCell) cell).getFamilyPosition(), fLen); - ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(), - ((ByteBufferedCell) cell).getQualifierPosition(), qLen); + if(cell instanceof Key) { + if (cell instanceof ByteBufferedCell) { + ByteBufferUtils.copyBufferToStream(out, ((Key) cell).getKeyBuffer(), + ((Key) cell).getKeyOffset(), ((Key) cell).getKeyLength()); + } else { + out.write(((Key) cell).getKeyArray(), ((Key) cell).getKeyOffset(), + ((Key) cell).getKeyLength()); + } } else { - out.writeShort(rowLen); - out.write(cell.getRowArray(), cell.getRowOffset(), rowLen); - out.writeByte(fLen); - out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen); - out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen); + short rowLen = cell.getRowLength(); + byte fLen = cell.getFamilyLength(); + int qLen = cell.getQualifierLength(); + // Using just one if/else loop instead of every time checking before writing every + // component of cell + if (cell instanceof ByteBufferedCell) { + out.writeShort(rowLen); + ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(), + ((ByteBufferedCell) cell).getRowPosition(), rowLen); + out.writeByte(fLen); + ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getFamilyByteBuffer(), + ((ByteBufferedCell) cell).getFamilyPosition(), fLen); + ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(), + ((ByteBufferedCell) cell).getQualifierPosition(), qLen); + } else { + out.writeShort(rowLen); + out.write(cell.getRowArray(), cell.getRowOffset(), rowLen); + out.writeByte(fLen); + out.write(cell.getFamilyArray(), cell.getFamilyOffset(), fLen); + out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qLen); + } + out.writeLong(cell.getTimestamp()); + out.writeByte(cell.getTypeByte()); } - out.writeLong(cell.getTimestamp()); - out.writeByte(cell.getTypeByte()); } /** @@ -1177,8 +1187,20 @@ public final class CellUtil { */ public static byte [] getCellKeySerializedAsKeyValueKey(final Cell cell) { if (cell == null) return null; - byte [] b = new byte[KeyValueUtil.keyLength(cell)]; - KeyValueUtil.appendKeyTo(cell, b, 0); + byte[] b = null; + if (cell instanceof Key) { + b = new byte[((Key) cell).getKeyLength()]; + if (cell instanceof ByteBufferedCell) { + ByteBufferUtils.copyFromBufferToArray(b, ((Key) cell).getKeyBuffer(), + ((Key) cell).getKeyOffset(), 0, ((Key) cell).getKeyLength()); + } else { + System.arraycopy(((Key) cell).getKeyArray(), ((Key) cell).getKeyOffset(), b, 0, + ((Key) cell).getKeyLength()); + } + } else { + b = new byte[KeyValueUtil.keyLength(cell)]; + KeyValueUtil.appendKeyTo(cell, b, 0); + } return b; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Key.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Key.java new file mode 100644 index 0000000..9638191 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Key.java @@ -0,0 +1,55 @@ +/* + * 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; + +/** + * Indicates if the cell is backed by a consecutive key byte[] or bytebuffer. Use the relevant + * getKeyArray() or getKeyBuffer() API based on the type of cell. For eg, if + * the cell is of type {@link ByteBufferedCell} then use {@link Key#getKeyBuffer()} + */ +@InterfaceAudience.Private +public interface Key { + + /** + * The key byte[] if the cell that implements this interface is backed by byte[] + * @return the key byte[] + */ + public byte[] getKeyArray(); + + /** + * The key ByteBuffer if the cell that implements this interface is backed by ByteBuffer + * @return the key ByteBuffer + */ + public ByteBuffer getKeyBuffer(); + + /** + * The key's offset position + * @return the key offset + */ + public int getKeyOffset(); + + /** + * The key length + * @return the key length + */ + public int getKeyLength(); +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java index f741f2c..0e47ef5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java @@ -82,7 +82,7 @@ import com.google.common.annotations.VisibleForTesting; */ @InterfaceAudience.Private public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, - SettableTimestamp, Streamable { + SettableTimestamp, Streamable, Key { private static final ArrayList EMPTY_ARRAY_LIST = new ArrayList(); private static final Log LOG = LogFactory.getLog(KeyValue.class); @@ -1243,6 +1243,15 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, return this.bytes; } + @Override + public byte[] getKeyArray() { + return this.bytes; + } + + @Override + public ByteBuffer getKeyBuffer() { + return ByteBuffer.wrap(this.bytes); + } /** * @return Offset into {@link #getBuffer()} at which this KeyValue starts. */ @@ -1279,6 +1288,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, /** * @return Key offset in backing buffer.. */ + @Override public int getKeyOffset() { return this.offset + ROW_OFFSET; } @@ -1290,6 +1300,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId, /** * @return Length of key portion. */ + @Override public int getKeyLength() { return Bytes.toInt(this.bytes, this.offset); } 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 index d060b02..cc3640e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/OffheapKeyValue.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.util.ClassSize; */ @InterfaceAudience.Private public class OffheapKeyValue extends ByteBufferedCell - implements HeapSize, SettableSequenceId, Streamable { + implements HeapSize, SettableSequenceId, Streamable, Key { protected final ByteBuffer buf; protected final int offset; @@ -236,6 +236,27 @@ public class OffheapKeyValue extends ByteBufferedCell } @Override + public byte[] getKeyArray() { + // Avoid calling this on on OffheapKeyvalue + throw new UnsupportedOperationException("Do not call getKeyArray on OffheapKeyValue"); + } + + @Override + public ByteBuffer getKeyBuffer() { + return this.buf; + } + + @Override + public int getKeyOffset() { + return this.offset + (2 * Bytes.SIZEOF_INT); + } + + @Override + public int getKeyLength() { + return ByteBufferUtils.toInt(this.buf, this.offset); + } + + @Override public long heapSize() { return ClassSize.align(FIXED_HEAP_SIZE_OVERHEAD + ClassSize.align(length)); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java index d873f7e..a994e45 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.Key; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue.Type; @@ -290,7 +291,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { // We return this as a Cell to the upper layers of read flow and might try setting a new SeqId // there. So this has to be an instance of SettableSequenceId. protected static class OnheapDecodedCell implements Cell, HeapSize, SettableSequenceId, - Streamable { + Streamable, Key { private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.ARRAY)); @@ -422,6 +423,26 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } @Override + public byte[] getKeyArray() { + return keyOnlyBuffer; + } + + @Override + public ByteBuffer getKeyBuffer() { + return ByteBuffer.wrap(keyOnlyBuffer); + } + + @Override + public int getKeyOffset() { + return 0; + } + + @Override + public int getKeyLength() { + return this.keyOnlyBuffer.length; + } + + @Override public String toString() { return KeyValue.keyToString(this.keyOnlyBuffer, 0, KeyValueUtil.keyLength(this)) + "/vlen=" + getValueLength() + "/seqid=" + seqId; @@ -466,7 +487,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } protected static class OffheapDecodedCell extends ByteBufferedCell implements HeapSize, - SettableSequenceId, Streamable { + SettableSequenceId, Streamable, Key { private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.BYTE_BUFFER)); @@ -651,6 +672,26 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } @Override + public byte[] getKeyArray() { + throw new UnsupportedOperationException("Do not call getKeyArray on this"); + } + + @Override + public ByteBuffer getKeyBuffer() { + return this.keyBuffer; + } + + @Override + public int getKeyOffset() { + return 0; + } + + @Override + public int getKeyLength() { + return this.keyBuffer.limit(); + } + + @Override public long heapSize() { return FIXED_OVERHEAD + rowLength + familyLength + qualifierLength + valueLength + tagsLength; }