1
0

[HUDI-1502] MOR rollback and restore support for metadata sync (#2421)

- Adds field to RollbackMetadata that capture the logs written for rollback blocks
- Adds field to RollbackMetadata that capture new logs files written by unsynced deltacommits

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Sivabalan Narayanan
2021-01-11 16:23:13 -05:00
committed by GitHub
parent de42adc230
commit e3d3677b7e
15 changed files with 270 additions and 118 deletions

View File

@@ -31,18 +31,24 @@
{"name": "partitionPath", "type": "string"},
{"name": "successDeleteFiles", "type": {"type": "array", "items": "string"}},
{"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}},
{"name": "appendFiles", "type": {
{"name": "rollbackLogFiles", "type": {
"type": "map",
"doc": "Files to which append blocks were written",
"doc": "Files to which append blocks were written to capture rollback commit",
"values": {
"type": "long",
"doc": "Size of this file in bytes"
}
}},
{"name": "writtenLogFiles", "type": {
"type": "map",
"doc": "Log files written that were expected to be rolledback",
"values": {
"type": "long",
"doc": "Size of this file in bytes"
}
}}
]
}
}
},
}}},
{
"name":"version",
"type":["int", "null"],

View File

@@ -38,13 +38,16 @@ public class HoodieRollbackStat implements Serializable {
private final List<String> failedDeleteFiles;
// Count of HoodieLogFile to commandBlocks written for a particular rollback
private final Map<FileStatus, Long> commandBlocksCount;
// all log files with same base instant as instant to be rolledback
private final Map<FileStatus, Long> writtenLogFileSizeMap;
public HoodieRollbackStat(String partitionPath, List<String> successDeleteFiles, List<String> failedDeleteFiles,
Map<FileStatus, Long> commandBlocksCount) {
Map<FileStatus, Long> commandBlocksCount, Map<FileStatus, Long> writtenLogFileSizeMap) {
this.partitionPath = partitionPath;
this.successDeleteFiles = successDeleteFiles;
this.failedDeleteFiles = failedDeleteFiles;
this.commandBlocksCount = commandBlocksCount;
this.writtenLogFileSizeMap = writtenLogFileSizeMap;
}
public Map<FileStatus, Long> getCommandBlocksCount() {
@@ -63,6 +66,10 @@ public class HoodieRollbackStat implements Serializable {
return failedDeleteFiles;
}
public Map<FileStatus, Long> getWrittenLogFileSizeMap() {
return writtenLogFileSizeMap;
}
public static HoodieRollbackStat.Builder newBuilder() {
return new Builder();
}
@@ -75,6 +82,7 @@ public class HoodieRollbackStat implements Serializable {
private List<String> successDeleteFiles;
private List<String> failedDeleteFiles;
private Map<FileStatus, Long> commandBlocksCount;
private Map<FileStatus, Long> writtenLogFileSizeMap;
private String partitionPath;
public Builder withDeletedFileResults(Map<FileStatus, Boolean> deletedFiles) {
@@ -100,6 +108,11 @@ public class HoodieRollbackStat implements Serializable {
return this;
}
public Builder withWrittenLogFileSizeMap(Map<FileStatus, Long> writtenLogFileSizeMap) {
this.writtenLogFileSizeMap = writtenLogFileSizeMap;
return this;
}
public Builder withPartitionPath(String partitionPath) {
this.partitionPath = partitionPath;
return this;
@@ -115,7 +128,10 @@ public class HoodieRollbackStat implements Serializable {
if (commandBlocksCount == null) {
commandBlocksCount = Collections.EMPTY_MAP;
}
return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles, commandBlocksCount);
if (writtenLogFileSizeMap == null) {
writtenLogFileSizeMap = Collections.EMPTY_MAP;
}
return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles, commandBlocksCount, writtenLogFileSizeMap);
}
}
}

View File

@@ -45,6 +45,7 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -426,10 +427,14 @@ public class FSUtils {
*/
public static Stream<HoodieLogFile> getAllLogFiles(FileSystem fs, Path partitionPath, final String fileId,
final String logFileExtension, final String baseCommitTime) throws IOException {
return Arrays
.stream(fs.listStatus(partitionPath,
path -> path.getName().startsWith("." + fileId) && path.getName().contains(logFileExtension)))
.map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime));
try {
return Arrays
.stream(fs.listStatus(partitionPath,
path -> path.getName().startsWith("." + fileId) && path.getName().contains(logFileExtension)))
.map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime));
} catch (FileNotFoundException e) {
return Stream.<HoodieLogFile>builder().build();
}
}
/**

View File

@@ -18,17 +18,6 @@
package org.apache.hudi.common.table.timeline;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.file.FileReader;
import org.apache.avro.file.SeekableByteArrayInput;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
@@ -44,6 +33,17 @@ import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.file.FileReader;
import org.apache.avro.file.SeekableByteArrayInput;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.fs.FileStatus;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Collections;
@@ -71,10 +71,12 @@ public class TimelineMetadataUtils {
Map<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder = new HashMap<>();
int totalDeleted = 0;
for (HoodieRollbackStat stat : rollbackStats) {
Map<String, Long> appendFiles = stat.getCommandBlocksCount().keySet().stream()
Map<String, Long> rollbackLogFiles = stat.getCommandBlocksCount().keySet().stream()
.collect(Collectors.toMap(f -> f.getPath().toString(), FileStatus::getLen));
Map<String, Long> probableLogFiles = stat.getWrittenLogFileSizeMap().keySet().stream()
.collect(Collectors.toMap(f -> f.getPath().toString(), FileStatus::getLen));
HoodieRollbackPartitionMetadata metadata = new HoodieRollbackPartitionMetadata(stat.getPartitionPath(),
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles(), appendFiles);
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles(), rollbackLogFiles, probableLogFiles);
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
totalDeleted += stat.getSuccessDeleteFiles().size();
}

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.metadata;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
@@ -34,6 +33,8 @@ import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -44,6 +45,7 @@ import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
@@ -246,12 +248,13 @@ public class HoodieTableMetadataUtil {
rollbackMetadata.getPartitionMetadata().values().forEach(pm -> {
// Has this rollback produced new files?
boolean hasAppendFiles = pm.getAppendFiles().values().stream().mapToLong(Long::longValue).sum() > 0;
boolean hasRollbackLogFiles = pm.getRollbackLogFiles() != null && !pm.getRollbackLogFiles().isEmpty();
boolean hasNonZeroRollbackLogFiles = hasRollbackLogFiles && pm.getRollbackLogFiles().values().stream().mapToLong(Long::longValue).sum() > 0;
// If commit being rolled back has not been synced to metadata table yet then there is no need to update metadata
boolean shouldSkip = lastSyncTs.isPresent()
&& HoodieTimeline.compareTimestamps(rollbackMetadata.getCommitsRollback().get(0), HoodieTimeline.GREATER_THAN, lastSyncTs.get());
if (!hasAppendFiles && shouldSkip) {
if (!hasNonZeroRollbackLogFiles && shouldSkip) {
LOG.info(String.format("Skipping syncing of rollbackMetadata at %s, given metadata table is already synced upto to %s",
rollbackMetadata.getCommitsRollback().get(0), lastSyncTs.get()));
return;
@@ -269,16 +272,31 @@ public class HoodieTableMetadataUtil {
partitionToDeletedFiles.get(partition).addAll(deletedFiles);
}
if (!pm.getAppendFiles().isEmpty()) {
BiFunction<Long, Long, Long> fileMergeFn = (oldSize, newSizeCopy) -> {
// if a file exists in both written log files and rollback log files, we want to pick the one that is higher
// as rollback file could have been updated after written log files are computed.
return oldSize > newSizeCopy ? oldSize : newSizeCopy;
};
if (hasRollbackLogFiles) {
if (!partitionToAppendedFiles.containsKey(partition)) {
partitionToAppendedFiles.put(partition, new HashMap<>());
}
// Extract appended file name from the absolute paths saved in getAppendFiles()
pm.getAppendFiles().forEach((path, size) -> {
partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, (oldSize, newSizeCopy) -> {
return size + oldSize;
});
pm.getRollbackLogFiles().forEach((path, size) -> {
partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, fileMergeFn);
});
}
if (pm.getWrittenLogFiles() != null && !pm.getWrittenLogFiles().isEmpty()) {
if (!partitionToAppendedFiles.containsKey(partition)) {
partitionToAppendedFiles.put(partition, new HashMap<>());
}
// Extract appended file name from the absolute paths saved in getWrittenLogFiles()
pm.getWrittenLogFiles().forEach((path, size) -> {
partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, fileMergeFn);
});
}
});