1
0

[MINOR] Reorder HoodieTimeline#compareTimestamp arguments for better readability (#1575)

- reads nicely as (instantTime1, GREATER_THAN_OR_EQUALS, instantTime2) etc
This commit is contained in:
vinoth chandar
2020-04-30 09:19:39 -07:00
committed by GitHub
parent 9059bce977
commit c4b71622b9
26 changed files with 64 additions and 62 deletions

View File

@@ -426,7 +426,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
try {
HoodieTable<T> table = HoodieTable.create(config, jsc);
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
.filter(instant -> HoodieActiveTimeline.EQUAL.test(instant.getTimestamp(), commitInstantTime))
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
.findFirst());
if (commitInstantOpt.isPresent()) {
HoodieRollbackMetadata rollbackMetadata = table.rollback(jsc, rollbackInstantTime, commitInstantOpt.get(), true);
@@ -537,7 +537,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
// if there are pending compactions, their instantTime must not be greater than that of this instant time
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending ->
ValidationUtils.checkArgument(
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), instantTime, HoodieTimeline.LESSER),
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
+ latestPending + ", Ingesting at " + instantTime));
HoodieTable<T> table = HoodieTable.create(metaClient, config, jsc);

View File

@@ -182,8 +182,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
// 2) is less than the first commit ts in the timeline
return !commitTimeline.empty()
&& (commitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|| HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
HoodieTimeline.GREATER));
|| HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), HoodieTimeline.GREATER_THAN, commitTs
));
}
/**

View File

@@ -169,10 +169,10 @@ public class HoodieTimelineArchiveLog {
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline.compareTimestamps(firstSavepoint.get().getTimestamp(),
s.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
HoodieTimeline.LESSER_THAN_OR_EQUALS, s.getTimestamp()));
}).filter(s -> {
// Ensure commits >= oldest pending compaction commit is retained
return oldestPendingCompactionInstant.map(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER)).orElse(true);
return oldestPendingCompactionInstant.map(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN, s.getTimestamp())).orElse(true);
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
}
@@ -243,7 +243,7 @@ public class HoodieTimelineArchiveLog {
List<HoodieInstant> instantsToBeDeleted =
instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(),
thresholdInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL)).collect(Collectors.toList());
HoodieTimeline.LESSER_THAN_OR_EQUALS, thresholdInstant.getTimestamp())).collect(Collectors.toList());
for (HoodieInstant deleteInstant : instantsToBeDeleted) {
LOG.info("Deleting instant " + deleteInstant + " in auxiliary meta path " + metaClient.getMetaAuxiliaryPath());

View File

@@ -123,9 +123,11 @@ public class CleanPlanner<T extends HoodieRecordPayload<T>> implements Serializa
LOG.warn("Incremental Cleaning mode is enabled. Looking up partition-paths that have since changed "
+ "since last cleaned at " + cleanMetadata.getEarliestCommitToRetain()
+ ". New Instant to retain : " + newInstantToRetain);
return hoodieTable.getCompletedCommitsTimeline().getInstants().filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
HoodieTimeline.GREATER_OR_EQUAL) && HoodieTimeline.compareTimestamps(instant.getTimestamp(),
newInstantToRetain.get().getTimestamp(), HoodieTimeline.LESSER)).flatMap(instant -> {
return hoodieTable.getCompletedCommitsTimeline().getInstants()
.filter(instant ->
HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, cleanMetadata.getEarliestCommitToRetain())
&& HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.LESSER_THAN, newInstantToRetain.get().getTimestamp())
).flatMap(instant -> {
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class);
return commitMetadata.getPartitionToWriteStats().keySet().stream();
@@ -252,7 +254,7 @@ public class CleanPlanner<T extends HoodieRecordPayload<T>> implements Serializa
// Always keep the last commit
if (!isFileSliceNeededForPendingCompaction(aSlice) && HoodieTimeline
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
.compareTimestamps(earliestCommitToRetain.getTimestamp(), HoodieTimeline.GREATER_THAN, fileCommitTime)) {
// this is a commit, that should be cleaned.
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getFileName()));
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
@@ -273,7 +275,7 @@ public class CleanPlanner<T extends HoodieRecordPayload<T>> implements Serializa
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList, HoodieInstant instantTime) {
for (FileSlice file : fileSliceList) {
String fileCommitTime = file.getBaseInstantTime();
if (HoodieTimeline.compareTimestamps(instantTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
if (HoodieTimeline.compareTimestamps(instantTime.getTimestamp(), HoodieTimeline.GREATER_THAN, fileCommitTime)) {
// fileList is sorted on the reverse, so the first commit we find <= instantTime is the
// one we want
return fileCommitTime;
@@ -324,8 +326,8 @@ public class CleanPlanner<T extends HoodieRecordPayload<T>> implements Serializa
CompactionOperation op = fgIdToPendingCompactionOperations.get(fileSlice.getFileGroupId());
if (null != op) {
// If file slice's instant time is newer or same as that of operation, do not clean
return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), op.getBaseInstantTime(),
HoodieTimeline.GREATER_OR_EQUAL);
return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), HoodieTimeline.GREATER_THAN_OR_EQUALS, op.getBaseInstantTime()
);
}
return false;
}

View File

@@ -89,7 +89,7 @@ public class ScheduleCompactionActionExecutor extends BaseActionExecutor<Option<
// if there are inflight writes, their instantTime must not be less than that of compaction instant time
table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant()
.ifPresent(earliestInflight -> ValidationUtils.checkArgument(
HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), instantTime, HoodieTimeline.GREATER),
HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime),
"Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
+ ", Compaction scheduled at " + instantTime));
@@ -97,7 +97,7 @@ public class ScheduleCompactionActionExecutor extends BaseActionExecutor<Option<
List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
.getCommitsAndCompactionTimeline().getInstants()
.filter(instant -> HoodieTimeline.compareTimestamps(
instant.getTimestamp(), instantTime, HoodieTimeline.GREATER_OR_EQUAL))
instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
.collect(Collectors.toList());
ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
"Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"

View File

@@ -64,7 +64,7 @@ public abstract class BaseRestoreActionExecutor extends BaseActionExecutor<Hoodi
// Get all the commits on the timeline after the provided commit time
List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline()
.getReverseOrderedInstants()
.filter(instant -> HoodieActiveTimeline.GREATER.test(instant.getTimestamp(), restoreInstantTime))
.filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), restoreInstantTime))
.collect(Collectors.toList());
Map<String, List<HoodieRollbackMetadata>> instantToMetadata = new HashMap<>();

View File

@@ -225,13 +225,13 @@ public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecuto
// For sanity, log instant time can never be less than base-commit on which we are rolling back
ValidationUtils
.checkArgument(HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId()),
rollbackInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
HoodieTimeline.LESSER_THAN_OR_EQUALS, rollbackInstant.getTimestamp()));
}
return validForRollback && HoodieTimeline.compareTimestamps(fileIdToBaseCommitTimeForLogMap.get(
// Base Ts should be strictly less. If equal (for inserts-to-logs), the caller employs another option
// to delete and we should not step on it
wStat.getFileId()), rollbackInstant.getTimestamp(), HoodieTimeline.LESSER);
wStat.getFileId()), HoodieTimeline.LESSER_THAN, rollbackInstant.getTimestamp());
}).map(wStat -> {
String baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId());
return RollbackRequest.createRollbackRequestWithAppendRollbackBlockAction(partitionPath, wStat.getFileId(),

View File

@@ -84,7 +84,7 @@ public class SavepointActionExecutor extends BaseActionExecutor<HoodieSavepointM
}
// Cannot allow savepoint time on a commit that could have been cleaned
ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, lastCommitRetained, HoodieTimeline.GREATER_OR_EQUAL),
ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN_OR_EQUALS, lastCommitRetained),
"Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained);
Map<String, List<String>> latestFilesMap = jsc.parallelize(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),