1
0

Take 2: Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

- Refactored timelines to be a single timeline for all active events and one for archived events. CommitTimeline and other timelines can be inferred by applying a filter on the activeTimelime
- Introduced HoodieInstant to abstract different types of action, commit time and if isInFlight
- Implemented other review comments
This commit is contained in:
Prasanna Rajaperumal
2017-01-18 01:00:36 -08:00
parent 8ee777a9bb
commit ccd8cb2407
46 changed files with 1194 additions and 1106 deletions

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.utilities;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullException;
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullSQLException;
@@ -267,8 +268,12 @@ public class HiveIncrementalPuller {
}
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
Optional<String> lastCommit = metadata.getActiveCommitTimeline().lastInstant();
return lastCommit.orElse("0");
Optional<HoodieInstant>
lastCommit = metadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
if(lastCommit.isPresent()) {
return lastCommit.get().getTimestamp();
}
return "0";
}
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
@@ -299,13 +304,14 @@ public class HiveIncrementalPuller {
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
List<String> commitsToSync = metadata.getActiveCommitTimeline()
.findInstantsAfter(config.fromCommitTime, config.maxCommits)
List<String> commitsToSync = metadata.getActiveTimeline().getCommitTimeline().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
.getActiveCommitTimeline().getInstants().collect(Collectors.toList())
+ " and from commit time is " + config.fromCommitTime);
.getActiveTimeline().getCommitTimeline().filterCompletedInstants().getInstants()
.collect(Collectors.toList()) + " and from commit time is "
+ config.fromCommitTime);
return null;
}
log.info("Syncing commits " + commitsToSync);