HBASE-16530 Reduce DBE code duplication
Signed-off-by: Yu Li <liyu@apache.org>
This commit is contained in:
parent
bc4ab47147
commit
ad67fd0816
@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.io.encoding;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public abstract class AbstractDataBlockEncoder 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 void postEncoding(HFileBlockEncodingContext encodingCtx)
|
||||
throws IOException {
|
||||
if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
|
||||
encodingCtx.postEncoding(BlockType.ENCODED_DATA);
|
||||
} else {
|
||||
encodingCtx.postEncoding(BlockType.DATA);
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract static class AbstractEncodedSeeker implements
|
||||
EncodedSeeker {
|
||||
protected HFileBlockDecodingContext decodingCtx;
|
||||
protected final KVComparator comparator;
|
||||
|
||||
public AbstractEncodedSeeker(KVComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx) {
|
||||
this.comparator = comparator;
|
||||
this.decodingCtx = decodingCtx;
|
||||
}
|
||||
|
||||
protected boolean includesMvcc() {
|
||||
return this.decodingCtx.getHFileContext().isIncludesMvcc();
|
||||
}
|
||||
|
||||
protected boolean includesTags() {
|
||||
return this.decodingCtx.getHFileContext().isIncludesTags();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -34,8 +34,6 @@ import org.apache.hadoop.hbase.SettableSequenceId;
|
||||
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.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
@ -46,7 +44,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
|
||||
*/
|
||||
@ -517,11 +515,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract static class
|
||||
BufferedEncodedSeeker<STATE extends SeekerState>
|
||||
implements EncodedSeeker {
|
||||
protected HFileBlockDecodingContext decodingCtx;
|
||||
protected final KVComparator comparator;
|
||||
protected abstract static class BufferedEncodedSeeker<STATE extends SeekerState>
|
||||
extends AbstractEncodedSeeker {
|
||||
protected final SamePrefixComparator<byte[]> samePrefixComparator;
|
||||
protected ByteBuffer currentBuffer;
|
||||
protected STATE current, previous;
|
||||
@ -529,9 +524,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
||||
|
||||
public BufferedEncodedSeeker(KVComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx) {
|
||||
this.comparator = comparator;
|
||||
super(comparator, decodingCtx);
|
||||
this.samePrefixComparator = comparator;
|
||||
this.decodingCtx = decodingCtx;
|
||||
if (decodingCtx.getHFileContext().isCompressTags()) {
|
||||
try {
|
||||
tagCompressionContext = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
|
||||
@ -542,14 +536,6 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
|
||||
current = createSeekerState(); // always valid
|
||||
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(KVComparator comparator, byte[] key, int offset, int length) {
|
||||
@ -880,17 +866,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;
|
||||
@ -970,10 +945,6 @@ 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);
|
||||
}
|
||||
}
|
||||
|
@ -21,15 +21,11 @@ import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
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.io.WritableUtils;
|
||||
|
||||
/**
|
||||
* Just copy data, do not do any kind of compression. Use for comparison and
|
||||
@ -38,31 +34,36 @@ import org.apache.hadoop.io.WritableUtils;
|
||||
@InterfaceAudience.Private
|
||||
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();
|
||||
private static class CopyKeyEncodingState extends EncodingState {
|
||||
NoneEncoder encoder = null;
|
||||
}
|
||||
|
||||
out.writeInt(klength);
|
||||
out.writeInt(vlength);
|
||||
CellUtil.writeFlatKey(cell, out);
|
||||
out.write(cell.getValueArray(), cell.getValueOffset(), 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) {
|
||||
out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
|
||||
}
|
||||
size += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
|
||||
@Override
|
||||
public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx,
|
||||
DataOutputStream out) throws IOException {
|
||||
if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
|
||||
throw new IOException(this.getClass().getName() + " only accepts "
|
||||
+ HFileBlockDefaultEncodingContext.class.getName() + " as the "
|
||||
+ "encoding context.");
|
||||
}
|
||||
if (encodingContext.getHFileContext().isIncludesMvcc()) {
|
||||
WritableUtils.writeVLong(out, cell.getSequenceId());
|
||||
size += WritableUtils.getVIntSize(cell.getSequenceId());
|
||||
}
|
||||
return size;
|
||||
|
||||
HFileBlockDefaultEncodingContext encodingCtx = (HFileBlockDefaultEncodingContext) blkEncodingCtx;
|
||||
encodingCtx.prepareEncoding(out);
|
||||
|
||||
NoneEncoder encoder = new NoneEncoder(out, encodingCtx);
|
||||
CopyKeyEncodingState state = new CopyKeyEncodingState();
|
||||
state.encoder = encoder;
|
||||
blkEncodingCtx.setEncodingState(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int internalEncode(Cell cell,
|
||||
HFileBlockDefaultEncodingContext encodingContext, DataOutputStream out)
|
||||
throws IOException {
|
||||
CopyKeyEncodingState state = (CopyKeyEncodingState) encodingContext
|
||||
.getEncodingState();
|
||||
NoneEncoder encoder = state.encoder;
|
||||
return encoder.write(cell);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -0,0 +1,68 @@
|
||||
/*
|
||||
*
|
||||
* 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.io.encoding;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
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.io.WritableUtils;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class NoneEncoder {
|
||||
|
||||
private DataOutputStream out;
|
||||
private HFileBlockDefaultEncodingContext encodingCtx;
|
||||
|
||||
public NoneEncoder(DataOutputStream out,
|
||||
HFileBlockDefaultEncodingContext encodingCtx) {
|
||||
this.out = out;
|
||||
this.encodingCtx = encodingCtx;
|
||||
}
|
||||
|
||||
public int write(Cell cell) throws IOException {
|
||||
int klength = KeyValueUtil.keyLength(cell);
|
||||
int vlength = cell.getValueLength();
|
||||
|
||||
out.writeInt(klength);
|
||||
out.writeInt(vlength);
|
||||
CellUtil.writeFlatKey(cell, out);
|
||||
out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
|
||||
int size = 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) {
|
||||
out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
|
||||
}
|
||||
size += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
|
||||
}
|
||||
if (encodingCtx.getHFileContext().isIncludesMvcc()) {
|
||||
WritableUtils.writeVLong(out, cell.getSequenceId());
|
||||
size += WritableUtils.getVIntSize(cell.getSequenceId());
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
}
|
@ -27,8 +27,6 @@ import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
@ -45,7 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RowIndexCodecV1 implements DataBlockEncoder {
|
||||
public class RowIndexCodecV1 extends AbstractDataBlockEncoder {
|
||||
|
||||
private static class RowIndexEncodingState extends EncodingState {
|
||||
RowIndexEncoderV1 encoder = null;
|
||||
@ -86,11 +84,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
|
||||
@ -151,15 +145,4 @@ public class RowIndexCodecV1 implements DataBlockEncoder {
|
||||
return new RowIndexSeekerV1(comparator, decodingCtx);
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,12 +16,9 @@ import java.io.IOException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
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 {
|
||||
@ -31,7 +28,7 @@ public class RowIndexEncoderV1 {
|
||||
private Cell lastCell = null;
|
||||
|
||||
private DataOutputStream out;
|
||||
private HFileBlockDefaultEncodingContext encodingCtx;
|
||||
private NoneEncoder encoder;
|
||||
private int startOffset = -1;
|
||||
private ByteArrayOutputStream rowsOffsetBAOS = new ByteArrayOutputStream(
|
||||
64 * 4);
|
||||
@ -39,7 +36,7 @@ public class RowIndexEncoderV1 {
|
||||
public RowIndexEncoderV1(DataOutputStream out,
|
||||
HFileBlockDefaultEncodingContext encodingCtx) {
|
||||
this.out = out;
|
||||
this.encodingCtx = encodingCtx;
|
||||
this.encoder = new NoneEncoder(out, encodingCtx);
|
||||
}
|
||||
|
||||
public int write(Cell cell) throws IOException {
|
||||
@ -50,31 +47,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
|
||||
out.write(cell.getValueArray(), cell.getValueOffset(), 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);
|
||||
// There are some tags to be written
|
||||
if (tagsLength > 0) {
|
||||
out.write(cell.getTagsArray(), cell.getTagsOffset(), 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 encoder.write(cell);
|
||||
}
|
||||
|
||||
protected boolean checkRow(final Cell cell) throws IOException {
|
||||
|
@ -25,17 +25,14 @@ import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.NoTagsKeyValue;
|
||||
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.util.ByteBufferUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class RowIndexSeekerV1 implements EncodedSeeker {
|
||||
|
||||
private HFileBlockDecodingContext decodingCtx;
|
||||
private final KVComparator comparator;
|
||||
public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
||||
|
||||
private ByteBuffer currentBuffer;
|
||||
private SeekerState current = new SeekerState(); // always valid
|
||||
@ -46,8 +43,7 @@ public class RowIndexSeekerV1 implements EncodedSeeker {
|
||||
|
||||
public RowIndexSeekerV1(KVComparator comparator,
|
||||
HFileBlockDecodingContext decodingCtx) {
|
||||
this.comparator = comparator;
|
||||
this.decodingCtx = decodingCtx;
|
||||
super(comparator, decodingCtx);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -318,14 +314,6 @@ public class RowIndexSeekerV1 implements EncodedSeeker {
|
||||
current.keyBuffer.getLength());
|
||||
}
|
||||
|
||||
protected boolean includesMvcc() {
|
||||
return this.decodingCtx.getHFileContext().isIncludesMvcc();
|
||||
}
|
||||
|
||||
protected boolean includesTags() {
|
||||
return this.decodingCtx.getHFileContext().isIncludesTags();
|
||||
}
|
||||
|
||||
protected void decodeTags() {
|
||||
current.tagsLength = currentBuffer.getShort();
|
||||
current.tagsOffset = currentBuffer.position();
|
||||
|
@ -19,17 +19,15 @@ package org.apache.hadoop.hbase.io.hfile;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
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.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.encoding.EncodingState;
|
||||
import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
|
||||
import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
|
||||
import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
|
||||
import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
import org.apache.hadoop.hbase.io.encoding.NoneEncoder;
|
||||
|
||||
/**
|
||||
* Does not perform any kind of encoding/decoding.
|
||||
@ -40,35 +38,21 @@ public class NoOpDataBlockEncoder implements HFileDataBlockEncoder {
|
||||
public static final NoOpDataBlockEncoder INSTANCE =
|
||||
new NoOpDataBlockEncoder();
|
||||
|
||||
private static class NoneEncodingState extends EncodingState {
|
||||
NoneEncoder encoder = null;
|
||||
}
|
||||
|
||||
/** Cannot be instantiated. Use {@link #INSTANCE} instead. */
|
||||
private NoOpDataBlockEncoder() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int encode(Cell cell, HFileBlockEncodingContext encodingCtx, DataOutputStream out)
|
||||
throws IOException {
|
||||
int klength = KeyValueUtil.keyLength(cell);
|
||||
int vlength = cell.getValueLength();
|
||||
|
||||
out.writeInt(klength);
|
||||
out.writeInt(vlength);
|
||||
CellUtil.writeFlatKey(cell, out);
|
||||
out.write(cell.getValueArray(), cell.getValueOffset(), 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) {
|
||||
out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
|
||||
}
|
||||
encodedKvSize += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
|
||||
}
|
||||
if (encodingCtx.getHFileContext().isIncludesMvcc()) {
|
||||
WritableUtils.writeVLong(out, cell.getSequenceId());
|
||||
encodedKvSize += WritableUtils.getVIntSize(cell.getSequenceId());
|
||||
}
|
||||
return encodedKvSize;
|
||||
public int encode(Cell cell, HFileBlockEncodingContext encodingCtx,
|
||||
DataOutputStream out) throws IOException {
|
||||
NoneEncodingState state = (NoneEncodingState) encodingCtx
|
||||
.getEncodingState();
|
||||
NoneEncoder encoder = state.encoder;
|
||||
return encoder.write(cell);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -107,8 +91,21 @@ public class NoOpDataBlockEncoder implements HFileDataBlockEncoder {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startBlockEncoding(HFileBlockEncodingContext encodingCtx, DataOutputStream out)
|
||||
throws IOException {
|
||||
public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx,
|
||||
DataOutputStream out) throws IOException {
|
||||
if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
|
||||
throw new IOException(this.getClass().getName() + " only accepts "
|
||||
+ HFileBlockDefaultEncodingContext.class.getName() + " as the "
|
||||
+ "encoding context.");
|
||||
}
|
||||
|
||||
HFileBlockDefaultEncodingContext encodingCtx = (HFileBlockDefaultEncodingContext) blkEncodingCtx;
|
||||
encodingCtx.prepareEncoding(out);
|
||||
|
||||
NoneEncoder encoder = new NoneEncoder(out, encodingCtx);
|
||||
NoneEncodingState state = new NoneEncodingState();
|
||||
state.encoder = encoder;
|
||||
blkEncodingCtx.setEncodingState(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
Loading…
x
Reference in New Issue
Block a user