HBASE-18927 Add the DataType which is subset of KeyValue#Type to CellBuilder for building cell

This commit is contained in:
Chia-Ping Tsai 2017-10-04 01:57:08 +08:00
parent 2f2513255d
commit 11aa6742f0
15 changed files with 67 additions and 36 deletions

View File

@ -31,8 +31,6 @@ import java.util.function.Function;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilder;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
@ -1328,7 +1326,7 @@ public final class ProtobufUtil {
}
List<Cell> cells = new ArrayList<>(values.size());
CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
for (CellProtos.Cell c : values) {
cells.add(toCell(builder, c));
}
@ -1371,7 +1369,7 @@ public final class ProtobufUtil {
if (!values.isEmpty()){
if (cells == null) cells = new ArrayList<>(values.size());
CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
for (CellProtos.Cell c: values) {
cells.add(toCell(builder, c));
}
@ -1632,7 +1630,7 @@ public final class ProtobufUtil {
return kvbuilder.build();
}
public static Cell toCell(CellBuilder cellBuilder, final CellProtos.Cell cell) {
public static Cell toCell(ExtendedCellBuilder cellBuilder, final CellProtos.Cell cell) {
return cellBuilder.clear()
.setRow(cell.getRow().toByteArray())
.setFamily(cell.getFamily().toByteArray())

View File

@ -1493,7 +1493,7 @@ public final class ProtobufUtil {
}
List<Cell> cells = new ArrayList<>(values.size());
CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
for (CellProtos.Cell c : values) {
cells.add(toCell(builder, c));
}
@ -1536,7 +1536,7 @@ public final class ProtobufUtil {
if (!values.isEmpty()){
if (cells == null) cells = new ArrayList<>(values.size());
CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
for (CellProtos.Cell c: values) {
cells.add(toCell(builder, c));
}
@ -2043,7 +2043,7 @@ public final class ProtobufUtil {
return UnsafeByteOperations.unsafeWrap(dup);
}
public static Cell toCell(CellBuilder cellBuilder, final CellProtos.Cell cell) {
public static Cell toCell(ExtendedCellBuilder cellBuilder, final CellProtos.Cell cell) {
return cellBuilder.clear()
.setRow(cell.getRow().toByteArray())
.setFamily(cell.getFamily().toByteArray())

View File

@ -25,9 +25,9 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
@ -260,7 +260,7 @@ public class TestProtobufUtil {
dbb.put(arr);
ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, kv1.getLength(), kv2.getLength());
CellProtos.Cell cell = ProtobufUtil.toCell(offheapKV);
Cell newOffheapKV = ProtobufUtil.toCell(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
Cell newOffheapKV = ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
}

View File

@ -26,6 +26,18 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Public
public interface CellBuilder {
/**
* The valid types for user to build the cell.
* Currently, This is subset of {@link KeyValue.Type}.
*/
enum DataType {
Put,
Delete,
DeleteFamilyVersion,
DeleteColumn,
DeleteFamily
}
CellBuilder setRow(final byte[] row);
CellBuilder setRow(final byte[] row, final int rOffset, final int rLength);
@ -37,7 +49,7 @@ public interface CellBuilder {
CellBuilder setTimestamp(final long timestamp);
CellBuilder setType(final byte type);
CellBuilder setType(final DataType type);
CellBuilder setValue(final byte[] value);
CellBuilder setValue(final byte[] value, final int vOffset, final int vLength);

View File

@ -372,7 +372,7 @@ public final class CellUtil {
@Deprecated
public static Cell createCell(final byte [] row, final byte [] family, final byte [] qualifier,
final long timestamp, final byte type, final byte [] value) {
return CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setRow(row)
.setFamily(family)
.setQualifier(qualifier)
@ -392,7 +392,7 @@ public final class CellUtil {
final byte [] familyArray, final int familyOffset, final int familyLength,
final byte [] qualifierArray, final int qualifierOffset, final int qualifierLength) {
// See createCell(final byte [] row, final byte [] value) for why we default Maximum type.
return CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setRow(rowArray, rowOffset, rowLength)
.setFamily(familyArray, familyOffset, familyLength)
.setQualifier(qualifierArray, qualifierOffset, qualifierLength)
@ -2381,7 +2381,7 @@ public final class CellUtil {
}
return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
}
public static Cell createFirstOnRow(final byte [] row, int roffset, short rlength) {
return new FirstOnRowCell(row, roffset, rlength);
}

View File

@ -47,6 +47,8 @@ public interface ExtendedCellBuilder extends CellBuilder {
ExtendedCellBuilder setTimestamp(final long timestamp);
@Override
ExtendedCellBuilder setType(final DataType type);
ExtendedCellBuilder setType(final byte type);
@Override

View File

@ -32,7 +32,7 @@ public abstract class ExtendedCellBuilderImpl implements ExtendedCellBuilder {
protected int qOffset = 0;
protected int qLength = 0;
protected long timestamp = HConstants.LATEST_TIMESTAMP;
protected Byte type = null;
protected KeyValue.Type type = null;
protected byte[] value = null;
protected int vOffset = 0;
protected int vLength = 0;
@ -86,9 +86,15 @@ public abstract class ExtendedCellBuilderImpl implements ExtendedCellBuilder {
return this;
}
@Override
public ExtendedCellBuilder setType(final DataType type) {
this.type = toKeyValueType(type);
return this;
}
@Override
public ExtendedCellBuilder setType(final byte type) {
this.type = type;
this.type = KeyValue.Type.codeToType(type);
return this;
}
@ -160,4 +166,15 @@ public abstract class ExtendedCellBuilderImpl implements ExtendedCellBuilder {
tagsLength = 0;
return this;
}
private static KeyValue.Type toKeyValueType(DataType type) {
switch (type) {
case Put: return KeyValue.Type.Put;
case Delete: return KeyValue.Type.Delete;
case DeleteColumn: return KeyValue.Type.DeleteColumn;
case DeleteFamilyVersion: return KeyValue.Type.DeleteFamilyVersion;
case DeleteFamily: return KeyValue.Type.DeleteFamily;
default: throw new UnsupportedOperationException("Unsupported data type:" + type);
}
}
}

View File

@ -26,7 +26,7 @@ class IndividualBytesFieldCellBuilder extends ExtendedCellBuilderImpl {
return new IndividualBytesFieldCell(row, rOffset, rLength,
family, fOffset, fLength,
qualifier, qOffset, qLength,
timestamp, KeyValue.Type.codeToType(type), seqId,
timestamp, type, seqId,
value, vOffset, vLength,
tags, tagsOffset, tagsLength);
}

View File

@ -26,7 +26,7 @@ class KeyValueBuilder extends ExtendedCellBuilderImpl {
KeyValue kv = new KeyValue(row, rOffset, rLength,
family, fOffset, fLength,
qualifier, qOffset, qLength,
timestamp, KeyValue.Type.codeToType(type),
timestamp, type,
value, vOffset, vLength,
tags, tagsOffset, tagsLength);
kv.setSequenceId(seqId);

View File

@ -41,7 +41,7 @@ public class TestCellBuilder {
.setRow(row)
.setFamily(family)
.setQualifier(qualifier)
.setType(KeyValue.Type.Put.getCode())
.setType(CellBuilder.DataType.Put)
.setValue(value)
.build();
row[0] = NEW_DATA;
@ -64,7 +64,7 @@ public class TestCellBuilder {
.setRow(row)
.setFamily(family)
.setQualifier(qualifier)
.setType(KeyValue.Type.Put.getCode())
.setType(CellBuilder.DataType.Put)
.setValue(value)
.build();
row[0] = NEW_DATA;

View File

@ -21,9 +21,9 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@ -54,6 +54,6 @@ public class TestPBCell {
pbr.setPosition(0);
decoded = CODEC.decode(pbr);
assertEquals(encodedLength, pbr.getPosition());
assertTrue(CellUtil.equals(cell, ProtobufUtil.toCell(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), decoded)));
assertTrue(CellUtil.equals(cell, ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), decoded)));
}
}

View File

@ -25,20 +25,20 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilder;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.ExtendedCellBuilder;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
public class BulkLoadCellFilter {
private static final Log LOG = LogFactory.getLog(BulkLoadCellFilter.class);
private final CellBuilder cellBuilder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
private final ExtendedCellBuilder cellBuilder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
/**
* Filters the bulk load cell using the supplied predicate.
* @param cell The WAL cell to filter.

View File

@ -24,9 +24,9 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.client.Append;
@ -333,7 +333,7 @@ public class TestProtobufUtil {
dbb.put(arr);
ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(dbb, kv1.getLength(), kv2.getLength());
CellProtos.Cell cell = ProtobufUtil.toCell(offheapKV);
Cell newOffheapKV = ProtobufUtil.toCell(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
Cell newOffheapKV = ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
assertTrue(CellComparator.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
}
}

View File

@ -26,6 +26,7 @@ import java.util.TreeMap;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilder;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -191,7 +192,7 @@ public class MockHStoreFile extends HStoreFile {
public Optional<Cell> getLastKey() {
if (splitPoint != null) {
return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setType(KeyValue.Type.Put.getCode())
.setType(CellBuilder.DataType.Put)
.setRow(Arrays.copyOf(splitPoint, splitPoint.length + 1)).build());
} else {
return Optional.empty();
@ -202,7 +203,7 @@ public class MockHStoreFile extends HStoreFile {
public Optional<Cell> midKey() throws IOException {
if (splitPoint != null) {
return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setType(KeyValue.Type.Put.getCode()).setRow(splitPoint).build());
.setType(CellBuilder.DataType.Put).setRow(splitPoint).build());
} else {
return Optional.empty();
}
@ -212,7 +213,7 @@ public class MockHStoreFile extends HStoreFile {
public Optional<Cell> getFirstKey() {
if (splitPoint != null) {
return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setType(KeyValue.Type.Put.getCode()).setRow(splitPoint, 0, splitPoint.length - 1)
.setType(CellBuilder.DataType.Put).setRow(splitPoint, 0, splitPoint.length - 1)
.build());
} else {
return Optional.empty();

View File

@ -59,6 +59,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilder;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellComparator;
@ -1014,13 +1015,13 @@ public class TestHStore {
long seqId = 100;
long timestamp = System.currentTimeMillis();
Cell cell0 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
.setQualifier(qf1).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
.setQualifier(qf1).setTimestamp(timestamp).setType(CellBuilder.DataType.Put)
.setValue(qf1).build();
CellUtil.setSequenceId(cell0, seqId);
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.emptyList());
Cell cell1 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
.setQualifier(qf2).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
.setQualifier(qf2).setTimestamp(timestamp).setType(CellBuilder.DataType.Put)
.setValue(qf1).build();
CellUtil.setSequenceId(cell1, seqId);
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));
@ -1028,7 +1029,7 @@ public class TestHStore {
seqId = 101;
timestamp = System.currentTimeMillis();
Cell cell2 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row2).setFamily(family)
.setQualifier(qf2).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
.setQualifier(qf2).setTimestamp(timestamp).setType(CellBuilder.DataType.Put)
.setValue(qf1).build();
CellUtil.setSequenceId(cell2, seqId);
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));
@ -1081,7 +1082,7 @@ public class TestHStore {
private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value)
throws IOException {
Cell c = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
.setQualifier(qualifier).setTimestamp(ts).setType(KeyValue.Type.Put.getCode())
.setQualifier(qualifier).setTimestamp(ts).setType(CellBuilder.DataType.Put)
.setValue(value).build();
CellUtil.setSequenceId(c, sequenceId);
return c;