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 e3882cd..43ae020 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 @@ -23,7 +23,6 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import org.apache.hadoop.hbase.ByteBufferedCell; -import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; @@ -36,8 +35,6 @@ import org.apache.hadoop.hbase.Streamable; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.TagCompressionContext; -import org.apache.hadoop.hbase.io.hfile.BlockType; -import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.util.LRUDictionary; import org.apache.hadoop.hbase.io.util.StreamUtils; import org.apache.hadoop.hbase.nio.ByteBuff; @@ -51,7 +48,7 @@ import org.apache.hadoop.io.WritableUtils; * Base class for all data block encoders that use a buffer. */ @InterfaceAudience.Private -abstract class BufferedDataBlockEncoder implements DataBlockEncoder { +abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder { /** * TODO: This datablockencoder is dealing in internals of hfileblocks. Purge reference to HFBs */ @@ -682,11 +679,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } } - protected abstract static class - BufferedEncodedSeeker - implements EncodedSeeker { - protected HFileBlockDecodingContext decodingCtx; - protected final CellComparator comparator; + protected abstract static class BufferedEncodedSeeker + extends AbstractEncodedSeeker { protected ByteBuff currentBuffer; protected TagCompressionContext tagCompressionContext = null; protected KeyValue.KeyOnlyKeyValue keyOnlyKV = new KeyValue.KeyOnlyKeyValue(); @@ -697,8 +691,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { public BufferedEncodedSeeker(CellComparator comparator, HFileBlockDecodingContext decodingCtx) { - this.comparator = comparator; - this.decodingCtx = decodingCtx; + super(comparator, decodingCtx); if (decodingCtx.getHFileContext().isCompressTags()) { try { tagCompressionContext = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE); @@ -710,14 +703,6 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { previous = createSeekerState(); // may not be valid } - protected boolean includesMvcc() { - return this.decodingCtx.getHFileContext().isIncludesMvcc(); - } - - protected boolean includesTags() { - return this.decodingCtx.getHFileContext().isIncludesTags(); - } - @Override public int compareKey(CellComparator comparator, Cell key) { keyOnlyKV.setKey(current.keyBuffer, 0, current.keyLength); @@ -1049,17 +1034,6 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { } } - @Override - public HFileBlockEncodingContext newDataBlockEncodingContext(DataBlockEncoding encoding, - byte[] header, HFileContext meta) { - return new HFileBlockDefaultEncodingContext(encoding, header, meta); - } - - @Override - public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) { - return new HFileBlockDefaultDecodingContext(meta); - } - protected abstract ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength, int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx) throws IOException; @@ -1139,19 +1113,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder { Bytes.putInt(uncompressedBytesWithHeader, HConstants.HFILEBLOCK_HEADER_SIZE + DataBlockEncoding.ID_SIZE, state.unencodedDataSizeWritten ); - if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) { - encodingCtx.postEncoding(BlockType.ENCODED_DATA); - } else { - encodingCtx.postEncoding(BlockType.DATA); - } + postEncoding(encodingCtx); } - protected Cell createFirstKeyCell(ByteBuffer key, int keyLength) { - if (key.hasArray()) { - return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset() + key.position(), - keyLength); - } else { - return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength); - } - } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java index 178f65d..f517373 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java @@ -23,14 +23,10 @@ import java.nio.ByteBuffer; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.WritableUtils; /** * Just copy data, do not do any kind of compression. Use for comparison and @@ -40,30 +36,10 @@ import org.apache.hadoop.io.WritableUtils; public class CopyKeyDataBlockEncoder extends BufferedDataBlockEncoder { @Override - public int internalEncode(Cell cell, HFileBlockDefaultEncodingContext encodingContext, - DataOutputStream out) throws IOException { - int klength = KeyValueUtil.keyLength(cell); - int vlength = cell.getValueLength(); - - out.writeInt(klength); - out.writeInt(vlength); - CellUtil.writeFlatKey(cell, out); - CellUtil.writeValue(out, cell, vlength); - int size = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; - // Write the additional tag into the stream - if (encodingContext.getHFileContext().isIncludesTags()) { - int tagsLength = cell.getTagsLength(); - out.writeShort(tagsLength); - if (tagsLength > 0) { - CellUtil.writeTags(out, cell, tagsLength); - } - size += tagsLength + KeyValue.TAGS_LENGTH_SIZE; - } - if (encodingContext.getHFileContext().isIncludesMvcc()) { - WritableUtils.writeVLong(out, cell.getSequenceId()); - size += WritableUtils.getVIntSize(cell.getSequenceId()); - } - return size; + public int internalEncode(Cell cell, + HFileBlockDefaultEncodingContext encodingContext, DataOutputStream out) + throws IOException { + return NoDataBlockEncoderUtil.noneEncode(cell, encodingContext, out); } @Override diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java index 8877032..d81bb4a 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java @@ -23,15 +23,12 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; -import org.apache.hadoop.hbase.io.hfile.BlockType; -import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.util.ByteBufferUtils; @@ -51,7 +48,7 @@ import org.apache.hadoop.io.WritableUtils; * */ @InterfaceAudience.Private -public class RowIndexCodecV1 implements DataBlockEncoder { +public class RowIndexCodecV1 extends AbstractDataBlockEncoder { private static class RowIndexEncodingState extends EncodingState { RowIndexEncoderV1 encoder = null; @@ -92,11 +89,7 @@ public class RowIndexCodecV1 implements DataBlockEncoder { .getEncodingState(); RowIndexEncoderV1 encoder = state.encoder; encoder.flush(); - if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) { - encodingCtx.postEncoding(BlockType.ENCODED_DATA); - } else { - encodingCtx.postEncoding(BlockType.DATA); - } + postEncoding(encodingCtx); } @Override @@ -139,17 +132,6 @@ public class RowIndexCodecV1 implements DataBlockEncoder { } @Override - public HFileBlockEncodingContext newDataBlockEncodingContext( - DataBlockEncoding encoding, byte[] header, HFileContext meta) { - return new HFileBlockDefaultEncodingContext(encoding, header, meta); - } - - @Override - public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) { - return new HFileBlockDefaultDecodingContext(meta); - } - - @Override public Cell getFirstKeyCellInBlock(ByteBuff block) { block.mark(); int keyLength = block.getInt(); @@ -165,13 +147,4 @@ public class RowIndexCodecV1 implements DataBlockEncoder { return new RowIndexSeekerV1(comparator, decodingCtx); } - protected Cell createFirstKeyCell(ByteBuffer key, int keyLength) { - if (key.hasArray()) { - return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset() - + key.position(), keyLength); - } else { - return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength); - } - } - } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java index 7c69aea..77cc61e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java @@ -17,12 +17,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; -import org.apache.hadoop.io.WritableUtils; @InterfaceAudience.Private public class RowIndexEncoderV1 { @@ -50,30 +46,8 @@ public class RowIndexEncoderV1 { } rowsOffsetBAOS.writeInt(out.size() - startOffset); } - int klength = KeyValueUtil.keyLength(cell); - int vlength = cell.getValueLength(); - out.writeInt(klength); - out.writeInt(vlength); - CellUtil.writeFlatKey(cell, out); - // Write the value part - CellUtil.writeValue(out, cell, vlength); - int encodedKvSize = klength + vlength - + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE; - // Write the additional tag into the stream - if (encodingCtx.getHFileContext().isIncludesTags()) { - int tagsLength = cell.getTagsLength(); - out.writeShort(tagsLength); - if (tagsLength > 0) { - CellUtil.writeTags(out, cell, tagsLength); - } - encodedKvSize += tagsLength + KeyValue.TAGS_LENGTH_SIZE; - } - if (encodingCtx.getHFileContext().isIncludesMvcc()) { - WritableUtils.writeVLong(out, cell.getSequenceId()); - encodedKvSize += WritableUtils.getVIntSize(cell.getSequenceId()); - } lastCell = cell; - return encodedKvSize; + return NoDataBlockEncoderUtil.noneEncode(cell, encodingCtx, out); } protected boolean checkRow(final Cell cell) throws IOException { 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 3bdc580..389ce01 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 @@ -29,17 +29,14 @@ import org.apache.hadoop.hbase.OffheapKeyValue; import org.apache.hadoop.hbase.SizeCachedKeyValue; import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker; +import org.apache.hadoop.hbase.io.encoding.AbstractDataBlockEncoder.AbstractEncodedSeeker; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ObjectIntPair; @InterfaceAudience.Private -public class RowIndexSeekerV1 implements EncodedSeeker { - - private HFileBlockDecodingContext decodingCtx; - private final CellComparator comparator; +public class RowIndexSeekerV1 extends AbstractEncodedSeeker { // A temp pair object which will be reused by ByteBuff#asSubByteBuffer calls. This avoids too // many object creations. @@ -54,8 +51,7 @@ public class RowIndexSeekerV1 implements EncodedSeeker { public RowIndexSeekerV1(CellComparator comparator, HFileBlockDecodingContext decodingCtx) { - this.comparator = comparator; - this.decodingCtx = decodingCtx; + super(comparator, decodingCtx); } @Override @@ -293,14 +289,6 @@ public class RowIndexSeekerV1 implements EncodedSeeker { current.keyLength); } - protected boolean includesMvcc() { - return this.decodingCtx.getHFileContext().isIncludesMvcc(); - } - - protected boolean includesTags() { - return this.decodingCtx.getHFileContext().isIncludesTags(); - } - protected void decodeTags() { current.tagsLength = currentBuffer.getShortAfterPosition(0); currentBuffer.skip(Bytes.SIZEOF_SHORT);