Skip to content

Commit

Permalink
HBASE-13300 Fixed casing in set/getTimeStamp for Mutations
Browse files Browse the repository at this point in the history
  • Loading branch information
HorizonNet authored and saintstack committed Mar 24, 2018
1 parent b3ee2ad commit 64ccd2b
Show file tree
Hide file tree
Showing 42 changed files with 229 additions and 128 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1311,7 +1311,7 @@ public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo split
.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());
Expand All @@ -1321,7 +1321,7 @@ public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo split
.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());
Expand Down Expand Up @@ -1440,7 +1440,7 @@ private static void addRegionStateToPut(Put put, RegionState.State state) throws
.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());
Expand Down Expand Up @@ -1569,15 +1569,15 @@ public static void mergeRegions(Connection connection, RegionInfo mergedRegion,
.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())
.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
.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());
Expand Down Expand Up @@ -1890,7 +1890,7 @@ public static Put addRegionInfo(final Put p, final RegionInfo hri)
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(HConstants.REGIONINFO_QUALIFIER)
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(RegionInfo.toByteArray(hri))
.build());
Expand All @@ -1904,23 +1904,23 @@ public static Put addLocation(Put p, ServerName sn, long openSeqNum, int replica
.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())
.add(builder.clear()
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getStartCodeColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Cell.Type.Put)
.setValue(Bytes.toBytes(sn.getStartcode()))
.build())
.add(builder.clear()
.setRow(p.getRow())
.setFamily(getCatalogFamily())
.setQualifier(getSeqNumColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(Bytes.toBytes(openSeqNum))
.build());
Expand Down Expand Up @@ -2000,21 +2000,21 @@ private static Put addEmptyLocation(Put p, int replicaId) throws IOException {
.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());
}
Expand Down Expand Up @@ -2148,7 +2148,7 @@ private static Put addSequenceNum(Put p, long openSeqNum, int replicaId) throws
.setRow(p.getRow())
.setFamily(HConstants.CATALOG_FAMILY)
.setQualifier(getSeqNumColumn(replicaId))
.setTimestamp(p.getTimeStamp())
.setTimestamp(p.getTimestamp())
.setType(Type.Put)
.setValue(Bytes.toBytes(openSeqNum))
.build());
Expand Down
20 changes: 16 additions & 4 deletions hbase-client/src/main/java/org/apache/hadoop/hbase/client/Get.java
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@
* 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}.
Expand Down Expand Up @@ -231,16 +231,28 @@ public Get setTimeRange(long minStamp, long maxStamp) throws IOException {
* 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;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ protected Mutation() {
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) -> {
Expand Down Expand Up @@ -344,8 +344,20 @@ public int compareTo(final Row d) {
/**
* 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;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@
* 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)
Expand Down Expand Up @@ -376,16 +376,34 @@ public Scan setTimeRange(long minStamp, long maxStamp) throws IOException {
* @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;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,9 @@
* <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 {
Expand All @@ -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.
Expand Down Expand Up @@ -90,7 +91,7 @@ public List<Long> getTimestamps() {

private void init() {
if (this.timestamps.size() > 0) {
minTimeStamp = this.timestamps.first();
minTimestamp = this.timestamps.first();
}
}

Expand All @@ -99,7 +100,7 @@ private void init() {
* @return minimum timestamp requested by filter.
*/
public long getMin() {
return minTimeStamp;
return minTimestamp;
}

@Override
Expand All @@ -118,7 +119,7 @@ public ReturnCode filterKeyValue(final Cell c) {
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1233,7 +1233,7 @@ private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final
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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,21 +18,29 @@
@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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}

Expand All @@ -45,8 +45,17 @@ public long getSizeOfLogQueue() {
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() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1403,7 +1403,7 @@ private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final
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();
Expand Down Expand Up @@ -3177,7 +3177,7 @@ public static ClusterStatusProtos.ReplicationLoadSource toReplicationLoadSource(
.setPeerID(rls.getPeerID())
.setAgeOfLastShippedOp(rls.getAgeOfLastShippedOp())
.setSizeOfLogQueue((int) rls.getSizeOfLogQueue())
.setTimeStampOfLastShippedOp(rls.getTimeStampOfLastShippedOp())
.setTimeStampOfLastShippedOp(rls.getTimestampOfLastShippedOp())
.setReplicationLag(rls.getReplicationLag())
.build();
}
Expand All @@ -3186,7 +3186,7 @@ public static ClusterStatusProtos.ReplicationLoadSink toReplicationLoadSink(
ReplicationLoadSink rls) {
return ClusterStatusProtos.ReplicationLoadSink.newBuilder()
.setAgeOfLastAppliedOp(rls.getAgeOfLastAppliedOp())
.setTimeStampsOfLastAppliedOp(rls.getTimeStampsOfLastAppliedOp())
.setTimeStampsOfLastAppliedOp(rls.getTimestampsOfLastAppliedOp())
.build();
}

Expand Down
Loading

0 comments on commit 64ccd2b

Please sign in to comment.