Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -639,8 +639,8 @@ public boolean rollback(final String commitInstantTime, Option<HoodiePendingRoll
.findFirst());
if (commitInstantOpt.isPresent()) {
LOG.info("Scheduling Rollback at instant time :" + rollbackInstantTime);
Option<HoodieRollbackPlan> rollbackPlanOption = pendingRollbackInfo.map(entry -> Option.of(entry.getRollbackPlan())).orElse(table.scheduleRollback(context, rollbackInstantTime,
commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers()));
Option<HoodieRollbackPlan> rollbackPlanOption = pendingRollbackInfo.map(entry -> Option.of(entry.getRollbackPlan()))
.orElseGet(() -> table.scheduleRollback(context, rollbackInstantTime, commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers()));
if (rollbackPlanOption.isPresent()) {
// execute rollback
HoodieRollbackMetadata rollbackMetadata = table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@

package org.apache.hudi.table.action.rollback;

import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hudi.avro.model.HoodieRollbackRequest;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.engine.HoodieEngineContext;
Expand All @@ -33,11 +36,6 @@
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.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

Expand Down Expand Up @@ -122,19 +120,11 @@ List<Pair<String, HoodieRollbackStat>> maybeDeleteAndCollectStats(HoodieEngineCo
rollbackStats.forEach(entry -> partitionToRollbackStats.add(Pair.of(entry.getPartitionPath(), entry)));
return partitionToRollbackStats.stream();
} else if (!rollbackRequest.getLogBlocksToBeDeleted().isEmpty()) {
Map<String, Long> logFilesToBeDeleted = rollbackRequest.getLogBlocksToBeDeleted();
String fileId = rollbackRequest.getFileId();
String latestBaseInstant = rollbackRequest.getLatestBaseInstant();
FileSystem fs = metaClient.getFs();
// collect all log files that is supposed to be deleted with this rollback
// what happens if file was deleted when invoking fs.getFileStatus(?) below.
// I understand we don't delete log files. but just curious if we need to handle this case.
Map<FileStatus, Long> writtenLogFileSizeMap = new HashMap<>();
for (Map.Entry<String, Long> entry : logFilesToBeDeleted.entrySet()) {
writtenLogFileSizeMap.put(fs.getFileStatus(new Path(entry.getKey())), entry.getValue());
}
HoodieLogFormat.Writer writer = null;
try {
String fileId = rollbackRequest.getFileId();
String latestBaseInstant = rollbackRequest.getLatestBaseInstant();

writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath()))
.withFileId(fileId)
Expand All @@ -156,7 +146,7 @@ List<Pair<String, HoodieRollbackStat>> maybeDeleteAndCollectStats(HoodieEngineCo
writer.close();
}
} catch (IOException io) {
throw new HoodieIOException("Error appending rollback block..", io);
throw new HoodieIOException("Error appending rollback block", io);
}
}

Expand All @@ -167,15 +157,21 @@ List<Pair<String, HoodieRollbackStat>> maybeDeleteAndCollectStats(HoodieEngineCo
metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()),
1L
);
return Collections.singletonList(Pair.of(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withRollbackBlockAppendResults(filesToNumBlocksRollback)
.withWrittenLogFileSizeMap(writtenLogFileSizeMap).build())).stream();

return Collections.singletonList(
Pair.of(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder()
.withPartitionPath(rollbackRequest.getPartitionPath())
.withRollbackBlockAppendResults(filesToNumBlocksRollback)
.build()))
.stream();
} else {
return Collections
.singletonList(Pair.of(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.build())).stream();
return Collections.singletonList(
Pair.of(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder()
.withPartitionPath(rollbackRequest.getPartitionPath())
.build()))
.stream();
}
}, numPartitions);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,26 +19,24 @@

package org.apache.hudi.table.action.rollback;

import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hudi.avro.model.HoodieRollbackRequest;
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.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;

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 java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -104,22 +102,20 @@ private List<HoodieRollbackRequest> getListingBasedRollbackRequests(HoodieEngine
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()));
Map<String, Long> logFilesToBeDeleted = new HashMap<>();
for (Map.Entry<FileStatus, Long> fileToBeDeleted : writtenLogFileSizeMap.entrySet()) {
logFilesToBeDeleted.put(fileToBeDeleted.getKey().getPath().toString(), fileToBeDeleted.getValue());
}
HoodieWriteStat writeStat = rollbackRequest.getWriteStat().get();

Path fullLogFilePath = FSUtils.getPartitionPath(config.getBasePath(), writeStat.getPath());

Map<String, Long> logFilesWithBlocksToRollback =
Collections.singletonMap(fullLogFilePath.toString(), writeStat.getTotalWriteBytes());

return new HoodieRollbackRequest(rollbackRequest.getPartitionPath(), fileId, latestBaseInstant,
Collections.EMPTY_LIST, logFilesToBeDeleted);
Collections.EMPTY_LIST, logFilesWithBlocksToRollback);
}
default:
throw new IllegalStateException("Unknown Rollback action " + rollbackRequest);
}
}, numPartitions).stream().collect(Collectors.toList());
}, numPartitions);
}

private FileStatus[] getBaseFilesToBeDeleted(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.table.action.rollback;

import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.util.Option;

import java.io.Serializable;
Expand Down Expand Up @@ -51,32 +52,42 @@ public enum Type {
*/
private final Option<String> latestBaseInstant;

/**
* TODO
*/
private final Option<HoodieWriteStat> writeStat;

private final Type type;

public ListingBasedRollbackRequest(String partitionPath, Type type) {
this(partitionPath, Option.empty(), Option.empty(), Option.empty(), type);
}

public ListingBasedRollbackRequest(String partitionPath,
Option<String> fileId,
Option<String> latestBaseInstant,
Option<HoodieWriteStat> writeStat,
Type type) {
this.partitionPath = partitionPath;
this.fileId = fileId;
this.latestBaseInstant = latestBaseInstant;
this.writeStat = writeStat;
this.type = type;
}

public static ListingBasedRollbackRequest createRollbackRequestWithDeleteDataFilesOnlyAction(String partitionPath) {
return new ListingBasedRollbackRequest(partitionPath, Option.empty(), Option.empty(),
Type.DELETE_DATA_FILES_ONLY);
return new ListingBasedRollbackRequest(partitionPath, Type.DELETE_DATA_FILES_ONLY);
}

public static ListingBasedRollbackRequest createRollbackRequestWithDeleteDataAndLogFilesAction(String partitionPath) {
return new ListingBasedRollbackRequest(partitionPath, Option.empty(), Option.empty(),
Type.DELETE_DATA_AND_LOG_FILES);
return new ListingBasedRollbackRequest(partitionPath, Type.DELETE_DATA_AND_LOG_FILES);
}

public static ListingBasedRollbackRequest createRollbackRequestWithAppendRollbackBlockAction(String partitionPath, String fileId,
String baseInstant) {
return new ListingBasedRollbackRequest(partitionPath, Option.of(fileId), Option.of(baseInstant),
Type.APPEND_ROLLBACK_BLOCK);
public static ListingBasedRollbackRequest createRollbackRequestWithAppendRollbackBlockAction(String partitionPath,
String fileId,
String baseInstant,
HoodieWriteStat writeStat) {
return new ListingBasedRollbackRequest(partitionPath, Option.of(fileId), Option.of(baseInstant), Option.of(writeStat), Type.APPEND_ROLLBACK_BLOCK);
}

public String getPartitionPath() {
Expand All @@ -91,6 +102,10 @@ public Option<String> getLatestBaseInstant() {
return latestBaseInstant;
}

public Option<HoodieWriteStat> getWriteStat() {
return writeStat;
}

public Type getType() {
return type;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.table.action.rollback;

import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieRollbackRequest;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
Expand All @@ -31,18 +32,13 @@
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.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;

import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import static org.apache.hudi.table.action.rollback.BaseRollbackHelper.EMPTY_STRING;

Expand Down Expand Up @@ -90,42 +86,41 @@ public List<HoodieRollbackRequest> getRollbackRequests(HoodieInstant instantToRo
Collections.singletonList(fullDeletePath.toString()),
Collections.emptyMap());
case APPEND:
// NOTE: This marker file-path does NOT correspond to a log-file, but rather is a phony
// path serving as a "container" for the following components:
// - Base file's file-id
// - Base file's commit instant
// - Partition path
return getRollbackRequestForAppend(WriteMarkers.stripMarkerSuffix(markerFilePath));
default:
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
}
}, parallelism).stream().collect(Collectors.toList());
}, parallelism);
} catch (Exception e) {
throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
}
}

protected HoodieRollbackRequest getRollbackRequestForAppend(String appendBaseFilePath) throws IOException {
Path baseFilePathForAppend = new Path(basePath, appendBaseFilePath);
protected HoodieRollbackRequest getRollbackRequestForAppend(String markerFilePath) throws IOException {
Path baseFilePathForAppend = new Path(basePath, markerFilePath);
String fileId = FSUtils.getFileIdFromFilePath(baseFilePathForAppend);
String baseCommitTime = FSUtils.getCommitTime(baseFilePathForAppend.getName());
String partitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), new Path(basePath, appendBaseFilePath).getParent());
Map<FileStatus, Long> writtenLogFileSizeMap = getWrittenLogFileSizeMap(partitionPath, baseCommitTime, fileId);
Map<String, Long> writtenLogFileStrSizeMap = new HashMap<>();
for (Map.Entry<FileStatus, Long> entry : writtenLogFileSizeMap.entrySet()) {
writtenLogFileStrSizeMap.put(entry.getKey().getPath().toString(), entry.getValue());
}
return new HoodieRollbackRequest(partitionPath, fileId, baseCommitTime, Collections.emptyList(), writtenLogFileStrSizeMap);
String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), baseFilePathForAppend.getParent());
Path partitionPath = FSUtils.getPartitionPath(config.getBasePath(), relativePartitionPath);

// NOTE: Since we're rolling back incomplete Delta Commit, it only could have appended its
// block to the latest log-file
// TODO(HUDI-1517) use provided marker-file's path instead
HoodieLogFile latestLogFile = FSUtils.getLatestLogFile(table.getMetaClient().getFs(), partitionPath, fileId,
HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime).get();

// NOTE: Marker's don't carry information about the cumulative size of the blocks that have been appended,
// therefore we simply stub this value.
Map<String, Long> logFilesWithBlocsToRollback =
Collections.singletonMap(latestLogFile.getFileStatus().getPath().toString(), -1L);

return new HoodieRollbackRequest(relativePartitionPath, fileId, baseCommitTime, Collections.emptyList(),
logFilesWithBlocsToRollback);
}

/**
* Returns written log file size map for the respective baseCommitTime to assist in metadata table syncing.
*
* @param partitionPathStr partition path of interest
* @param baseCommitTime base commit time of interest
* @param fileId fileId of interest
* @return Map<FileStatus, File size>
* @throws IOException
*/
private 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()));
}
}
Loading