Introduce getCommitsAndCompactionsTimeline() explicitly & adjust usage across code base
This commit is contained in:
@@ -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
|
||||
|
||||
@@ -35,7 +35,6 @@ import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -92,8 +91,6 @@ public class HoodieCommitArchiveLog {
|
||||
log.info("Deleting commits " + commitsToArchive);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant commitToArchive : commitsToArchive) {
|
||||
@@ -126,7 +123,7 @@ public class HoodieCommitArchiveLog {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||
|
||||
HoodieAppendLog.Writer writer = null;
|
||||
try {
|
||||
|
||||
@@ -197,9 +197,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits
|
||||
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION));
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
@@ -222,7 +220,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
public HoodieTimeline getCompactionCommitTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitTimeline();
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits in tagging
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
|
||||
Reference in New Issue
Block a user