1
0

Savepoint should not create a hole in the commit timeline

This commit is contained in:
Prasanna Rajaperumal
2017-06-22 15:00:27 -07:00
committed by prazanna
parent 29b906b763
commit 5cc071f74e
4 changed files with 64 additions and 8 deletions

View File

@@ -27,6 +27,8 @@ import com.uber.hoodie.common.file.HoodieAppendLog;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.util.Optional;
import java.util.function.Function;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
@@ -103,12 +105,16 @@ public class HoodieCommitArchiveLog {
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
List<String> savepoints = table.getSavepoints();
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
return commitTimeline.getInstants().filter(s -> !savepoints.contains(s.getTimestamp()))
.limit(commitTimeline.countInstants() - minCommitsToKeep);
return commitTimeline.getInstants().filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
HoodieTimeline.LESSER_OR_EQUAL));
}).limit(commitTimeline.countInstants() - minCommitsToKeep);
}
return Stream.empty();
}