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
@@ -16,9 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
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.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
@@ -28,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -51,6 +54,8 @@ import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -59,11 +64,24 @@ import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.Partitioner;
|
||||
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.PairFlatMapFunction;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where
|
||||
*
|
||||
@@ -73,7 +91,7 @@ import scala.Tuple2;
|
||||
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
|
||||
*
|
||||
*/
|
||||
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable {
|
||||
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable<T> {
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
@@ -501,6 +519,65 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a set of commits
|
||||
* @param partitionPath
|
||||
* @param commits
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits) throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileStatus[] toBeDeleted =
|
||||
fs.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 = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException {
|
||||
String actionType = this.getCompactedCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights = this.getInflightCommitTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Atomically unpublish all the commits
|
||||
commits.stream().filter(s -> !inflights.contains(s))
|
||||
.map(s -> new HoodieInstant(false, actionType, s))
|
||||
.forEach(activeTimeline::revertToInflight);
|
||||
logger.info("Unpublished " + commits);
|
||||
|
||||
// delete all the data files for all these commits
|
||||
logger.info("Clean out all parquet files generated for commits: " + commits);
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(FSUtils.getFs(), this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
Map<FileStatus, Boolean> results = deleteCleanedFiles(partitionPath, commits);
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
commits.stream().map(s -> new HoodieInstant(true, actionType, s))
|
||||
.forEach(activeTimeline::deleteInflight);
|
||||
logger.info("Deleted inflight commits " + commits);
|
||||
return stats;
|
||||
}
|
||||
|
||||
private static class PartitionCleanStat implements Serializable {
|
||||
private final String partitionPath;
|
||||
private final List<String> deletePathPatterns = new ArrayList<>();
|
||||
|
||||
@@ -16,25 +16,47 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
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.log.HoodieLogFormat;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import java.util.Optional;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where
|
||||
@@ -45,6 +67,8 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id.
|
||||
* Compaction merges the log file into the base file.
|
||||
*
|
||||
* WARNING - MOR table type does not support nested rollbacks, every rollback
|
||||
* must be followed by an attempted commit action
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
@@ -94,4 +118,117 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException {
|
||||
|
||||
//At the moment, MOR table type does not support nested rollbacks
|
||||
if(commits.size() > 1) {
|
||||
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
|
||||
}
|
||||
Map<String, HoodieInstant> commitsAndCompactions =
|
||||
this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
|
||||
// Atomically un-publish all non-inflight commits
|
||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
||||
|
||||
logger.info("Unpublished " + commits);
|
||||
|
||||
Long startTime = System.currentTimeMillis();
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
List<HoodieRollbackStat> stats = null;
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback Commit/Compaction " + instant);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
logger.info("Finished rollback of Commit/Compaction " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback delta commit " + instant);
|
||||
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
|
||||
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
|
||||
filesToDeletedStatus = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
|
||||
// append rollback blocks for updates
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT).forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(FSUtils.getFs()).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
// if update belongs to an existing log file
|
||||
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
|
||||
numRollbackBlocks++;
|
||||
if(wStat.getNumDeletes() > 0) {
|
||||
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
|
||||
numRollbackBlocks++;
|
||||
}
|
||||
filesToNumBlocksRollback.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException("Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
});
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}).collect();
|
||||
logger.info("Fnished rollback of delta commit " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
}
|
||||
return stats;
|
||||
}).flatMap(x -> x.stream()).collect(Collectors.toList());
|
||||
|
||||
commitsAndCompactions.entrySet().stream()
|
||||
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp()))
|
||||
.forEach(this.getActiveTimeline()::deleteInflight);
|
||||
|
||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
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.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
@@ -33,6 +34,7 @@ import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.exception.HoodieSavepointException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
@@ -327,4 +329,17 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* of files cleaned.
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time.
|
||||
* Four steps:
|
||||
* (1) Atomically unpublish this commit
|
||||
* (2) clean indexing data
|
||||
* (3) clean new generated parquet files / log blocks
|
||||
* (4) Finally, delete .<action>.commit or .<action>.inflight file
|
||||
* @param commits
|
||||
* @return
|
||||
* @throws HoodieRollbackException
|
||||
*/
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user