1
0

[HUDI-2433] Refactor rollback actions in hudi-client module (#3664)

This commit is contained in:
Y Ethan Guo
2021-09-15 15:52:43 -07:00
committed by GitHub
parent 86a7351c39
commit 916f12b7dd
34 changed files with 512 additions and 1462 deletions

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.client.SparkTaskContextSupplier;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.EngineProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.function.SerializableBiFunction;
import org.apache.hudi.common.function.SerializableConsumer;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
@@ -73,6 +74,14 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
return javaSparkContext.parallelize(data, parallelism).map(func::apply).collect();
}
@Override
public <I, K, V> List<V> mapToPairAndReduceByKey(List<I> data, SerializablePairFunction<I, K, V> mapToPairFunc, SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
return javaSparkContext.parallelize(data, parallelism).mapToPair(input -> {
Pair<K, V> pair = mapToPairFunc.call(input);
return new Tuple2<>(pair.getLeft(), pair.getRight());
}).reduceByKey(reduceFunc::apply).map(Tuple2::_2).collect();
}
@Override
public <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism) {
return javaSparkContext.parallelize(data, parallelism).flatMap(x -> func.apply(x).iterator()).collect();

View File

@@ -65,7 +65,7 @@ import org.apache.hudi.table.action.commit.SparkMergeHelper;
import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor;
import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor;
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -247,7 +247,7 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
@Override
public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) {
return new SparkCopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
return new CopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
}
@Override

View File

@@ -49,7 +49,8 @@ import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExec
import org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor;
import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
import org.apache.hudi.table.action.restore.SparkMergeOnReadRestoreActionExecutor;
import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor;
import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
import org.apache.spark.api.java.JavaRDD;
import java.util.List;
@@ -146,7 +147,7 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
String rollbackInstantTime,
HoodieInstant commitInstant,
boolean deleteInstants) {
return new SparkMergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
return new MergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
}
@Override

View File

@@ -30,7 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor;
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
import org.apache.spark.api.java.JavaRDD;
@@ -49,7 +49,7 @@ public class SparkCopyOnWriteRestoreActionExecutor<T extends HoodieRecordPayload
@Override
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
table.getMetaClient().reloadActiveTimeline();
SparkCopyOnWriteRollbackActionExecutor rollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(
CopyOnWriteRollbackActionExecutor rollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(
(HoodieSparkEngineContext) context,
config,
table,

View File

@@ -29,7 +29,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor;
import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
import org.apache.spark.api.java.JavaRDD;
@@ -48,7 +48,7 @@ public class SparkMergeOnReadRestoreActionExecutor<T extends HoodieRecordPayload
@Override
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
table.getMetaClient().reloadActiveTimeline();
SparkMergeOnReadRollbackActionExecutor rollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor(
MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor(
context,
config,
table,

View File

@@ -1,252 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.rollback;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.io.Serializable;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
import scala.Tuple2;
/**
* Performs Rollback of Hoodie Tables.
*/
public class ListingBasedRollbackHelper implements Serializable {
private static final Logger LOG = LogManager.getLogger(ListingBasedRollbackHelper.class);
private final HoodieTableMetaClient metaClient;
private final HoodieWriteConfig config;
public ListingBasedRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
this.metaClient = metaClient;
this.config = config;
}
/**
* Performs all rollback actions that we have collected in parallel.
*/
public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, true);
return partitionPathRollbackStatsPairRDD.reduceByKey(RollbackUtils::mergeRollbackStat).map(Tuple2::_2).collect();
}
/**
* Collect all file info that needs to be rollbacked.
*/
public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, false);
return partitionPathRollbackStatsPairRDD.map(Tuple2::_2).collect();
}
/**
* May be delete interested files and collect stats or collect stats only.
*
* @param context instance of {@link HoodieEngineContext} to use.
* @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested.
* @param rollbackRequests List of {@link ListingBasedRollbackRequest} to be operated on.
* @param sparkPartitions number of spark partitions to use for parallelism.
* @param doDelete {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes.
* @return stats collected with or w/o actual deletions.
*/
JavaPairRDD<String, HoodieRollbackStat> maybeDeleteAndCollectStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests,
int sparkPartitions, boolean doDelete) {
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
return jsc.parallelize(rollbackRequests, sparkPartitions).mapToPair(rollbackRequest -> {
switch (rollbackRequest.getType()) {
case DELETE_DATA_FILES_ONLY: {
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(),
rollbackRequest.getPartitionPath(), doDelete);
return new Tuple2<>(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withDeletedFileResults(filesToDeletedStatus).build());
}
case DELETE_DATA_AND_LOG_FILES: {
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete);
return new Tuple2<>(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withDeletedFileResults(filesToDeletedStatus).build());
}
case APPEND_ROLLBACK_BLOCK: {
String fileId = rollbackRequest.getFileId().get();
String latestBaseInstant = rollbackRequest.getLatestBaseInstant().get();
// collect all log files that is supposed to be deleted with this rollback
Map<FileStatus, Long> writtenLogFileSizeMap = FSUtils.getAllLogFiles(metaClient.getFs(),
FSUtils.getPartitionPath(config.getBasePath(), rollbackRequest.getPartitionPath()),
fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), latestBaseInstant)
.collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen()));
Writer writer = null;
try {
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath()))
.withFileId(fileId)
.overBaseCommit(latestBaseInstant)
.withFs(metaClient.getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
if (doDelete) {
Map<HeaderMetadataType, String> header = generateHeader(instantToRollback.getTimestamp());
// if update belongs to an existing log file
writer.appendBlock(new HoodieCommandBlock(header));
}
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException("Failed to rollback for instant " + instantToRollback, io);
} finally {
try {
if (writer != null) {
writer.close();
}
} catch (IOException io) {
throw new HoodieIOException("Error appending rollback block..", io);
}
}
// This step is intentionally done after writer is closed. Guarantees that
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
// cloud-storage : HUDI-168
Map<FileStatus, Long> filesToNumBlocksRollback = Collections.singletonMap(
metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()),
1L
);
return new Tuple2<>(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withRollbackBlockAppendResults(filesToNumBlocksRollback)
.withWrittenLogFileSizeMap(writtenLogFileSizeMap).build());
}
default:
throw new IllegalStateException("Unknown Rollback action " + rollbackRequest);
}
});
}
/**
* Common method used for cleaning out base files under a partition path during rollback of a set of commits.
*/
private Map<FileStatus, Boolean> deleteBaseAndLogFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
String commit, String partitionPath, boolean doDelete) throws IOException {
LOG.info("Cleaning path " + partitionPath);
String basefileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
SerializablePathFilter filter = (path) -> {
if (path.toString().endsWith(basefileExtension)) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commit.equals(fileCommitTime);
} else if (FSUtils.isLogFile(path)) {
// Since the baseCommitTime is the only commit for new log files, it's okay here
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
return commit.equals(fileCommitTime);
}
return false;
};
final Map<FileStatus, Boolean> results = new HashMap<>();
FileSystem fs = metaClient.getFs();
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
for (FileStatus file : toBeDeleted) {
if (doDelete) {
boolean success = fs.delete(file.getPath(), false);
results.put(file, success);
LOG.info("Delete file " + file.getPath() + "\t" + success);
} else {
results.put(file, true);
}
}
return results;
}
/**
* Common method used for cleaning out base files under a partition path during rollback of a set of commits.
*/
private Map<FileStatus, Boolean> deleteBaseFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
String commit, String partitionPath, boolean doDelete) throws IOException {
final Map<FileStatus, Boolean> results = new HashMap<>();
LOG.info("Cleaning path " + partitionPath);
FileSystem fs = metaClient.getFs();
String basefileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
PathFilter filter = (path) -> {
if (path.toString().contains(basefileExtension)) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commit.equals(fileCommitTime);
}
return false;
};
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
for (FileStatus file : toBeDeleted) {
if (doDelete) {
boolean success = fs.delete(file.getPath(), false);
results.put(file, success);
LOG.info("Delete file " + file.getPath() + "\t" + success);
} else {
results.put(file, true);
}
}
return results;
}
private Map<HeaderMetadataType, String> generateHeader(String commit) {
// generate metadata
Map<HeaderMetadataType, String> header = new HashMap<>(3);
header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
return header;
}
public interface SerializablePathFilter extends PathFilter, Serializable {
}
}

View File

@@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.rollback;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import java.util.List;
@SuppressWarnings("checkstyle:LineLength")
public class SparkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayload> extends
BaseCopyOnWriteRollbackActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
String instantTime,
HoodieInstant commitInstant,
boolean deleteInstants) {
super(context, config, table, instantTime, commitInstant, deleteInstants);
}
public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
String instantTime,
HoodieInstant commitInstant,
boolean deleteInstants,
boolean skipTimelinePublish,
boolean useMarkerBasedStrategy) {
super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
}
@Override
protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
if (useMarkerBasedStrategy) {
return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime);
} else {
return this::executeRollbackUsingFileListing;
}
}
@Override
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context,
table.getMetaClient().getBasePath(), config);
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
}
}

View File

@@ -1,93 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.rollback;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.marker.MarkerBasedRollbackUtils;
import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hadoop.fs.FileStatus;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import scala.Tuple2;
@SuppressWarnings("checkstyle:LineLength")
public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> extends AbstractMarkerBasedRollbackStrategy<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
public SparkMarkerBasedRollbackStrategy(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) {
super(table, context, config, instantTime);
}
@Override
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
try {
List<String> markerPaths = MarkerBasedRollbackUtils.getAllMarkerPaths(
table, context, instantToRollback.getTimestamp(), config.getRollbackParallelism());
int parallelism = Math.max(Math.min(markerPaths.size(), config.getRollbackParallelism()), 1);
jsc.setJobGroup(this.getClass().getSimpleName(), "Rolling back using marker files");
return jsc.parallelize(markerPaths, parallelism)
.map(markerFilePath -> {
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
IOType type = IOType.valueOf(typeStr);
switch (type) {
case MERGE:
return undoMerge(WriteMarkers.stripMarkerSuffix(markerFilePath));
case APPEND:
return undoAppend(WriteMarkers.stripMarkerSuffix(markerFilePath), instantToRollback);
case CREATE:
return undoCreate(WriteMarkers.stripMarkerSuffix(markerFilePath));
default:
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
}
})
.mapToPair(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat))
.reduceByKey(RollbackUtils::mergeRollbackStat)
.map(Tuple2::_2).collect();
} catch (Exception e) {
throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
}
}
protected Map<FileStatus, Long> getWrittenLogFileSizeMap(String partitionPathStr, String baseCommitTime, String fileId) throws IOException {
// collect all log files that is supposed to be deleted with this rollback
return FSUtils.getAllLogFiles(table.getMetaClient().getFs(),
FSUtils.getPartitionPath(config.getBasePath(), partitionPathStr), fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime)
.collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen()));
}
}

View File

@@ -1,82 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.rollback;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.List;
@SuppressWarnings("checkstyle:LineLength")
public class SparkMergeOnReadRollbackActionExecutor<T extends HoodieRecordPayload> extends
BaseMergeOnReadRollbackActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
String instantTime,
HoodieInstant commitInstant,
boolean deleteInstants) {
super(context, config, table, instantTime, commitInstant, deleteInstants);
}
public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
String instantTime,
HoodieInstant commitInstant,
boolean deleteInstants,
boolean skipTimelinePublish,
boolean useMarkerBasedStrategy) {
super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
}
@Override
protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
if (useMarkerBasedStrategy) {
return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime);
} else {
return this::executeRollbackUsingFileListing;
}
}
@Override
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant resolvedInstant) {
List<ListingBasedRollbackRequest> rollbackRequests;
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
try {
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(resolvedInstant, table, context);
} catch (IOException e) {
throw new HoodieIOException("Error generating rollback requests by file listing.", e);
}
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, resolvedInstant, rollbackRequests);
}
}