1
0

Implement Savepoints and required metadata timeline (#86)

- Introduce avro to save clean metadata with details about the last commit that was retained
- Save rollback metadata in the meta timeline
- Create savepoint metadata and add API to createSavepoint, deleteSavepoint and rollbackToSavepoint
- Savepointed commit should not be rolledback or cleaned or archived
- introduce cli commands to show, create and rollback to savepoints
- Write unit tests to test savepoints and rollbackToSavepoints
This commit is contained in:
prazanna
2017-03-13 15:12:03 -07:00
committed by GitHub
parent 69d3950a32
commit 6f36e1eaaf
27 changed files with 1423 additions and 130 deletions

View File

@@ -25,6 +25,7 @@ import com.uber.hoodie.config.HoodieWriteConfig;
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 org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
@@ -71,17 +72,17 @@ public class HoodieCommitArchiveLog {
}
private Stream<HoodieInstant> getCommitsToArchive() {
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
List<String> savepoints = table.getSavepoints();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
return commitTimeline.getInstants()
return commitTimeline.getInstants().filter(s -> !savepoints.contains(s.getTimestamp()))
.limit(commitTimeline.countInstants() - minCommitsToKeep);
}
return Stream.empty();