HBASE-26713 Default to LATEST_TIMESTAMP if no timestamp sent along on Increment/Append (#4075)
Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
836f9be0cf
commit
76eb1b091d
|
@ -627,10 +627,7 @@ public final class ProtobufUtil {
|
|||
if (qv.hasQualifier()) {
|
||||
qualifier = qv.getQualifier().toByteArray();
|
||||
}
|
||||
long ts = HConstants.LATEST_TIMESTAMP;
|
||||
if (qv.hasTimestamp()) {
|
||||
ts = qv.getTimestamp();
|
||||
}
|
||||
long ts = cellTimestampOrLatest(qv);
|
||||
if (deleteType == DeleteType.DELETE_ONE_VERSION) {
|
||||
delete.addColumn(family, qualifier, ts);
|
||||
} else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
|
||||
|
@ -694,7 +691,7 @@ public final class ProtobufUtil {
|
|||
if (qv.hasTags()) {
|
||||
tags = qv.getTags().toByteArray();
|
||||
}
|
||||
consumer.accept(mutation, CellUtil.createCell(mutation.getRow(), family, qualifier, qv.getTimestamp(),
|
||||
consumer.accept(mutation, CellUtil.createCell(mutation.getRow(), family, qualifier, cellTimestampOrLatest(qv),
|
||||
KeyValue.Type.Put, value, tags));
|
||||
}
|
||||
}
|
||||
|
@ -706,6 +703,14 @@ public final class ProtobufUtil {
|
|||
return mutation;
|
||||
}
|
||||
|
||||
private static long cellTimestampOrLatest(QualifierValue cell) {
|
||||
if (cell.hasTimestamp()) {
|
||||
return cell.getTimestamp();
|
||||
} else {
|
||||
return HConstants.LATEST_TIMESTAMP;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a protocol buffer Mutate to an Append
|
||||
* @param cellScanner
|
||||
|
|
|
@ -787,10 +787,7 @@ public final class ProtobufUtil {
|
|||
if (qv.hasQualifier()) {
|
||||
qualifier = qv.getQualifier().toByteArray();
|
||||
}
|
||||
long ts = HConstants.LATEST_TIMESTAMP;
|
||||
if (qv.hasTimestamp()) {
|
||||
ts = qv.getTimestamp();
|
||||
}
|
||||
long ts = cellTimestampOrLatest(qv);
|
||||
if (deleteType == DeleteType.DELETE_ONE_VERSION) {
|
||||
delete.addColumn(family, qualifier, ts);
|
||||
} else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
|
||||
|
@ -857,7 +854,7 @@ public final class ProtobufUtil {
|
|||
.setRow(mutation.getRow())
|
||||
.setFamily(family)
|
||||
.setQualifier(qualifier)
|
||||
.setTimestamp(qv.getTimestamp())
|
||||
.setTimestamp(cellTimestampOrLatest(qv))
|
||||
.setType(KeyValue.Type.Put.getCode())
|
||||
.setValue(value)
|
||||
.setTags(tags)
|
||||
|
@ -872,6 +869,14 @@ public final class ProtobufUtil {
|
|||
return mutation;
|
||||
}
|
||||
|
||||
private static long cellTimestampOrLatest(QualifierValue cell) {
|
||||
if (cell.hasTimestamp()) {
|
||||
return cell.getTimestamp();
|
||||
} else {
|
||||
return HConstants.LATEST_TIMESTAMP;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a protocol buffer Mutate to an Append
|
||||
* @param cellScanner
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
|
|||
import org.apache.hadoop.hbase.ExtendedCellBuilder;
|
||||
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
|
@ -294,7 +295,22 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testIncrement() throws IOException {
|
||||
long timeStamp = 111111;
|
||||
|
||||
MutationProto proto = getIncrementMutation(111111L);
|
||||
// default fields
|
||||
assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
|
||||
|
||||
// set the default value for equal comparison
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
|
||||
|
||||
Increment increment = ProtobufUtil.toIncrement(proto, null);
|
||||
mutateBuilder.setTimestamp(increment.getTimestamp());
|
||||
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
|
||||
}
|
||||
|
||||
private MutationProto getIncrementMutation(Long timestamp) {
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutationProto.MutationType.INCREMENT);
|
||||
|
@ -303,25 +319,33 @@ public class TestProtobufUtil {
|
|||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(11L)));
|
||||
qualifierBuilder.setTimestamp(timeStamp);
|
||||
|
||||
if (timestamp != null) {
|
||||
qualifierBuilder.setTimestamp(timestamp);
|
||||
}
|
||||
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(22L)));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
// default fields
|
||||
assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
|
||||
return mutateBuilder.build();
|
||||
}
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
|
||||
|
||||
Increment increment = ProtobufUtil.toIncrement(proto, null);
|
||||
mutateBuilder.setTimestamp(increment.getTimestamp());
|
||||
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
|
||||
/**
|
||||
* Older clients may not send along a timestamp in the MutationProto. Check that we
|
||||
* default correctly.
|
||||
*/
|
||||
@Test
|
||||
public void testIncrementNoTimestamp() throws IOException {
|
||||
MutationProto mutation = getIncrementMutation(null);
|
||||
Increment increment = ProtobufUtil.toIncrement(mutation, null);
|
||||
assertEquals(HConstants.LATEST_TIMESTAMP, increment.getTimestamp());
|
||||
increment.getFamilyCellMap().values()
|
||||
.forEach(cells ->
|
||||
cells.forEach(cell ->
|
||||
assertEquals(HConstants.LATEST_TIMESTAMP, cell.getTimestamp())));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -331,29 +355,12 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testAppend() throws IOException {
|
||||
long timeStamp = 111111;
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutationType.APPEND);
|
||||
mutateBuilder.setTimestamp(timeStamp);
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
|
||||
qualifierBuilder.setTimestamp(timeStamp);
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
MutationProto proto = getAppendMutation(111111L);
|
||||
// default fields
|
||||
assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
|
||||
|
||||
Append append = ProtobufUtil.toAppend(proto, null);
|
||||
|
@ -365,6 +372,42 @@ public class TestProtobufUtil {
|
|||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
|
||||
}
|
||||
|
||||
/**
|
||||
* Older clients may not send along a timestamp in the MutationProto. Check that we
|
||||
* default correctly.
|
||||
*/
|
||||
@Test
|
||||
public void testAppendNoTimestamp() throws IOException {
|
||||
MutationProto mutation = getAppendMutation(null);
|
||||
Append append = ProtobufUtil.toAppend(mutation, null);
|
||||
assertEquals(HConstants.LATEST_TIMESTAMP, append.getTimestamp());
|
||||
append.getFamilyCellMap().values().forEach(cells -> cells.forEach(cell -> assertEquals(HConstants.LATEST_TIMESTAMP, cell.getTimestamp())));
|
||||
}
|
||||
|
||||
private MutationProto getAppendMutation(Long timestamp) {
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutationType.APPEND);
|
||||
if (timestamp != null) {
|
||||
mutateBuilder.setTimestamp(timestamp);
|
||||
}
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
|
||||
if (timestamp != null) {
|
||||
qualifierBuilder.setTimestamp(timestamp);
|
||||
}
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
return mutateBuilder.build();
|
||||
}
|
||||
|
||||
private static ProcedureProtos.Procedure.Builder createProcedureBuilder(long procId) {
|
||||
ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder();
|
||||
builder.setProcId(procId);
|
||||
|
|
|
@ -122,29 +122,12 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testAppend() throws IOException {
|
||||
long timeStamp = 111111;
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutationType.APPEND);
|
||||
mutateBuilder.setTimestamp(timeStamp);
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
|
||||
qualifierBuilder.setTimestamp(timeStamp);
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
MutationProto proto = getAppendMutation(111111L);
|
||||
// default fields
|
||||
assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
|
||||
|
||||
Append append = ProtobufUtil.toAppend(proto, null);
|
||||
|
@ -156,6 +139,42 @@ public class TestProtobufUtil {
|
|||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
|
||||
}
|
||||
|
||||
/**
|
||||
* Older clients may not send along a timestamp in the MutationProto. Check that we
|
||||
* default correctly.
|
||||
*/
|
||||
@Test
|
||||
public void testAppendNoTimestamp() throws IOException {
|
||||
MutationProto mutation = getAppendMutation(null);
|
||||
Append append = ProtobufUtil.toAppend(mutation, null);
|
||||
assertEquals(HConstants.LATEST_TIMESTAMP, append.getTimestamp());
|
||||
append.getFamilyCellMap().values().forEach(cells -> cells.forEach(cell -> assertEquals(HConstants.LATEST_TIMESTAMP, cell.getTimestamp())));
|
||||
}
|
||||
|
||||
private MutationProto getAppendMutation(Long timestamp) {
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutationType.APPEND);
|
||||
if (timestamp != null) {
|
||||
mutateBuilder.setTimestamp(timestamp);
|
||||
}
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v1"));
|
||||
if (timestamp != null) {
|
||||
qualifierBuilder.setTimestamp(timestamp);
|
||||
}
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFromUtf8("v2"));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
return mutateBuilder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Delete Mutate conversions.
|
||||
*
|
||||
|
@ -210,7 +229,36 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testIncrement() throws IOException {
|
||||
long timeStamp = 111111;
|
||||
MutationProto proto = getIncrementMutation(111111L);
|
||||
// default fields
|
||||
assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
|
||||
|
||||
// set the default value for equal comparison
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
|
||||
|
||||
Increment increment = ProtobufUtil.toIncrement(proto, null);
|
||||
mutateBuilder.setTimestamp(increment.getTimestamp());
|
||||
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
|
||||
}
|
||||
|
||||
/**
|
||||
* Older clients may not send along a timestamp in the MutationProto. Check that we
|
||||
* default correctly.
|
||||
*/
|
||||
@Test
|
||||
public void testIncrementNoTimestamp() throws IOException {
|
||||
MutationProto mutation = getIncrementMutation(null);
|
||||
Increment increment = ProtobufUtil.toIncrement(mutation, null);
|
||||
assertEquals(HConstants.LATEST_TIMESTAMP, increment.getTimestamp());
|
||||
increment.getFamilyCellMap().values()
|
||||
.forEach(cells ->
|
||||
cells.forEach(cell ->
|
||||
assertEquals(HConstants.LATEST_TIMESTAMP, cell.getTimestamp())));
|
||||
}
|
||||
|
||||
private MutationProto getIncrementMutation(Long timestamp) {
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutationType.INCREMENT);
|
||||
|
@ -219,25 +267,16 @@ public class TestProtobufUtil {
|
|||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c1"));
|
||||
qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(11L)));
|
||||
qualifierBuilder.setTimestamp(timeStamp);
|
||||
if (timestamp != null) {
|
||||
qualifierBuilder.setTimestamp(timestamp);
|
||||
}
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
qualifierBuilder.setQualifier(ByteString.copyFromUtf8("c2"));
|
||||
qualifierBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(22L)));
|
||||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
// default fields
|
||||
assertEquals(MutationProto.Durability.USE_DEFAULT, proto.getDurability());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
|
||||
|
||||
Increment increment = ProtobufUtil.toIncrement(proto, null);
|
||||
mutateBuilder.setTimestamp(increment.getTimestamp());
|
||||
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
|
||||
return mutateBuilder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue