Refactor HoodieTable Rollback to write one rollback instant for a batch of commits to rollback
This commit is contained in:
@@ -19,10 +19,10 @@ package com.uber.hoodie;
|
|||||||
import com.codahale.metrics.Timer;
|
import com.codahale.metrics.Timer;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||||
|
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||||
import com.uber.hoodie.common.HoodieCleanStat;
|
import com.uber.hoodie.common.HoodieCleanStat;
|
||||||
@@ -65,13 +65,13 @@ import java.io.IOException;
|
|||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
@@ -758,7 +758,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||||
logger.info("Rolling back commits " + commitsToRollback);
|
logger.info("Rolling back commits " + commitsToRollback);
|
||||||
|
|
||||||
rollback(commitsToRollback);
|
restoreToInstant(savepointTime);
|
||||||
|
|
||||||
// Make sure the rollback was successful
|
// Make sure the rollback was successful
|
||||||
Optional<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline()
|
Optional<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline()
|
||||||
@@ -776,7 +776,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
* files. (4) Finally delete .commit or .inflight file,
|
* files. (4) Finally delete .commit or .inflight file,
|
||||||
*/
|
*/
|
||||||
public boolean rollback(final String commitTime) throws HoodieRollbackException {
|
public boolean rollback(final String commitTime) throws HoodieRollbackException {
|
||||||
rollback(Lists.newArrayList(commitTime));
|
rollbackInternal(commitTime);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -787,137 +787,190 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
* files and/or append rollback to existing log files. (4) Finally delete .commit, .inflight, .compaction.inflight
|
* files and/or append rollback to existing log files. (4) Finally delete .commit, .inflight, .compaction.inflight
|
||||||
* or .compaction.requested file
|
* or .compaction.requested file
|
||||||
*/
|
*/
|
||||||
public void restoreToCommit(final String commitTime) throws HoodieRollbackException {
|
public void restoreToInstant(final String instantTime) throws HoodieRollbackException {
|
||||||
|
|
||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
||||||
// Get all the commits on the timeline after the provided commit time
|
// Get all the commits on the timeline after the provided commit time
|
||||||
List<HoodieInstant> commitsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline().getInstants()
|
List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getCommitsAndCompactionTimeline().getInstants()
|
||||||
.filter(instant -> HoodieActiveTimeline.GREATER.test(instant.getTimestamp(), commitTime))
|
.filter(instant -> HoodieActiveTimeline.GREATER.test(instant.getTimestamp(), instantTime))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
// reverse the commits to descending order of commit time
|
// reverse the commits to descending order of commit time
|
||||||
Collections.reverse(commitsToRollback);
|
Collections.reverse(instantsToRollback);
|
||||||
commitsToRollback.stream().forEach(instant -> {
|
// Start a rollback instant for all commits to be rolled back
|
||||||
switch (instant.getAction()) {
|
String startRollbackInstant = startInstant();
|
||||||
case HoodieTimeline.COMMIT_ACTION:
|
// Start the timer
|
||||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
final Timer.Context context = startContext();
|
||||||
rollback(instant.getTimestamp());
|
ImmutableMap.Builder<String, List<HoodieRollbackStat>> instantsToStats =
|
||||||
break;
|
ImmutableMap.builder();
|
||||||
case HoodieTimeline.COMPACTION_ACTION:
|
instantsToRollback.stream().forEach(instant -> {
|
||||||
if (instant.isRequested()) {
|
|
||||||
// TODO : Get file status and create a rollback stat and file
|
|
||||||
// TODO : Delete the .aux files along with the instant file, okay for now since the archival process will
|
|
||||||
// delete these files when it does not see a corresponding instant file under .hoodie
|
|
||||||
deleteRequestedCompaction(instant.getTimestamp());
|
|
||||||
logger.info("Deleted pending scheduled compaction " + instant.getTimestamp());
|
|
||||||
} else {
|
|
||||||
rollback(instant.getTimestamp());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("invalid action name " + instant.getAction());
|
|
||||||
}
|
|
||||||
try {
|
try {
|
||||||
// Ensure unique rollback instants for seconds granularity
|
switch (instant.getAction()) {
|
||||||
Thread.sleep(1000);
|
case HoodieTimeline.COMMIT_ACTION:
|
||||||
} catch (InterruptedException e) {
|
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||||
throw new HoodieRollbackException("unable to rollback instant " + instant, e);
|
List<HoodieRollbackStat> statsForInstant = doRollbackAndGetStats(instant.getTimestamp());
|
||||||
|
instantsToStats.put(instant.getTimestamp(), statsForInstant);
|
||||||
|
break;
|
||||||
|
case HoodieTimeline.COMPACTION_ACTION:
|
||||||
|
if (instant.isRequested()) {
|
||||||
|
// TODO : Get file status and create a rollback stat and file
|
||||||
|
// TODO : Delete the .aux files along with the instant file, okay for now since the archival process will
|
||||||
|
// delete these files when it does not see a corresponding instant file under .hoodie
|
||||||
|
deleteRequestedCompaction(instant.getTimestamp());
|
||||||
|
logger.info("Deleted pending scheduled compaction " + instant.getTimestamp());
|
||||||
|
} else {
|
||||||
|
List<HoodieRollbackStat> statsForCompaction = doRollbackAndGetStats(instant.getTimestamp());
|
||||||
|
instantsToStats.put(instant.getTimestamp(), statsForCompaction);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("invalid action name " + instant.getAction());
|
||||||
|
}
|
||||||
|
} catch (IOException io) {
|
||||||
|
throw new HoodieRollbackException("unable to rollback instant " + instant, io);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
try {
|
||||||
|
finishRestore(context, instantsToStats.build(),
|
||||||
|
instantsToRollback.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
|
||||||
|
startRollbackInstant, instantTime);
|
||||||
|
} catch (IOException io) {
|
||||||
|
throw new HoodieRollbackException("unable to rollback instants " + instantsToRollback, io);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void rollback(List<String> commitsToRollback) {
|
private String startInstant() {
|
||||||
if (commitsToRollback.isEmpty()) {
|
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||||
logger.info("List of commits to rollback is empty");
|
}
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
final Timer.Context context = metrics.getRollbackCtx();
|
private Timer.Context startContext() {
|
||||||
String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
return metrics.getRollbackCtx();
|
||||||
|
}
|
||||||
|
|
||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
private List<HoodieRollbackStat> doRollbackAndGetStats(final String commitToRollback) throws
|
||||||
|
IOException {
|
||||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
||||||
Set<String> pendingCompactions =
|
|
||||||
table.getActiveTimeline().filterPendingCompactionTimeline().getInstants()
|
|
||||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
|
|
||||||
HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline();
|
HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline();
|
||||||
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
|
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
|
||||||
|
|
||||||
// Check if any of the commits is a savepoint - do not allow rollback on those commits
|
// Check if any of the commits is a savepoint - do not allow rollback on those commits
|
||||||
List<String> savepoints = table.getCompletedSavepointTimeline().getInstants()
|
List<String> savepoints = table.getCompletedSavepointTimeline().getInstants()
|
||||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||||
commitsToRollback.forEach(s -> {
|
savepoints.stream().forEach(s -> {
|
||||||
if (savepoints.contains(s)) {
|
if (s.contains(commitToRollback)) {
|
||||||
throw new HoodieRollbackException(
|
throw new HoodieRollbackException(
|
||||||
"Could not rollback a savepointed commit. Delete savepoint first before rolling back"
|
"Could not rollback a savepointed commit. Delete savepoint first before rolling back"
|
||||||
+ s);
|
+ s);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
if (commitTimeline.empty() && inflightCommitTimeline.empty()) {
|
||||||
|
// nothing to rollback
|
||||||
|
logger.info("No commits to rollback " + commitToRollback);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Make sure only the last n commits are being rolled back
|
||||||
|
// If there is a commit in-between or after that is not rolled back, then abort
|
||||||
|
String lastCommit = commitToRollback;
|
||||||
|
|
||||||
|
if ((lastCommit != null) && !commitTimeline.empty() && !commitTimeline
|
||||||
|
.findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) {
|
||||||
|
throw new HoodieRollbackException(
|
||||||
|
"Found commits after time :" + lastCommit + ", please rollback greater commits first");
|
||||||
|
}
|
||||||
|
|
||||||
|
List<String> inflights = inflightCommitTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
if ((lastCommit != null) && !inflights.isEmpty() && (inflights.indexOf(lastCommit) != inflights.size() - 1)) {
|
||||||
|
throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit
|
||||||
|
+ ", please rollback greater commits first");
|
||||||
|
}
|
||||||
|
|
||||||
|
List<HoodieRollbackStat> stats = table.rollback(jsc, commitToRollback, true);
|
||||||
|
|
||||||
|
logger.info("Deleted inflight commits " + commitToRollback);
|
||||||
|
|
||||||
|
// cleanup index entries
|
||||||
|
if (!index.rollbackCommit(commitToRollback)) {
|
||||||
|
throw new HoodieRollbackException("Rollback index changes failed, for time :" + commitToRollback);
|
||||||
|
}
|
||||||
|
logger.info("Index rolled back for commits " + commitToRollback);
|
||||||
|
return stats;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void finishRollback(final Timer.Context context, List<HoodieRollbackStat> rollbackStats,
|
||||||
|
List<String> commitsToRollback, final String startRollbackTime) throws IOException {
|
||||||
|
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||||
|
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
||||||
|
Optional<Long> durationInMs = Optional.empty();
|
||||||
|
Long numFilesDeleted = rollbackStats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum();
|
||||||
|
if (context != null) {
|
||||||
|
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
||||||
|
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||||
|
}
|
||||||
|
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
||||||
|
.convertRollbackMetadata(startRollbackTime, durationInMs, commitsToRollback, rollbackStats);
|
||||||
|
table.getActiveTimeline().saveAsComplete(
|
||||||
|
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime),
|
||||||
|
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||||
|
logger.info("Commits " + commitsToRollback + " rollback is complete");
|
||||||
|
|
||||||
|
if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
|
||||||
|
logger.info("Cleaning up older rollback meta files");
|
||||||
|
// Cleanup of older cleaner meta files
|
||||||
|
// TODO - make the commit archival generic and archive rollback metadata
|
||||||
|
FSUtils.deleteOlderRollbackMetaFiles(fs, table.getMetaClient().getMetaPath(),
|
||||||
|
table.getActiveTimeline().getRollbackTimeline().getInstants());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void finishRestore(final Timer.Context context, Map<String, List<HoodieRollbackStat>> commitToStats,
|
||||||
|
List<String> commitsToRollback, final String startRestoreTime, final String restoreToInstant) throws IOException {
|
||||||
|
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||||
|
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
||||||
|
Optional<Long> durationInMs = Optional.empty();
|
||||||
|
Long numFilesDeleted = 0L;
|
||||||
|
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
|
||||||
|
List<HoodieRollbackStat> stats = commitToStat.getValue();
|
||||||
|
numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size())
|
||||||
|
.sum();
|
||||||
|
}
|
||||||
|
if (context != null) {
|
||||||
|
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
||||||
|
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||||
|
}
|
||||||
|
HoodieRestoreMetadata restoreMetadata = AvroUtils
|
||||||
|
.convertRestoreMetadata(startRestoreTime, durationInMs, commitsToRollback, commitToStats);
|
||||||
|
table.getActiveTimeline().saveAsComplete(
|
||||||
|
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, startRestoreTime),
|
||||||
|
AvroUtils.serializeRestoreMetadata(restoreMetadata));
|
||||||
|
logger.info("Commits " + commitsToRollback + " rollback is complete. Restored dataset to " + restoreToInstant);
|
||||||
|
|
||||||
|
if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
|
||||||
|
logger.info("Cleaning up older restore meta files");
|
||||||
|
// Cleanup of older cleaner meta files
|
||||||
|
// TODO - make the commit archival generic and archive rollback metadata
|
||||||
|
FSUtils.deleteOlderRollbackMetaFiles(fs, table.getMetaClient().getMetaPath(),
|
||||||
|
table.getActiveTimeline().getRestoreTimeline().getInstants());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void rollbackInternal(String commitToRollback) {
|
||||||
|
if (commitToRollback.isEmpty()) {
|
||||||
|
logger.info("List of commits to rollback is empty");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
final String startRollbackTime = startInstant();
|
||||||
|
final Timer.Context context = startContext();
|
||||||
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
try {
|
try {
|
||||||
if (commitTimeline.empty() && inflightCommitTimeline.empty()) {
|
List<HoodieRollbackStat> stats = doRollbackAndGetStats(commitToRollback);
|
||||||
// nothing to rollback
|
Map<String, List<HoodieRollbackStat>> statToCommit = new HashMap<>();
|
||||||
logger.info("No commits to rollback " + commitsToRollback);
|
finishRollback(context, stats, Arrays.asList(commitToRollback), startRollbackTime);
|
||||||
}
|
|
||||||
|
|
||||||
// Make sure only the last n commits are being rolled back
|
|
||||||
// If there is a commit in-between or after that is not rolled back, then abort
|
|
||||||
String lastCommit = null;
|
|
||||||
if (!commitsToRollback.isEmpty()) {
|
|
||||||
lastCommit = commitsToRollback.get(commitsToRollback.size() - 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
if ((lastCommit != null) && !commitTimeline.empty() && !commitTimeline
|
|
||||||
.findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) {
|
|
||||||
throw new HoodieRollbackException(
|
|
||||||
"Found commits after time :" + lastCommit + ", please rollback greater commits first");
|
|
||||||
}
|
|
||||||
|
|
||||||
List<String> inflights = inflightCommitTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
if ((lastCommit != null) && !inflights.isEmpty() && (inflights.indexOf(lastCommit) != inflights.size() - 1)) {
|
|
||||||
throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit
|
|
||||||
+ ", please rollback greater commits first");
|
|
||||||
}
|
|
||||||
|
|
||||||
List<HoodieRollbackStat> stats = table.rollback(jsc, commitsToRollback, true);
|
|
||||||
|
|
||||||
logger.info("Deleted inflight commits " + commitsToRollback);
|
|
||||||
|
|
||||||
// cleanup index entries
|
|
||||||
commitsToRollback.forEach(s -> {
|
|
||||||
if (!index.rollbackCommit(s)) {
|
|
||||||
throw new HoodieRollbackException("Rollback index changes failed, for time :" + s);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
logger.info("Index rolled back for commits " + commitsToRollback);
|
|
||||||
|
|
||||||
Optional<Long> durationInMs = Optional.empty();
|
|
||||||
if (context != null) {
|
|
||||||
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
|
||||||
Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size())
|
|
||||||
.sum();
|
|
||||||
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
|
||||||
}
|
|
||||||
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
|
||||||
.convertRollbackMetadata(startRollbackTime, durationInMs, commitsToRollback, stats);
|
|
||||||
table.getActiveTimeline().saveAsComplete(
|
|
||||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime),
|
|
||||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
|
||||||
logger.info("Commits " + commitsToRollback + " rollback is complete");
|
|
||||||
|
|
||||||
if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
|
|
||||||
logger.info("Cleaning up older rollback meta files");
|
|
||||||
// Cleanup of older cleaner meta files
|
|
||||||
// TODO - make the commit archival generic and archive rollback metadata
|
|
||||||
FSUtils.deleteOlderRollbackMetaFiles(fs, table.getMetaClient().getMetaPath(),
|
|
||||||
table.getActiveTimeline().getRollbackTimeline().getInstants());
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieRollbackException(
|
throw new HoodieRollbackException(
|
||||||
"Failed to rollback " + config.getBasePath() + " commits " + commitsToRollback, e);
|
"Failed to rollback " + config.getBasePath() + " commits " + commitToRollback, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -1251,7 +1304,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) throws IOException {
|
void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) throws IOException {
|
||||||
table.rollback(jsc, ImmutableList.copyOf(new String[]{inflightInstant.getTimestamp()}), false);
|
table.rollback(jsc, inflightInstant.getTimestamp(), false);
|
||||||
// Revert instant state file
|
// Revert instant state file
|
||||||
table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
|
table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -330,7 +330,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||||
* set of commits
|
* set of commits
|
||||||
*/
|
*/
|
||||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, List<String> commits, String
|
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String commit, String
|
||||||
partitionPath)
|
partitionPath)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
logger.info("Cleaning path " + partitionPath);
|
logger.info("Cleaning path " + partitionPath);
|
||||||
@@ -338,7 +338,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
PathFilter filter = (path) -> {
|
PathFilter filter = (path) -> {
|
||||||
if (path.toString().contains(".parquet")) {
|
if (path.toString().contains(".parquet")) {
|
||||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||||
return commits.contains(fileCommitTime);
|
return commit.equals(fileCommitTime);
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
};
|
};
|
||||||
@@ -352,28 +352,27 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits, boolean deleteInstants)
|
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
String actionType = metaClient.getCommitActionType();
|
String actionType = metaClient.getCommitActionType();
|
||||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||||
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
||||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||||
|
// Atomically unpublish the commits
|
||||||
|
if (!inflights.contains(commit)) {
|
||||||
|
activeTimeline.revertToInflight(new HoodieInstant(false, actionType, commit));
|
||||||
|
}
|
||||||
|
logger.info("Unpublished " + commit);
|
||||||
|
|
||||||
// Atomically unpublish all the commits
|
// delete all the data files for this commit
|
||||||
commits.stream().filter(s -> !inflights.contains(s))
|
logger.info("Clean out all parquet files generated for commit: " + commit);
|
||||||
.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
|
List<HoodieRollbackStat> stats = jsc.parallelize(FSUtils
|
||||||
.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
||||||
config.shouldAssumeDatePartitioning()))
|
config.shouldAssumeDatePartitioning()))
|
||||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||||
// Scan all partitions files with this commit time
|
// Scan all partitions files with this commit time
|
||||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||||
deleteCleanedFiles(filesToDeletedStatus, commits, partitionPath);
|
deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
|
||||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||||
.withDeletedFileResults(filesToDeletedStatus).build();
|
.withDeletedFileResults(filesToDeletedStatus).build();
|
||||||
}).collect();
|
}).collect();
|
||||||
@@ -381,26 +380,26 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
|||||||
// clean temporary data files
|
// clean temporary data files
|
||||||
cleanTemporaryDataFiles(jsc);
|
cleanTemporaryDataFiles(jsc);
|
||||||
|
|
||||||
// Delete Inflight instants if enabled
|
// Delete Inflight instant if enabled
|
||||||
deleteInflightInstants(deleteInstants, activeTimeline,
|
deleteInflightInstant(deleteInstants, activeTimeline,
|
||||||
commits.stream().map(s -> new HoodieInstant(true, actionType, s)).collect(Collectors.toList()));
|
new HoodieInstant(true, actionType, commit));
|
||||||
return stats;
|
return stats;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete Inflight instants if enabled
|
* Delete Inflight instant if enabled
|
||||||
* @param deleteInstants Enable Deletion of Inflight instants
|
* @param deleteInstant Enable Deletion of Inflight instant
|
||||||
* @param activeTimeline Hoodie active timeline
|
* @param activeTimeline Hoodie active timeline
|
||||||
* @param instantsToBeDeleted Instants to be deleted
|
* @param instantToBeDeleted Instant to be deleted
|
||||||
*/
|
*/
|
||||||
protected static void deleteInflightInstants(boolean deleteInstants, HoodieActiveTimeline activeTimeline,
|
protected static void deleteInflightInstant(boolean deleteInstant, HoodieActiveTimeline activeTimeline,
|
||||||
List<HoodieInstant> instantsToBeDeleted) {
|
HoodieInstant instantToBeDeleted) {
|
||||||
// Remove the rolled back inflight commits
|
// Remove the rolled back inflight commits
|
||||||
if (deleteInstants) {
|
if (deleteInstant) {
|
||||||
instantsToBeDeleted.forEach(activeTimeline::deleteInflight);
|
activeTimeline.deleteInflight(instantToBeDeleted);
|
||||||
logger.info("Deleted inflight commits " + instantsToBeDeleted);
|
logger.info("Deleted inflight commit " + instantToBeDeleted);
|
||||||
} else {
|
} else {
|
||||||
logger.warn("Rollback finished without deleting inflight instant files. Instants=" + instantsToBeDeleted);
|
logger.warn("Rollback finished without deleting inflight instant file. Instant=" + instantToBeDeleted);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -174,7 +174,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits, boolean deleteInstants)
|
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
// At the moment, MOR table type does not support bulk nested rollbacks. Nested rollbacks is an experimental
|
// At the moment, MOR table type does not support bulk nested rollbacks. Nested rollbacks is an experimental
|
||||||
@@ -182,26 +182,23 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
// (commitToRollback).
|
// (commitToRollback).
|
||||||
// NOTE {@link HoodieCompactionConfig#withCompactionLazyBlockReadEnabled} needs to be set to TRUE. This is
|
// NOTE {@link HoodieCompactionConfig#withCompactionLazyBlockReadEnabled} needs to be set to TRUE. This is
|
||||||
// required to avoid OOM when merging multiple LogBlocks performed during nested rollbacks.
|
// required to avoid OOM when merging multiple LogBlocks performed during nested rollbacks.
|
||||||
if (commits.size() > 1) {
|
|
||||||
throw new UnsupportedOperationException("Bulk Nested Rollbacks are not supported");
|
|
||||||
}
|
|
||||||
// Atomically un-publish all non-inflight commits
|
// Atomically un-publish all non-inflight commits
|
||||||
Map<String, HoodieInstant> commitsAndCompactions = this.getActiveTimeline()
|
Optional<HoodieInstant> commitOrCompactionOption = this.getActiveTimeline()
|
||||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
|
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
|
||||||
.filter(i -> commits.contains(i.getTimestamp()))
|
.filter(i -> commit.equals(i.getTimestamp()))
|
||||||
.collect(Collectors.toMap(HoodieInstant::getTimestamp, i -> i));
|
.findFirst();
|
||||||
|
HoodieInstant instantToRollback = commitOrCompactionOption.get();
|
||||||
// Atomically un-publish all non-inflight commits
|
// Atomically un-publish all non-inflight commits
|
||||||
commitsAndCompactions.entrySet().stream().map(Map.Entry::getValue)
|
if (!instantToRollback.isInflight()) {
|
||||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
this.getActiveTimeline().revertToInflight(instantToRollback);
|
||||||
logger.info("Unpublished " + commits);
|
}
|
||||||
|
logger.info("Unpublished " + commit);
|
||||||
Long startTime = System.currentTimeMillis();
|
Long startTime = System.currentTimeMillis();
|
||||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
||||||
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||||
config.shouldAssumeDatePartitioning()))
|
config.shouldAssumeDatePartitioning()))
|
||||||
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> commits.stream().map(commit -> {
|
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
|
||||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
|
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
|
||||||
HoodieRollbackStat hoodieRollbackStats = null;
|
HoodieRollbackStat hoodieRollbackStats = null;
|
||||||
// Need to put the path filter here since Filter is not serializable
|
// Need to put the path filter here since Filter is not serializable
|
||||||
@@ -209,18 +206,18 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
PathFilter filter = (path) -> {
|
PathFilter filter = (path) -> {
|
||||||
if (path.toString().contains(".parquet")) {
|
if (path.toString().contains(".parquet")) {
|
||||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||||
return commits.contains(fileCommitTime);
|
return commit.equals(fileCommitTime);
|
||||||
} else if (path.toString().contains(".log")) {
|
} else if (path.toString().contains(".log")) {
|
||||||
// Since the baseCommitTime is the only commit for new log files, it's okay here
|
// Since the baseCommitTime is the only commit for new log files, it's okay here
|
||||||
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
|
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
|
||||||
return commits.contains(fileCommitTime);
|
return commit.equals(fileCommitTime);
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
};
|
};
|
||||||
|
|
||||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||||
|
|
||||||
switch (instant.getAction()) {
|
switch (instantToRollback.getAction()) {
|
||||||
case HoodieTimeline.COMMIT_ACTION:
|
case HoodieTimeline.COMMIT_ACTION:
|
||||||
try {
|
try {
|
||||||
// Rollback of a commit should delete the newly created parquet files along with any log
|
// Rollback of a commit should delete the newly created parquet files along with any log
|
||||||
@@ -244,7 +241,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
// and has not yet finished. In this scenario we should delete only the newly created parquet files
|
// and has not yet finished. In this scenario we should delete only the newly created parquet files
|
||||||
// and not corresponding base commit log files created with this as baseCommit since updates would
|
// and not corresponding base commit log files created with this as baseCommit since updates would
|
||||||
// have been written to the log files.
|
// have been written to the log files.
|
||||||
super.deleteCleanedFiles(filesToDeletedStatus, commits, partitionPath);
|
super.deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
|
||||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||||
} else {
|
} else {
|
||||||
@@ -286,7 +283,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
try {
|
try {
|
||||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||||
metaClient.getCommitTimeline().getInstantDetails(
|
metaClient.getCommitTimeline().getInstantDetails(
|
||||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp()))
|
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()))
|
||||||
.get(), HoodieCommitMetadata.class);
|
.get(), HoodieCommitMetadata.class);
|
||||||
|
|
||||||
// read commit file and (either append delete blocks or delete file)
|
// read commit file and (either append delete blocks or delete file)
|
||||||
@@ -315,13 +312,11 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
return hoodieRollbackStats;
|
return hoodieRollbackStats;
|
||||||
}).collect(Collectors.toList())).flatMap(List::iterator).filter(Objects::nonNull).collect();
|
}).filter(Objects::nonNull).collect();
|
||||||
|
|
||||||
// Delete Inflight instants if enabled
|
// Delete Inflight instants if enabled
|
||||||
deleteInflightInstants(deleteInstants, this.getActiveTimeline(),
|
deleteInflightInstant(deleteInstants, this.getActiveTimeline(), new HoodieInstant(true, instantToRollback
|
||||||
commitsAndCompactions.entrySet().stream().map(
|
.getAction(), instantToRollback.getTimestamp()));
|
||||||
entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp()))
|
|
||||||
.collect(Collectors.toList()));
|
|
||||||
|
|
||||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||||
|
|
||||||
|
|||||||
@@ -253,7 +253,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
|||||||
* Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files
|
* 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 if deleteInstants = true
|
* / log blocks (4) Finally, delete .<action>.commit or .<action>.inflight file if deleteInstants = true
|
||||||
*/
|
*/
|
||||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits, boolean deleteInstants)
|
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|||||||
@@ -114,7 +114,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
|||||||
// Reload and rollback inflight compaction
|
// Reload and rollback inflight compaction
|
||||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||||
hoodieTable.rollback(jsc, Arrays.asList(compactionInstantTime), false);
|
hoodieTable.rollback(jsc, compactionInstantTime, false);
|
||||||
|
|
||||||
client.rollbackInflightCompaction(
|
client.rollbackInflightCompaction(
|
||||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||||
|
|||||||
@@ -55,7 +55,6 @@ import com.uber.hoodie.table.HoodieTable;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@@ -628,19 +627,12 @@ public class TestCleaner extends TestHoodieClientBase {
|
|||||||
assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles());
|
assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles());
|
||||||
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
.withUseTempFolderCopyOnWriteForCreate(false)
|
.withUseTempFolderCopyOnWriteForCreate(true)
|
||||||
.withUseTempFolderCopyOnWriteForMerge(false).build();
|
.withUseTempFolderCopyOnWriteForMerge(false).build();
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(
|
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config
|
||||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
|
.getBasePath(), true),
|
||||||
jsc);
|
|
||||||
table.rollback(jsc, Collections.emptyList(), true);
|
|
||||||
assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles());
|
|
||||||
|
|
||||||
config = HoodieWriteConfig.newBuilder().withPath(basePath).withUseTempFolderCopyOnWriteForCreate(true)
|
|
||||||
.withUseTempFolderCopyOnWriteForMerge(false).build();
|
|
||||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
|
||||||
config, jsc);
|
config, jsc);
|
||||||
table.rollback(jsc, Collections.emptyList(), true);
|
table.rollback(jsc, "000", true);
|
||||||
assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
|
assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -721,7 +721,7 @@ public class TestMergeOnReadTable {
|
|||||||
copyOfRecords.clear();
|
copyOfRecords.clear();
|
||||||
|
|
||||||
// Rollback latest commit first
|
// Rollback latest commit first
|
||||||
client.restoreToCommit("000");
|
client.restoreToInstant("000");
|
||||||
|
|
||||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||||
|
|||||||
@@ -62,6 +62,7 @@
|
|||||||
<import>${basedir}/src/main/avro/HoodieCompactionMetadata.avsc</import>
|
<import>${basedir}/src/main/avro/HoodieCompactionMetadata.avsc</import>
|
||||||
<import>${basedir}/src/main/avro/HoodieCleanMetadata.avsc</import>
|
<import>${basedir}/src/main/avro/HoodieCleanMetadata.avsc</import>
|
||||||
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
|
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
|
||||||
|
<import>${basedir}/src/main/avro/HoodieRestoreMetadata.avsc</import>
|
||||||
</imports>
|
</imports>
|
||||||
</configuration>
|
</configuration>
|
||||||
</plugin>
|
</plugin>
|
||||||
|
|||||||
17
hoodie-common/src/main/avro/HoodieRestoreMetadata.avsc
Normal file
17
hoodie-common/src/main/avro/HoodieRestoreMetadata.avsc
Normal file
@@ -0,0 +1,17 @@
|
|||||||
|
{"namespace": "com.uber.hoodie.avro.model",
|
||||||
|
"type": "record",
|
||||||
|
"name": "HoodieRestoreMetadata",
|
||||||
|
"fields": [
|
||||||
|
{"name": "startRestoreTime", "type": "string"},
|
||||||
|
{"name": "timeTakenInMillis", "type": "long"},
|
||||||
|
{"name": "instantsToRollback", "type": {"type": "array", "items": "string"}},
|
||||||
|
{"name": "hoodieRestoreMetadata", "type": {
|
||||||
|
"type" : "map", "values" : {
|
||||||
|
"type": "array",
|
||||||
|
"default": "null",
|
||||||
|
"items": "HoodieRollbackMetadata",
|
||||||
|
"name": "hoodieRollbackMetadata"
|
||||||
|
}
|
||||||
|
}}
|
||||||
|
]
|
||||||
|
}
|
||||||
@@ -48,6 +48,7 @@ public interface HoodieTimeline extends Serializable {
|
|||||||
// (compaction-requested), (compaction-inflight), (completed)
|
// (compaction-requested), (compaction-inflight), (completed)
|
||||||
String COMPACTION_ACTION = "compaction";
|
String COMPACTION_ACTION = "compaction";
|
||||||
String REQUESTED_EXTENSION = ".requested";
|
String REQUESTED_EXTENSION = ".requested";
|
||||||
|
String RESTORE_ACTION = "restore";
|
||||||
|
|
||||||
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
|
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
|
||||||
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
|
String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION;
|
||||||
@@ -66,6 +67,8 @@ public interface HoodieTimeline extends Serializable {
|
|||||||
StringUtils.join(".", REQUESTED_COMPACTION_SUFFIX);
|
StringUtils.join(".", REQUESTED_COMPACTION_SUFFIX);
|
||||||
String INFLIGHT_COMPACTION_EXTENSION =
|
String INFLIGHT_COMPACTION_EXTENSION =
|
||||||
StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION);
|
StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION);
|
||||||
|
String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION;
|
||||||
|
String RESTORE_EXTENSION = "." + RESTORE_ACTION;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Filter this timeline to just include the in-flights
|
* Filter this timeline to just include the in-flights
|
||||||
@@ -248,6 +251,14 @@ public interface HoodieTimeline extends Serializable {
|
|||||||
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION);
|
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static String makeRestoreFileName(String instant) {
|
||||||
|
return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION);
|
||||||
|
}
|
||||||
|
|
||||||
|
static String makeInflightRestoreFileName(String instant) {
|
||||||
|
return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION);
|
||||||
|
}
|
||||||
|
|
||||||
static String makeDeltaFileName(String commitTime) {
|
static String makeDeltaFileName(String commitTime) {
|
||||||
return commitTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
|
return commitTime + HoodieTimeline.DELTA_COMMIT_EXTENSION;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -54,7 +54,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
|||||||
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
|
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
|
||||||
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
||||||
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
|
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
|
||||||
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION}));
|
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION,
|
||||||
|
INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
|
||||||
|
|
||||||
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
|
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||||
private HoodieTableMetaClient metaClient;
|
private HoodieTableMetaClient metaClient;
|
||||||
@@ -186,6 +187,14 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
|||||||
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get only the restore action (inflight and completed) in the active timeline
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getRestoreTimeline() {
|
||||||
|
return new HoodieDefaultTimeline(filterInstantsByAction(RESTORE_ACTION),
|
||||||
|
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
|
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
|
||||||
return instants.stream().filter(s -> s.getAction().equals(action));
|
return instants.stream().filter(s -> s.getAction().equals(action));
|
||||||
|
|||||||
@@ -131,6 +131,9 @@ public class HoodieInstant implements Serializable {
|
|||||||
} else {
|
} else {
|
||||||
return HoodieTimeline.makeCommitFileName(timestamp);
|
return HoodieTimeline.makeCommitFileName(timestamp);
|
||||||
}
|
}
|
||||||
|
} else if (HoodieTimeline.RESTORE_ACTION.equals(action)) {
|
||||||
|
return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp)
|
||||||
|
: HoodieTimeline.makeRestoreFileName(timestamp);
|
||||||
}
|
}
|
||||||
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
|
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,6 +22,7 @@ import com.google.common.collect.Lists;
|
|||||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
|
import com.uber.hoodie.avro.model.HoodieCleanPartitionMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||||
|
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieRollbackPartitionMetadata;
|
import com.uber.hoodie.avro.model.HoodieRollbackPartitionMetadata;
|
||||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||||
@@ -34,6 +35,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
|
|||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
@@ -114,16 +116,28 @@ public class AvroUtils {
|
|||||||
totalDeleted, earliestCommitToRetain, partitionMetadataBuilder.build());
|
totalDeleted, earliestCommitToRetain, partitionMetadataBuilder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTime,
|
||||||
|
Optional<Long> durationInMs, List<String> commits, Map<String, List<HoodieRollbackStat>> commitToStats) {
|
||||||
|
ImmutableMap.Builder<String, List<HoodieRollbackMetadata>> commitToStatBuilder = ImmutableMap.builder();
|
||||||
|
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
|
||||||
|
commitToStatBuilder.put(commitToStat.getKey(), Arrays.asList(convertRollbackMetadata(startRestoreTime,
|
||||||
|
durationInMs, commits, commitToStat.getValue())));
|
||||||
|
}
|
||||||
|
return new HoodieRestoreMetadata(startRestoreTime, durationInMs.orElseGet(() -> -1L), commits,
|
||||||
|
commitToStatBuilder.build());
|
||||||
|
}
|
||||||
|
|
||||||
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
|
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
|
||||||
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> stats) {
|
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> rollbackStats) {
|
||||||
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
|
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
|
||||||
ImmutableMap.builder();
|
ImmutableMap.builder();
|
||||||
int totalDeleted = 0;
|
int totalDeleted = 0;
|
||||||
for (HoodieRollbackStat stat : stats) {
|
for (HoodieRollbackStat stat : rollbackStats) {
|
||||||
HoodieRollbackPartitionMetadata metadata =
|
HoodieRollbackPartitionMetadata metadata =
|
||||||
new HoodieRollbackPartitionMetadata(stat.getPartitionPath(),
|
new HoodieRollbackPartitionMetadata(stat.getPartitionPath(),
|
||||||
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles());
|
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles());
|
||||||
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
|
partitionMetadataBuilder
|
||||||
|
.put(stat.getPartitionPath(), metadata);
|
||||||
totalDeleted += stat.getSuccessDeleteFiles().size();
|
totalDeleted += stat.getSuccessDeleteFiles().size();
|
||||||
}
|
}
|
||||||
return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L),
|
return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L),
|
||||||
@@ -163,6 +177,11 @@ public class AvroUtils {
|
|||||||
return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class);
|
return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Optional<byte[]> serializeRestoreMetadata(
|
||||||
|
HoodieRestoreMetadata restoreMetadata) throws IOException {
|
||||||
|
return serializeAvroMetadata(restoreMetadata, HoodieRestoreMetadata.class);
|
||||||
|
}
|
||||||
|
|
||||||
public static <T extends SpecificRecordBase> Optional<byte[]> serializeAvroMetadata(T metadata,
|
public static <T extends SpecificRecordBase> Optional<byte[]> serializeAvroMetadata(T metadata,
|
||||||
Class<T> clazz) throws IOException {
|
Class<T> clazz) throws IOException {
|
||||||
DatumWriter<T> datumWriter = new SpecificDatumWriter<>(clazz);
|
DatumWriter<T> datumWriter = new SpecificDatumWriter<>(clazz);
|
||||||
|
|||||||
@@ -398,6 +398,20 @@ public class FSUtils {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void deleteOlderRestoreMetaFiles(FileSystem fs, String metaPath,
|
||||||
|
Stream<HoodieInstant> instants) {
|
||||||
|
//TODO - this should be archived when archival is made general for all meta-data
|
||||||
|
// skip MIN_ROLLBACK_TO_KEEP and delete rest
|
||||||
|
instants.skip(MIN_ROLLBACK_TO_KEEP).map(s -> {
|
||||||
|
try {
|
||||||
|
return fs.delete(new Path(metaPath, s.getFileName()), false);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException(
|
||||||
|
"Could not delete restore meta files " + s.getFileName(), e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
public static void createPathIfNotExists(FileSystem fs, Path partitionPath) throws IOException {
|
public static void createPathIfNotExists(FileSystem fs, Path partitionPath) throws IOException {
|
||||||
if (!fs.exists(partitionPath)) {
|
if (!fs.exists(partitionPath)) {
|
||||||
fs.mkdirs(partitionPath);
|
fs.mkdirs(partitionPath);
|
||||||
|
|||||||
Reference in New Issue
Block a user