Removing compaction action type and associated compaction timeline operations, replace with commit action type
This commit is contained in:
committed by
vinoth chandar
parent
a1c0d0dbad
commit
44839b88c6
@@ -294,7 +294,7 @@ public class HiveIncrementalPuller {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
|
||||
|
||||
Optional<HoodieInstant>
|
||||
lastCommit = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
lastCommit = metadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
if (lastCommit.isPresent()) {
|
||||
return lastCommit.get().getTimestamp();
|
||||
@@ -332,14 +332,14 @@ public class HiveIncrementalPuller {
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation)
|
||||
throws IOException {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline()
|
||||
.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().getCommitsAndCompactionsTimeline().filterCompletedInstants()
|
||||
.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
|
||||
.getInstants()
|
||||
.collect(Collectors.toList()) + " and from commit time is "
|
||||
+ config.fromCommitTime);
|
||||
|
||||
@@ -74,11 +74,11 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView(
|
||||
tableMetadata,
|
||||
tableMetadata.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
tableMetadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants());
|
||||
// Get the latest commit
|
||||
Optional<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline()
|
||||
.getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
if (!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
return;
|
||||
|
||||
@@ -120,7 +120,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath);
|
||||
this.commitTimelineOpt = Optional
|
||||
.of(meta.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
.of(meta.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants());
|
||||
} else {
|
||||
this.commitTimelineOpt = Optional.empty();
|
||||
|
||||
Reference in New Issue
Block a user