1
0

1) Separated rollback as a table operation 2) Implement rollback for MOR

This commit is contained in:
Nishith Agarwal
2017-08-17 17:31:55 -07:00
committed by vinoth chandar
parent e1fe3ab937
commit c7d63a7622
28 changed files with 1200 additions and 260 deletions

View File

@@ -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 =

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.io;
import com.beust.jcommander.internal.Maps;
import com.clearspring.analytics.util.Lists;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
@@ -27,6 +28,7 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
@@ -45,6 +47,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
@@ -157,6 +160,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
List<IndexedRecord> recordList = new ArrayList<>();
List<String> keysToDelete = new ArrayList<>();
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
records.stream().forEach(record -> {
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if(indexedRecord.isPresent()) {
@@ -166,9 +171,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
}
});
try {
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema));
if(recordList.size() > 0) {
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
}
if(keysToDelete.size() > 0) {
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new)));
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
}
} catch (Exception e) {
throw new HoodieAppendException(

View File

@@ -52,6 +52,7 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Archiver to bound the growth of <action>.commit files
*/

View File

@@ -18,6 +18,7 @@ package com.uber.hoodie.io.compact;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.CompactionWriteStat;
import com.uber.hoodie.common.model.HoodieAvroPayload;
@@ -147,7 +148,14 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, operation.getDeltaFilePaths(), readerSchema);
String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.COMPACTION_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
if (!scanner.iterator().hasNext()) {
return Lists.newArrayList();
}

View File

@@ -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<>();

View File

@@ -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;
}
}

View File

@@ -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;
}