1) Separated rollback as a table operation 2) Implement rollback for MOR
This commit is contained in:
committed by
vinoth chandar
parent
e1fe3ab937
commit
c7d63a7622
@@ -19,7 +19,6 @@ package com.uber.hoodie;
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||
@@ -54,20 +53,16 @@ import com.uber.hoodie.metrics.HoodieMetrics;
|
||||
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import com.uber.hoodie.table.WorkloadProfile;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import com.uber.hoodie.table.WorkloadStat;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
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.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
@@ -297,6 +292,39 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return records;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Save the workload profile in an intermediate file (here re-using commit files)
|
||||
* This is useful when performing rollback for MOR datasets. Only updates are recorded
|
||||
* in the workload profile metadata since updates to log blocks are unknown across batches
|
||||
* Inserts (which are new parquet files) are rolled back based on commit time.
|
||||
* // TODO : Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
|
||||
* @param profile
|
||||
* @param commitTime
|
||||
* @throws HoodieCommitException
|
||||
*/
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table, String commitTime) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
profile.getPartitionPaths().stream().forEach(path -> {
|
||||
WorkloadStat partitionStat = profile.getWorkloadStat(path.toString());
|
||||
partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> {
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId(entry.getKey());
|
||||
writeStat.setNumUpdateWrites(entry.getValue());
|
||||
metadata.addWriteStat(path.toString(), writeStat);
|
||||
});
|
||||
});
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
Optional<HoodieInstant> instant = activeTimeline.filterInflights().lastInstant();
|
||||
activeTimeline.saveToInflight(instant.get(),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch(IOException io) {
|
||||
throw new HoodieCommitException("Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
@@ -309,6 +337,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
if (hoodieTable.isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(preppedRecords);
|
||||
logger.info("Workload profile :" + profile);
|
||||
saveWorkloadProfileMetadataToInflight(profile, hoodieTable, commitTime);
|
||||
}
|
||||
|
||||
// partition using the insert partitioner
|
||||
@@ -608,7 +637,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
private void rollback(List<String> commits) {
|
||||
if(commits.isEmpty()) {
|
||||
logger.info("List of commits to rollback is empty");
|
||||
@@ -660,60 +688,21 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
", please rollback greater commits first");
|
||||
}
|
||||
|
||||
// Atomically unpublish all the commits
|
||||
commits.stream().filter(s -> !inflights.contains(s))
|
||||
.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s))
|
||||
.forEach(activeTimeline::revertToInflight);
|
||||
logger.info("Unpublished " + commits);
|
||||
List<HoodieRollbackStat> stats = table.rollback(jsc, commits);
|
||||
|
||||
// cleanup index entries
|
||||
commits.stream().forEach(s -> {
|
||||
if (!index.rollbackCommit(s)) {
|
||||
throw new HoodieRollbackException(
|
||||
"Clean out index changes failed, for time :" + s);
|
||||
"Rollback index changes failed, for time :" + s);
|
||||
}
|
||||
});
|
||||
logger.info("Index rolled back for commits " + commits);
|
||||
|
||||
// delete all the data files for all these commits
|
||||
logger.info("Clean out all parquet files generated for commits: " + commits);
|
||||
final LongAccumulator numFilesDeletedCounter = jsc.sc().longAccumulator();
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs1 = FSUtils.getFs();
|
||||
FileStatus[] toBeDeleted =
|
||||
fs1.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
|
||||
if(!path.toString().contains(".parquet")) {
|
||||
return false;
|
||||
}
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commits.contains(fileCommitTime);
|
||||
});
|
||||
Map<FileStatus, Boolean> results = Maps.newHashMap();
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs1.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
if (success) {
|
||||
numFilesDeletedCounter.add(1);
|
||||
}
|
||||
}
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
commits.stream().map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))
|
||||
.forEach(activeTimeline::deleteInflight);
|
||||
logger.info("Deleted inflight commits " + commits);
|
||||
|
||||
Optional<Long> durationInMs = Optional.empty();
|
||||
if (context != null) {
|
||||
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
||||
Long numFilesDeleted = numFilesDeletedCounter.value();
|
||||
Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum();
|
||||
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||
}
|
||||
HoodieRollbackMetadata rollbackMetadata =
|
||||
|
||||
Reference in New Issue
Block a user