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

@@ -21,6 +21,7 @@ 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.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
@@ -35,6 +36,7 @@ import org.apache.spark.Partitioner;
import java.io.Serializable;
import java.util.Iterator;
import java.util.List;
import java.util.stream.Collectors;
/**
* Abstract implementation of a HoodieTable
@@ -111,6 +113,36 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
return getCommitTimeline().filterCompletedInstants();
}
/**
* Get only the inflights (no-completed) commit timeline
* @return
*/
public HoodieTimeline getInflightCommitTimeline() {
return getCommitTimeline().filterInflights();
}
/**
* Get only the completed (no-inflights) clean timeline
* @return
*/
public HoodieTimeline getCompletedCleanTimeline() {
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
}
/**
* Get only the completed (no-inflights) savepoint timeline
* @return
*/
public HoodieTimeline getCompletedSavepointTimeline() {
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
}
public List<String> getSavepoints() {
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
}
public HoodieActiveTimeline getActiveTimeline() {
return metaClient.getActiveTimeline();
}