HBASE-13300 Fixed casing in set/getTimeStamp for Mutations

This commit is contained in:
Jan Hentschel 2017-12-28 11:14:22 +01:00 committed by Michael Stack
parent b3ee2adead
commit 64ccd2b295
42 changed files with 228 additions and 127 deletions

View File

@ -1311,7 +1311,7 @@ public class MetaTableAccessor {
.setRow(put.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(HConstants.SPLITA_QUALIFIER)
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Type.Put)
.setValue(RegionInfo.toByteArray(splitA))
.build());
@ -1321,7 +1321,7 @@ public class MetaTableAccessor {
.setRow(put.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(HConstants.SPLITB_QUALIFIER)
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Type.Put)
.setValue(RegionInfo.toByteArray(splitB))
.build());
@ -1440,7 +1440,7 @@ public class MetaTableAccessor {
.setRow(put.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(getRegionStateColumn())
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Cell.Type.Put)
.setValue(Bytes.toBytes(state.name()))
.build());
@ -1569,7 +1569,7 @@ public class MetaTableAccessor {
.setRow(putOfMerged.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(HConstants.MERGEA_QUALIFIER)
.setTimestamp(putOfMerged.getTimeStamp())
.setTimestamp(putOfMerged.getTimestamp())
.setType(Type.Put)
.setValue(RegionInfo.toByteArray(regionA))
.build())
@ -1577,7 +1577,7 @@ public class MetaTableAccessor {
.setRow(putOfMerged.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(HConstants.MERGEB_QUALIFIER)
.setTimestamp(putOfMerged.getTimeStamp())
.setTimestamp(putOfMerged.getTimestamp())
.setType(Type.Put)
.setValue(RegionInfo.toByteArray(regionB))
.build());
@ -1890,7 +1890,7 @@ public class MetaTableAccessor {
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(HConstants.REGIONINFO_QUALIFIER)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(RegionInfo.toByteArray(hri))
.build());
@ -1904,7 +1904,7 @@ public class MetaTableAccessor {
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getServerColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(Bytes.toBytes(sn.getAddress().toString()))
.build())
@ -1912,7 +1912,7 @@ public class MetaTableAccessor {
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getStartCodeColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(Bytes.toBytes(sn.getStartcode()))
.build())
@ -1920,7 +1920,7 @@ public class MetaTableAccessor {
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getSeqNumColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(Bytes.toBytes(openSeqNum))
.build());
@ -2000,21 +2000,21 @@ public class MetaTableAccessor {
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getServerColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.build())
.add(builder.clear()
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getStartCodeColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.build())
.add(builder.clear()
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getSeqNumColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.build());
}
@ -2148,7 +2148,7 @@ public class MetaTableAccessor {
.setRow(p.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(getSeqNumColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(Bytes.toBytes(openSeqNum))
.build());

View File

@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* execute {@link #setTimeRange(long, long) setTimeRange}.
* <p>
* To only retrieve columns with a specific timestamp, execute
* {@link #setTimeStamp(long) setTimestamp}.
* {@link #setTimestamp(long) setTimestamp}.
* <p>
* To limit the number of versions of each column to be returned, execute
* {@link #setMaxVersions(int) setMaxVersions}.
@ -231,16 +231,28 @@ public class Get extends Query implements Row {
* Get versions of columns with the specified timestamp.
* @param timestamp version timestamp
* @return this for invocation chaining
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #setTimestamp(long)} instead
*/
public Get setTimeStamp(long timestamp)
throws IOException {
@Deprecated
public Get setTimeStamp(long timestamp) throws IOException {
return this.setTimestamp(timestamp);
}
/**
* Get versions of columns with the specified timestamp.
* @param timestamp version timestamp
* @return this for invocation chaining
*/
public Get setTimestamp(long timestamp) {
try {
tr = new TimeRange(timestamp, timestamp+1);
tr = new TimeRange(timestamp, timestamp + 1);
} catch(Exception e) {
// This should never happen, unless integer overflow or something extremely wrong...
LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
throw e;
}
return this;
}

View File

@ -113,7 +113,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
protected Mutation(Mutation clone) {
super(clone);
this.row = clone.getRow();
this.ts = clone.getTimeStamp();
this.ts = clone.getTimestamp();
this.familyMap = clone.getFamilyCellMap().entrySet().stream()
.collect(Collectors.toMap(e -> e.getKey(), e -> new ArrayList<>(e.getValue()),
(k, v) -> {
@ -344,8 +344,20 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
/**
* Method for retrieving the timestamp
* @return timestamp
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #getTimestamp()} instead
*/
@Deprecated
public long getTimeStamp() {
return this.getTimestamp();
}
/**
* Method for retrieving the timestamp.
*
* @return timestamp
*/
public long getTimestamp() {
return this.ts;
}

View File

@ -59,7 +59,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* To only retrieve columns within a specific range of version timestamps, call
* {@link #setTimeRange(long, long) setTimeRange}.
* <p>
* To only retrieve columns with a specific timestamp, call {@link #setTimeStamp(long) setTimestamp}
* To only retrieve columns with a specific timestamp, call {@link #setTimestamp(long) setTimestamp}
* .
* <p>
* To limit the number of versions of each column to be returned, call {@link #setMaxVersions(int)
@ -376,16 +376,34 @@ public class Scan extends Query {
* @see #setMaxVersions()
* @see #setMaxVersions(int)
* @return this
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #setTimestamp(long)} instead
*/
@Deprecated
public Scan setTimeStamp(long timestamp)
throws IOException {
return this.setTimestamp(timestamp);
}
/**
* Get versions of columns with the specified timestamp. Note, default maximum
* versions to return is 1. If your time range spans more than one version
* and you want all versions returned, up the number of versions beyond the
* defaut.
* @param timestamp version timestamp
* @see #setMaxVersions()
* @see #setMaxVersions(int)
* @return this
*/
public Scan setTimestamp(long timestamp) {
try {
tr = new TimeRange(timestamp, timestamp+1);
tr = new TimeRange(timestamp, timestamp + 1);
} catch(Exception e) {
// This should never happen, unless integer overflow or something extremely wrong...
LOG.error("TimeRange failed, likely caused by integer overflow. ", e);
throw e;
}
return this;
}

View File

@ -37,8 +37,9 @@ import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferExce
* <p>
* Note: Use of this filter overrides any time range/time stamp
* options specified using {@link org.apache.hadoop.hbase.client.Get#setTimeRange(long, long)},
* {@link org.apache.hadoop.hbase.client.Scan#setTimeRange(long, long)}, {@link org.apache.hadoop.hbase.client.Get#setTimeStamp(long)},
* or {@link org.apache.hadoop.hbase.client.Scan#setTimeStamp(long)}.
* {@link org.apache.hadoop.hbase.client.Scan#setTimeRange(long, long)},
* {@link org.apache.hadoop.hbase.client.Get#setTimestamp(long)},
* or {@link org.apache.hadoop.hbase.client.Scan#setTimestamp(long)}.
*/
@InterfaceAudience.Public
public class TimestampsFilter extends FilterBase {
@ -48,8 +49,8 @@ public class TimestampsFilter extends FilterBase {
private static final int MAX_LOG_TIMESTAMPS = 5;
// Used during scans to hint the scan to stop early
// once the timestamps fall below the minTimeStamp.
long minTimeStamp = Long.MAX_VALUE;
// once the timestamps fall below the minTimestamp.
long minTimestamp = Long.MAX_VALUE;
/**
* Constructor for filter that retains only the specified timestamps in the list.
@ -90,7 +91,7 @@ public class TimestampsFilter extends FilterBase {
private void init() {
if (this.timestamps.size() > 0) {
minTimeStamp = this.timestamps.first();
minTimestamp = this.timestamps.first();
}
}
@ -99,7 +100,7 @@ public class TimestampsFilter extends FilterBase {
* @return minimum timestamp requested by filter.
*/
public long getMin() {
return minTimeStamp;
return minTimestamp;
}
@Override
@ -118,7 +119,7 @@ public class TimestampsFilter extends FilterBase {
public ReturnCode filterCell(final Cell c) {
if (this.timestamps.contains(c.getTimestamp())) {
return ReturnCode.INCLUDE;
} else if (c.getTimestamp() < minTimeStamp) {
} else if (c.getTimestamp() < minTimestamp) {
// The remaining versions of this column are guaranteed
// to be lesser than all of the other values.
return ReturnCode.NEXT_COL;

View File

@ -1233,7 +1233,7 @@ public final class ProtobufUtil {
builder.setRow(ByteStringer.wrap(mutation.getRow()));
builder.setMutateType(type);
builder.setDurability(toDurability(mutation.getDurability()));
builder.setTimestamp(mutation.getTimeStamp());
builder.setTimestamp(mutation.getTimestamp());
Map<String, byte[]> attributes = mutation.getAttributesMap();
if (!attributes.isEmpty()) {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();

View File

@ -18,21 +18,29 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Public
public class ReplicationLoadSink {
private final long ageOfLastAppliedOp;
private final long timeStampsOfLastAppliedOp;
private final long timestampsOfLastAppliedOp;
// TODO: add the builder for this class
@InterfaceAudience.Private
public ReplicationLoadSink(long age, long timeStamp) {
public ReplicationLoadSink(long age, long timestamp) {
this.ageOfLastAppliedOp = age;
this.timeStampsOfLastAppliedOp = timeStamp;
this.timestampsOfLastAppliedOp = timestamp;
}
public long getAgeOfLastAppliedOp() {
return this.ageOfLastAppliedOp;
}
/**
* @deprecated Since hbase-2.0.0. Will be removed in 3.0.0.
* @see #getTimestampsOfLastAppliedOp()
*/
@Deprecated
public long getTimeStampsOfLastAppliedOp() {
return this.timeStampsOfLastAppliedOp;
return getTimestampsOfLastAppliedOp();
}
public long getTimestampsOfLastAppliedOp() {
return this.timestampsOfLastAppliedOp;
}
}

View File

@ -20,16 +20,16 @@ public class ReplicationLoadSource {
private final String peerID;
private final long ageOfLastShippedOp;
private final int sizeOfLogQueue;
private final long timeStampOfLastShippedOp;
private final long timestampOfLastShippedOp;
private final long replicationLag;
// TODO: add the builder for this class
@InterfaceAudience.Private
public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
public ReplicationLoadSource(String id, long age, int size, long timestamp, long lag) {
this.peerID = id;
this.ageOfLastShippedOp = age;
this.sizeOfLogQueue = size;
this.timeStampOfLastShippedOp = timeStamp;
this.timestampOfLastShippedOp = timestamp;
this.replicationLag = lag;
}
@ -45,8 +45,17 @@ public class ReplicationLoadSource {
return this.sizeOfLogQueue;
}
/**
* @deprecated Since 2.0.0. Will be removed in 3.0.0.
* @see #getTimestampOfLastShippedOp()
*/
@Deprecated
public long getTimeStampOfLastShippedOp() {
return this.timeStampOfLastShippedOp;
return getTimestampOfLastShippedOp();
}
public long getTimestampOfLastShippedOp() {
return this.timestampOfLastShippedOp;
}
public long getReplicationLag() {

View File

@ -1403,7 +1403,7 @@ public final class ProtobufUtil {
builder.setRow(UnsafeByteOperations.unsafeWrap(mutation.getRow()));
builder.setMutateType(type);
builder.setDurability(toDurability(mutation.getDurability()));
builder.setTimestamp(mutation.getTimeStamp());
builder.setTimestamp(mutation.getTimestamp());
Map<String, byte[]> attributes = mutation.getAttributesMap();
if (!attributes.isEmpty()) {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
@ -3177,7 +3177,7 @@ public final class ProtobufUtil {
.setPeerID(rls.getPeerID())
.setAgeOfLastShippedOp(rls.getAgeOfLastShippedOp())
.setSizeOfLogQueue((int) rls.getSizeOfLogQueue())
.setTimeStampOfLastShippedOp(rls.getTimeStampOfLastShippedOp())
.setTimeStampOfLastShippedOp(rls.getTimestampOfLastShippedOp())
.setReplicationLag(rls.getReplicationLag())
.build();
}
@ -3186,7 +3186,7 @@ public final class ProtobufUtil {
ReplicationLoadSink rls) {
return ClusterStatusProtos.ReplicationLoadSink.newBuilder()
.setAgeOfLastAppliedOp(rls.getAgeOfLastAppliedOp())
.setTimeStampsOfLastAppliedOp(rls.getTimeStampsOfLastAppliedOp())
.setTimeStampsOfLastAppliedOp(rls.getTimestampsOfLastAppliedOp())
.build();
}

View File

@ -157,7 +157,7 @@ public class TestMutation {
byte[] cloneValue = clone.getAttributesMap().get(name);
assertTrue(Bytes.equals(originValue, cloneValue));
}
Assert.assertEquals(origin.getTimeStamp(), clone.getTimeStamp());
Assert.assertEquals(origin.getTimestamp(), clone.getTimestamp());
Assert.assertEquals(origin.getPriority(), clone.getPriority());
if (origin instanceof Append) {
assertEquals(((Append)origin).getTimeRange(), ((Append)clone).getTimeRange());
@ -205,7 +205,7 @@ public class TestMutation {
.setRow(row)
.setFamily(family)
.setQualifier(qualifier0)
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Type.Put)
.setValue(value0)
.build())
@ -227,7 +227,7 @@ public class TestMutation {
assertTrue(cell0.getValueArray() == value0);
// Verify timestamp
assertTrue(cell0.getTimestamp() == put.getTimeStamp());
assertTrue(cell0.getTimestamp() == put.getTimestamp());
// Verify the cell of family:qualifier1
Cell cell1 = put.get(family, qualifier1).get(0);

View File

@ -367,14 +367,14 @@ public class TestOperation {
Put p = new Put(ROW);
List<Cell> c = p.get(FAMILY, QUALIFIER);
Assert.assertEquals(0, c.size());
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimestamp());
p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 1984L, ByteBuffer.wrap(VALUE));
c = p.get(FAMILY, QUALIFIER);
Assert.assertEquals(1, c.size());
Assert.assertEquals(1984L, c.get(0).getTimestamp());
Assert.assertArrayEquals(VALUE, CellUtil.cloneValue(c.get(0)));
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimestamp());
Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
p = new Put(ROW);
@ -383,7 +383,7 @@ public class TestOperation {
Assert.assertEquals(1, c.size());
Assert.assertEquals(2013L, c.get(0).getTimestamp());
Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimestamp());
Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
p = new Put(ByteBuffer.wrap(ROW));
@ -393,7 +393,7 @@ public class TestOperation {
Assert.assertEquals(2001L, c.get(0).getTimestamp());
Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
Assert.assertArrayEquals(ROW, CellUtil.cloneRow(c.get(0)));
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimestamp());
Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
p = new Put(ByteBuffer.wrap(ROW), 1970L);
@ -403,7 +403,7 @@ public class TestOperation {
Assert.assertEquals(2001L, c.get(0).getTimestamp());
Assert.assertArrayEquals(new byte[]{}, CellUtil.cloneValue(c.get(0)));
Assert.assertArrayEquals(ROW, CellUtil.cloneRow(c.get(0)));
Assert.assertEquals(1970L, p.getTimeStamp());
Assert.assertEquals(1970L, p.getTimestamp());
Assert.assertEquals(0, CellComparatorImpl.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
}

View File

@ -199,7 +199,7 @@ public class TestProtobufUtil {
// put value always use the default timestamp if no
// value level timestamp specified,
// add the timestamp to the original mutate
long timestamp = put.getTimeStamp();
long timestamp = put.getTimestamp();
for (ColumnValue.Builder column:
mutateBuilder.getColumnValueBuilderList()) {
for (QualifierValue.Builder qualifier:
@ -306,7 +306,7 @@ public class TestProtobufUtil {
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
Increment increment = ProtobufUtil.toIncrement(proto, null);
mutateBuilder.setTimestamp(increment.getTimeStamp());
mutateBuilder.setTimestamp(increment.getTimestamp());
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
}
@ -347,7 +347,7 @@ public class TestProtobufUtil {
// append always use the latest timestamp,
// reset the timestamp to the original mutate
mutateBuilder.setTimestamp(append.getTimeStamp());
mutateBuilder.setTimestamp(append.getTimestamp());
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(append.getTimeRange()));
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
}

View File

@ -227,7 +227,7 @@ public class MultiThreadedClientExample extends Configured implements Tool {
.setRow(rk)
.setFamily(FAMILY)
.setQualifier(QUAL)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(value)
.build());
@ -264,7 +264,7 @@ public class MultiThreadedClientExample extends Configured implements Tool {
.setRow(rk)
.setFamily(FAMILY)
.setQualifier(QUAL)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(value)
.build());

View File

@ -164,7 +164,7 @@ implements Configurable {
}
if (conf.get(SCAN_TIMESTAMP) != null) {
scan.setTimeStamp(Long.parseLong(conf.get(SCAN_TIMESTAMP)));
scan.setTimestamp(Long.parseLong(conf.get(SCAN_TIMESTAMP)));
}
if (conf.get(SCAN_TIMERANGE_START) != null && conf.get(SCAN_TIMERANGE_END) != null) {

View File

@ -201,7 +201,7 @@ public class RemoteHTable implements Table {
protected CellSetModel buildModelFromPut(Put put) {
RowModel row = new RowModel(put.getRow());
long ts = put.getTimeStamp();
long ts = put.getTimestamp();
for (List<Cell> cells: put.getFamilyCellMap().values()) {
for (Cell cell: cells) {
row.addCell(new CellModel(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell),
@ -468,7 +468,7 @@ public class RemoteHTable implements Table {
@Override
public void delete(Delete delete) throws IOException {
String spec = buildRowSpec(delete.getRow(), delete.getFamilyCellMap(),
delete.getTimeStamp(), delete.getTimeStamp(), 1);
delete.getTimestamp(), delete.getTimestamp(), 1);
for (int i = 0; i < maxRetries; i++) {
Response response = client.delete(spec);
int code = response.getCode();

View File

@ -223,7 +223,7 @@ public class TestRemoteTable {
get = new Get(ROW_2);
get.addFamily(COLUMN_1);
get.addFamily(COLUMN_2);
get.setTimeStamp(TS_1);
get.setTimestamp(TS_1);
result = remoteTable.get(get);
value1 = result.getValue(COLUMN_1, QUALIFIER_1);
value2 = result.getValue(COLUMN_2, QUALIFIER_2);

View File

@ -160,7 +160,7 @@ public class TableNamespaceManager implements Stoppable {
.setRow(row)
.setFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES)
.setQualifier(TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray())
.build());

View File

@ -177,7 +177,7 @@ public class RegionStateStore {
.setRow(put.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(getServerNameColumn(replicaId))
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Cell.Type.Put)
.setValue(Bytes.toBytes(regionLocation.getServerName()))
.build());
@ -187,7 +187,7 @@ public class RegionStateStore {
.setRow(put.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(getStateColumn(replicaId))
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Cell.Type.Put)
.setValue(Bytes.toBytes(state.name()))
.build());

View File

@ -180,8 +180,8 @@ class DefaultStoreFileManager implements StoreFileManager {
return files.stream().limit(Math.max(0, files.size() - 1)).filter(sf -> {
long fileTs = sf.getReader().getMaxTimestamp();
if (fileTs < maxTs && !filesCompacting.contains(sf)) {
LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is " +
fileTs + ", which is below " + maxTs);
LOG.info("Found an expired store file {} whose maxTimestamp is {}, which is below {}",
sf.getPath(), fileTs, maxTs);
return true;
} else {
return false;

View File

@ -2838,7 +2838,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if(delete.getFamilyCellMap().isEmpty()){
for(byte [] family : this.htableDescriptor.getColumnFamilyNames()){
// Don't eat the timestamp
delete.addFamily(family, delete.getTimeStamp());
delete.addFamily(family, delete.getTimestamp());
}
} else {
for(byte [] family : delete.getFamilyCellMap().keySet()) {
@ -2921,10 +2921,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (coprocessorHost != null) {
if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
byteNow, get)) {
updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
}
} else {
updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
}
} else {
PrivateCellUtil.updateLatestStamp(cell, byteNow);
@ -2933,7 +2933,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
void updateDeleteLatestVersionTimeStamp(Cell cell, Get get, int count, byte[] byteNow)
void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
throws IOException {
List<Cell> result = get(get, false);

View File

@ -282,6 +282,11 @@ public class HStoreFile implements StoreFile {
@Override
public long getModificationTimeStamp() throws IOException {
return getModificationTimestamp();
}
@Override
public long getModificationTimestamp() throws IOException {
return fileInfo.getModificationTime();
}

View File

@ -98,9 +98,18 @@ public interface StoreFile {
/**
* Get the modification time of this store file. Usually will access the file system so throws
* IOException.
* @deprecated Since 2.0.0. Will be removed in 3.0.0.
* @see #getModificationTimestamp()
*/
@Deprecated
long getModificationTimeStamp() throws IOException;
/**
* Get the modification time of this store file. Usually will access the file system so throws
* IOException.
*/
long getModificationTimestamp() throws IOException;
/**
* Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we
* append {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless

View File

@ -204,7 +204,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
if (this.generalBloomFilterWriter != null) {
/*
* http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
* Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
* Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + Timestamp
*
* 2 Types of Filtering:
* 1. Row = Row

View File

@ -61,7 +61,7 @@ public class StoreUtils {
public static long getLowestTimestamp(Collection<HStoreFile> candidates) throws IOException {
long minTs = Long.MAX_VALUE;
for (HStoreFile storeFile : candidates) {
minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
minTs = Math.min(minTs, storeFile.getModificationTimestamp());
}
return minTs;
}

View File

@ -1034,7 +1034,7 @@ public class StripeStoreFileManager
synchronized (sf) {
long fileTs = sf.getReader().getMaxTimestamp();
if (fileTs < maxTs && !filesCompacting.contains(sf)) {
LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is "
LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimestamp is "
+ fileTs + ", which is below " + maxTs);
if (expiredStoreFiles == null) {
expiredStoreFiles = new ArrayList<>();

View File

@ -142,11 +142,11 @@ public abstract class Compactor<T extends CellSink> {
private FileDetails getFileDetails(
Collection<HStoreFile> filesToCompact, boolean allFiles) throws IOException {
FileDetails fd = new FileDetails();
long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() -
long oldestHFileTimestampToKeepMVCC = System.currentTimeMillis() -
(1000L * 60 * 60 * 24 * this.keepSeqIdPeriod);
for (HStoreFile file : filesToCompact) {
if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) {
if(allFiles && (file.getModificationTimestamp() < oldestHFileTimestampToKeepMVCC)) {
// when isAllFiles is true, all files are compacted so we can calculate the smallest
// MVCC value to keep
if(fd.minSeqIdToKeep < file.getMaxMemStoreTS()) {

View File

@ -94,9 +94,20 @@ public class MetricsSink {
* Get the TimeStampOfLastAppliedOp. If no replication Op applied yet, the value is the timestamp
* at which hbase instance starts
* @return timeStampsOfLastAppliedOp;
* @deprecated Since 2.0.0. Will be removed in 3.0.0.
* @see #getTimestampOfLastAppliedOp()
*/
@Deprecated
public long getTimeStampOfLastAppliedOp() {
return this.lastTimestampForAge;
return getTimestampOfLastAppliedOp();
}
/**
* Get the TimestampOfLastAppliedOp. If no replication Op applied yet, the value is the timestamp
* at which hbase instance starts
* @return timeStampsOfLastAppliedOp;
*/
public long getTimestampOfLastAppliedOp() {
return this.lastTimestampForAge;
}
}

View File

@ -39,7 +39,7 @@ public class MetricsSource implements BaseSource {
private static final Logger LOG = LoggerFactory.getLogger(MetricsSource.class);
// tracks last shipped timestamp for each wal group
private Map<String, Long> lastTimeStamps = new HashMap<>();
private Map<String, Long> lastTimestamps = new HashMap<>();
private long lastHFileRefsQueueSize = 0;
private String id;
@ -82,7 +82,7 @@ public class MetricsSource implements BaseSource {
long age = EnvironmentEdgeManager.currentTime() - timestamp;
singleSourceSource.setLastShippedAge(age);
globalSourceSource.setLastShippedAge(age);
this.lastTimeStamps.put(walGroup, timestamp);
this.lastTimestamps.put(walGroup, timestamp);
}
/**
@ -91,9 +91,9 @@ public class MetricsSource implements BaseSource {
* @param walGroupId id of the group to update
*/
public void refreshAgeOfLastShippedOp(String walGroupId) {
Long lastTimestamp = this.lastTimeStamps.get(walGroupId);
Long lastTimestamp = this.lastTimestamps.get(walGroupId);
if (lastTimestamp == null) {
this.lastTimeStamps.put(walGroupId, 0L);
this.lastTimestamps.put(walGroupId, 0L);
lastTimestamp = 0L;
}
if (lastTimestamp > 0) {
@ -185,7 +185,7 @@ public class MetricsSource implements BaseSource {
singleSourceSource.decrSizeOfLogQueue(lastQueueSize);
singleSourceSource.clear();
globalSourceSource.decrSizeOfHFileRefsQueue(lastHFileRefsQueueSize);
lastTimeStamps.clear();
lastTimestamps.clear();
lastHFileRefsQueueSize = 0;
}
@ -208,10 +208,21 @@ public class MetricsSource implements BaseSource {
/**
* Get the timeStampsOfLastShippedOp, if there are multiple groups, return the latest one
* @return lastTimestampForAge
* @deprecated Since 2.0.0. Removed in 3.0.0.
* @see #getTimestampOfLastShippedOp()
*/
@Deprecated
public long getTimeStampOfLastShippedOp() {
return getTimestampOfLastShippedOp();
}
/**
* Get the timestampsOfLastShippedOp, if there are multiple groups, return the latest one
* @return lastTimestampForAge
*/
public long getTimestampOfLastShippedOp() {
long lastTimestamp = 0L;
for (long ts : lastTimeStamps.values()) {
for (long ts : lastTimestamps.values()) {
if (ts > lastTimestamp) {
lastTimestamp = ts;
}

View File

@ -64,11 +64,12 @@ public class ReplicationLoad {
ClusterStatusProtos.ReplicationLoadSink.Builder rLoadSinkBuild =
ClusterStatusProtos.ReplicationLoadSink.newBuilder();
rLoadSinkBuild.setAgeOfLastAppliedOp(sinkMetrics.getAgeOfLastAppliedOp());
rLoadSinkBuild.setTimeStampsOfLastAppliedOp(sinkMetrics.getTimeStampOfLastAppliedOp());
rLoadSinkBuild.setTimeStampsOfLastAppliedOp(sinkMetrics.getTimestampOfLastAppliedOp());
this.replicationLoadSink = rLoadSinkBuild.build();
// build the SourceLoad List
Map<String, ClusterStatusProtos.ReplicationLoadSource> replicationLoadSourceMap = new HashMap<>();
Map<String, ClusterStatusProtos.ReplicationLoadSource> replicationLoadSourceMap =
new HashMap<>();
for (MetricsSource sm : this.sourceMetricsList) {
// Get the actual peer id
String peerId = sm.getPeerID();
@ -77,7 +78,7 @@ public class ReplicationLoad {
long ageOfLastShippedOp = sm.getAgeOfLastShippedOp();
int sizeOfLogQueue = sm.getSizeOfLogQueue();
long timeStampOfLastShippedOp = sm.getTimeStampOfLastShippedOp();
long timeStampOfLastShippedOp = sm.getTimestampOfLastShippedOp();
long replicationLag;
long timePassedAfterLastShippedOp =
EnvironmentEdgeManager.currentTime() - timeStampOfLastShippedOp;
@ -128,7 +129,7 @@ public class ReplicationLoad {
sb = Strings.appendKeyValue(sb, "AgeOfLastShippedOp", rls.getAgeOfLastShippedOp());
sb = Strings.appendKeyValue(sb, "SizeOfLogQueue", rls.getSizeOfLogQueue());
sb =
Strings.appendKeyValue(sb, "TimeStampsOfLastShippedOp",
Strings.appendKeyValue(sb, "TimestampsOfLastShippedOp",
(new Date(rls.getTimeStampOfLastShippedOp()).toString()));
sb = Strings.appendKeyValue(sb, "Replication Lag", rls.getReplicationLag());
}
@ -148,7 +149,7 @@ public class ReplicationLoad {
Strings.appendKeyValue(sb, "AgeOfLastAppliedOp",
this.replicationLoadSink.getAgeOfLastAppliedOp());
sb =
Strings.appendKeyValue(sb, "TimeStampsOfLastAppliedOp",
Strings.appendKeyValue(sb, "TimestampsOfLastAppliedOp",
(new Date(this.replicationLoadSink.getTimeStampsOfLastAppliedOp()).toString()));
return sb.toString();

View File

@ -173,7 +173,7 @@ public class AccessControlLists {
.setRow(p.getRow())
.setFamily(ACL_LIST_FAMILY)
.setQualifier(key)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(value)
.build());

View File

@ -1574,7 +1574,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
}
AuthResult authResult = null;
if (checkCoveringPermission(user, opType, c.getEnvironment(), m.getRow(),
m.getFamilyCellMap(), m.getTimeStamp(), Action.WRITE)) {
m.getFamilyCellMap(), m.getTimestamp(), Action.WRITE)) {
authResult = AuthResult.allow(opType.toString(), "Covering cell set",
user, Action.WRITE, table, m.getFamilyCellMap());
} else {

View File

@ -217,7 +217,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
.setRow(row)
.setFamily(LABELS_TABLE_FAMILY)
.setQualifier(LABEL_QUALIFIER)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(Bytes.toBytes(SYSTEM_LABEL))
.build());
@ -245,7 +245,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
.setRow(row)
.setFamily(LABELS_TABLE_FAMILY)
.setQualifier(LABEL_QUALIFIER)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(label)
.setTags(TagUtil.fromList(Arrays.asList(LABELS_TABLE_TAGS)))
@ -285,7 +285,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
.setRow(row)
.setFamily(LABELS_TABLE_FAMILY)
.setQualifier(user)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(DUMMY_VALUE)
.setTags(TagUtil.fromList(Arrays.asList(LABELS_TABLE_TAGS)))

View File

@ -363,31 +363,30 @@ public abstract class HBaseTestCase extends TestCase {
protected void assertResultEquals(final HRegion region, final byte [] row,
final byte [] family, final byte [] qualifier, final long timestamp,
final byte [] value)
throws IOException {
Get get = new Get(row);
get.setTimeStamp(timestamp);
Result res = region.get(get);
NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> map =
res.getMap();
byte [] res_value = map.get(family).get(qualifier).get(timestamp);
final byte [] value) throws IOException {
Get get = new Get(row);
get.setTimestamp(timestamp);
Result res = region.get(get);
NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> map =
res.getMap();
byte [] res_value = map.get(family).get(qualifier).get(timestamp);
if (value == null) {
if (value == null) {
assertEquals(Bytes.toString(family) + " " + Bytes.toString(qualifier) +
" at timestamp " + timestamp, null, res_value);
} else {
if (res_value == null) {
fail(Bytes.toString(family) + " " + Bytes.toString(qualifier) +
" at timestamp " + timestamp + "\" was expected to be \"" +
Bytes.toStringBinary(value) + " but was null");
}
if (res_value != null) {
assertEquals(Bytes.toString(family) + " " + Bytes.toString(qualifier) +
" at timestamp " + timestamp, null, res_value);
} else {
if (res_value == null) {
fail(Bytes.toString(family) + " " + Bytes.toString(qualifier) +
" at timestamp " + timestamp + "\" was expected to be \"" +
Bytes.toStringBinary(value) + " but was null");
}
if (res_value != null) {
assertEquals(Bytes.toString(family) + " " + Bytes.toString(qualifier) +
" at timestamp " +
timestamp, value, new String(res_value, StandardCharsets.UTF_8));
}
" at timestamp " +
timestamp, value, new String(res_value, StandardCharsets.UTF_8));
}
}
}
/**
* Common method to close down a MiniDFSCluster and the associated file system

View File

@ -238,7 +238,7 @@ public class TestMultiVersions {
for (int j = 0; j < timestamp.length; j++) {
Get get = new Get(rows[i]);
get.addFamily(HConstants.CATALOG_FAMILY);
get.setTimeStamp(timestamp[j]);
get.setTimestamp(timestamp[j]);
Result result = table.get(get);
int cellCount = 0;
for(@SuppressWarnings("unused")Cell kv : result.listCells()) {
@ -286,7 +286,7 @@ public class TestMultiVersions {
count = 0;
scan = new Scan();
scan.setTimeStamp(1000L);
scan.setTimestamp(1000L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = table.getScanner(scan);
@ -322,7 +322,7 @@ public class TestMultiVersions {
count = 0;
scan = new Scan();
scan.setTimeStamp(100L);
scan.setTimestamp(100L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = table.getScanner(scan);

View File

@ -2754,7 +2754,7 @@ public class TestFromClientSide {
throws Exception {
Get get = new Get(row);
get.addColumn(family, qualifier);
get.setTimeStamp(stamp);
get.setTimestamp(stamp);
get.setMaxVersions(Integer.MAX_VALUE);
Result result = ht.get(get);
assertSingleResult(result, row, family, qualifier, stamp, value);
@ -2765,7 +2765,7 @@ public class TestFromClientSide {
throws Exception {
Get get = new Get(row);
get.addColumn(family, qualifier);
get.setTimeStamp(stamp);
get.setTimestamp(stamp);
get.setMaxVersions(Integer.MAX_VALUE);
Result result = ht.get(get);
assertEmptyResult(result);
@ -2776,7 +2776,7 @@ public class TestFromClientSide {
throws Exception {
Scan scan = new Scan(row);
scan.addColumn(family, qualifier);
scan.setTimeStamp(stamp);
scan.setTimestamp(stamp);
scan.setMaxVersions(Integer.MAX_VALUE);
Result result = getSingleScanResult(ht, scan);
assertSingleResult(result, row, family, qualifier, stamp, value);
@ -2787,7 +2787,7 @@ public class TestFromClientSide {
throws Exception {
Scan scan = new Scan(row);
scan.addColumn(family, qualifier);
scan.setTimeStamp(stamp);
scan.setTimestamp(stamp);
scan.setMaxVersions(Integer.MAX_VALUE);
Result result = getSingleScanResult(ht, scan);
assertNullResult(result);

View File

@ -161,7 +161,7 @@ public class TestMultiRespectsLimits {
.setRow(row)
.setFamily(FAMILY)
.setQualifier(col)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(value)
.build());

View File

@ -147,7 +147,7 @@ public class TestProtobufUtil {
// append always use the latest timestamp,
// reset the timestamp to the original mutate
mutateBuilder.setTimestamp(append.getTimeStamp());
mutateBuilder.setTimestamp(append.getTimestamp());
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(append.getTimeRange()));
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
}
@ -231,7 +231,7 @@ public class TestProtobufUtil {
mutateBuilder.setDurability(MutationProto.Durability.USE_DEFAULT);
Increment increment = ProtobufUtil.toIncrement(proto, null);
mutateBuilder.setTimestamp(increment.getTimeStamp());
mutateBuilder.setTimestamp(increment.getTimestamp());
mutateBuilder.setTimeRange(ProtobufUtil.toTimeRange(increment.getTimeRange()));
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.INCREMENT, increment));
}
@ -272,7 +272,7 @@ public class TestProtobufUtil {
// put value always use the default timestamp if no
// value level timestamp specified,
// add the timestamp to the original mutate
long timestamp = put.getTimeStamp();
long timestamp = put.getTimestamp();
for (ColumnValue.Builder column:
mutateBuilder.getColumnValueBuilderList()) {
for (QualifierValue.Builder qualifier:

View File

@ -135,6 +135,11 @@ public class MockHStoreFile extends HStoreFile {
@Override
public long getModificationTimeStamp() {
return getModificationTimestamp();
}
@Override
public long getModificationTimestamp() {
return modificationTime;
}

View File

@ -92,7 +92,7 @@ public class TestReplicationStatus extends TestReplicationBase {
assertTrue("failed to get ReplicationLoadSink.AgeOfLastShippedOp ",
(rLoadSink.getAgeOfLastAppliedOp() >= 0));
assertTrue("failed to get ReplicationLoadSink.TimeStampsOfLastAppliedOp ",
(rLoadSink.getTimeStampsOfLastAppliedOp() >= 0));
(rLoadSink.getTimestampsOfLastAppliedOp() >= 0));
}
// Stop rs1, then the queue of rs1 will be transfered to rs0

View File

@ -110,7 +110,7 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer
.setRow(p.getRow())
.setFamily(LABELS_TABLE_FAMILY)
.setQualifier(auth)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(DUMMY_VALUE)
.build());

View File

@ -1363,7 +1363,7 @@ public class ThriftServerRunner implements Runnable {
.setRow(put.getRow())
.setFamily(famAndQf[0])
.setQualifier(famAndQf[1])
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Type.Put)
.setValue(m.value != null ? getBytes(m.value)
: HConstants.EMPTY_BYTE_ARRAY)
@ -1433,7 +1433,7 @@ public class ThriftServerRunner implements Runnable {
.setRow(put.getRow())
.setFamily(famAndQf[0])
.setQualifier(famAndQf[1])
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Type.Put)
.setValue(m.value != null ? getBytes(m.value)
: HConstants.EMPTY_BYTE_ARRAY)
@ -1919,7 +1919,7 @@ public class ThriftServerRunner implements Runnable {
.setRow(put.getRow())
.setFamily(famAndQf[0])
.setQualifier(famAndQf[1])
.setTimestamp(put.getTimeStamp())
.setTimestamp(put.getTimestamp())
.setType(Type.Put)
.setValue(mput.value != null ? getBytes(mput.value)
: HConstants.EMPTY_BYTE_ARRAY)

View File

@ -96,7 +96,7 @@ public final class ThriftUtilities {
// Timestamp overwrites time range if both are set
if (in.isSetTimestamp()) {
out.setTimeStamp(in.getTimestamp());
out.setTimestamp(in.getTimestamp());
} else if (in.isSetTimeRange()) {
out.setTimeRange(in.getTimeRange().getMinStamp(), in.getTimeRange().getMaxStamp());
}
@ -234,7 +234,7 @@ public final class ThriftUtilities {
.setRow(out.getRow())
.setFamily(columnValue.getFamily())
.setQualifier(columnValue.getQualifier())
.setTimestamp(out.getTimeStamp())
.setTimestamp(out.getTimestamp())
.setType(Cell.Type.Put)
.setValue(columnValue.getValue())
.build());
@ -364,7 +364,7 @@ public final class ThriftUtilities {
TDelete out = new TDelete(ByteBuffer.wrap(in.getRow()));
List<TColumn> columns = new ArrayList<>(in.getFamilyCellMap().entrySet().size());
long rowTimestamp = in.getTimeStamp();
long rowTimestamp = in.getTimestamp();
if (rowTimestamp != HConstants.LATEST_TIMESTAMP) {
out.setTimestamp(rowTimestamp);
}