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:
@@ -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);
|
||||
|
||||
@@ -66,11 +66,11 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
JavaSparkContext sc = getSparkContext(cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetaClient targetHoodieMetadata = new HoodieTableMetaClient(fs, cfg.targetPath);
|
||||
HoodieTimeline timeline = targetHoodieMetadata.getActiveCommitTimeline();
|
||||
HoodieTimeline timeline = targetHoodieMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
String lastCommitPulled = findLastCommitPulled(fs, cfg.dataPath);
|
||||
log.info("Last commit pulled on the source dataset is " + lastCommitPulled);
|
||||
if (!timeline.getInstants().iterator().hasNext() && timeline
|
||||
.compareInstants(timeline.lastInstant().get(), lastCommitPulled,
|
||||
.compareTimestamps(timeline.lastInstant().get().getTimestamp(), lastCommitPulled,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this should never be the case
|
||||
throw new IllegalStateException(
|
||||
|
||||
@@ -24,6 +24,7 @@ import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
@@ -66,7 +67,8 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView fsView = new ReadOptimizedTableView(fs, tableMetadata);
|
||||
// Get the latest commit
|
||||
final Optional<String> latestCommit = tableMetadata.getActiveCommitTimeline().lastInstant();
|
||||
final Optional<HoodieInstant>
|
||||
latestCommit = tableMetadata.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
if(!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
} else {
|
||||
@@ -91,7 +93,7 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<Tuple2<String, String>> filePaths = new ArrayList<>();
|
||||
for (HoodieDataFile hoodieDataFile : fsView
|
||||
.streamLatestVersionInPartition(partition, latestCommit.get())
|
||||
.getLatestVersionInPartition(partition, latestCommit.get().getTimestamp())
|
||||
.collect(Collectors.toList())) {
|
||||
filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()));
|
||||
}
|
||||
@@ -124,8 +126,8 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
} else {
|
||||
String commitTime =
|
||||
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return tableMetadata.getActiveCommitTimeline()
|
||||
.compareInstants(commitTime, latestCommit.get(), HoodieTimeline.GREATER);
|
||||
return tableMetadata.getActiveTimeline().getCommitTimeline()
|
||||
.compareTimestamps(commitTime, latestCommit.get().getTimestamp(), HoodieTimeline.GREATER);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
Reference in New Issue
Block a user