1
0

Introduce getCommitsAndCompactionsTimeline() explicitly & adjust usage across code base

This commit is contained in:
Vinoth Chandar
2017-04-26 13:36:49 -07:00
committed by prazanna
parent bae0528013
commit da17c5c607
15 changed files with 42 additions and 48 deletions

View File

@@ -484,7 +484,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
List<String> latestFiles =
view.getLatestVersionInPartition(partitionPath, commitTime)
.map(HoodieDataFile::getFileName).collect(Collectors.toList());
return new Tuple2<String, List<String>>(partitionPath, latestFiles);
return new Tuple2<>(partitionPath, latestFiles);
}).collectAsMap();
HoodieSavepointMetadata metadata =
@@ -558,7 +558,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Make sure the rollback was successful
Optional<HoodieInstant> lastInstant =
activeTimeline.reload().getCommitTimeline().filterCompletedInstants().lastInstant();
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
Preconditions.checkArgument(lastInstant.isPresent());
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
savepointTime + "is not the last commit after rolling back " + commitsToRollback