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

@@ -271,7 +271,7 @@ public class HiveIncrementalPuller {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
Optional<HoodieInstant>
lastCommit = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
lastCommit = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
if(lastCommit.isPresent()) {
return lastCommit.get().getTimestamp();
}
@@ -306,12 +306,12 @@ public class HiveIncrementalPuller {
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
List<String> commitsToSync = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants()
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants()
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
if (commitsToSync.isEmpty()) {
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata
.getActiveTimeline().getCommitTimeline().filterCompletedInstants().getInstants()
.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants().getInstants()
.collect(Collectors.toList()) + " and from commit time is "
+ config.fromCommitTime);
return null;