Implement Savepoints and required metadata timeline - Part 2
This commit is contained in:
@@ -26,6 +26,7 @@ import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
@@ -33,8 +34,10 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
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.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -53,9 +56,12 @@ import com.uber.hoodie.metrics.HoodieMetrics;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import com.uber.hoodie.table.WorkloadProfile;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Accumulator;
|
||||
@@ -63,6 +69,7 @@ import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
@@ -78,6 +85,7 @@ import java.util.Collections;
|
||||
import java.util.Date;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
@@ -396,16 +404,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* Savepoint the latest commit. The data files and commit files for that commit will never be rolledback,
|
||||
* cleaned or archived. This gives an option to rollback the state to the savepoint anytime.
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
|
||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
|
||||
* will never be rolledback or archived.
|
||||
*
|
||||
* This gives an option to rollback the state to the savepoint anytime.
|
||||
* Savepoint needs to be manually created and deleted.
|
||||
*
|
||||
* Savepoint should be on a commit that is not cleaned.
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
* @param savePointMetadata - metadata about the savepoint
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(HoodieSavepointMetadata savePointMetadata) {
|
||||
public boolean savepoint(String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
if (table.getCompletedCommitTimeline().empty()) {
|
||||
@@ -414,20 +426,25 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
String latestCommit = table.getCompletedCommitTimeline().lastInstant().get().getTimestamp();
|
||||
logger.info("Savepointing latest commit " + latestCommit);
|
||||
return savepoint(latestCommit, savePointMetadata);
|
||||
return savepoint(latestCommit, user, comment);
|
||||
}
|
||||
|
||||
/**
|
||||
* Savepoint a specific commit. The data files and commit files for that commit will never be rolledback,
|
||||
* cleaned or archived. This gives an option to rollback the state to the savepoint anytime.
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
|
||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
|
||||
* will never be rolledback or archived.
|
||||
*
|
||||
* This gives an option to rollback the state to the savepoint anytime.
|
||||
* Savepoint needs to be manually created and deleted.
|
||||
*
|
||||
* Savepoint should be on a commit that is not cleaned.
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
* @param savePointMetadata - metadata about the savepoint
|
||||
* @param commitTime - commit that should be savepointed
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String commitTime, HoodieSavepointMetadata savePointMetadata) {
|
||||
public boolean savepoint(String commitTime, String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
@@ -455,10 +472,24 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
"Could not savepoint commit " + commitTime + " as this is beyond the lookup window "
|
||||
+ lastCommitRetained);
|
||||
|
||||
Map<String, List<String>> latestFilesMap = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath()))
|
||||
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
logger.info("Collecting latest files in partition path " + partitionPath);
|
||||
TableFileSystemView view = table.getFileSystemView();
|
||||
List<String> latestFiles =
|
||||
view.getLatestVersionInPartition(partitionPath, commitTime)
|
||||
.map(HoodieDataFile::getFileName).collect(Collectors.toList());
|
||||
return new Tuple2<String, List<String>>(partitionPath, latestFiles);
|
||||
}).collectAsMap();
|
||||
|
||||
HoodieSavepointMetadata metadata =
|
||||
AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap);
|
||||
// Nothing to save in the savepoint
|
||||
table.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, commitTime),
|
||||
AvroUtils.serializeSavepointMetadata(savePointMetadata));
|
||||
AvroUtils.serializeSavepointMetadata(metadata));
|
||||
logger.info("Savepoint " + commitTime + " created");
|
||||
return true;
|
||||
} catch (IOException e) {
|
||||
|
||||
Reference in New Issue
Block a user