1
0

[HUDI-2917] rollback insert data appended to log file when using Hbase Index (#4840)

Co-authored-by: guanziyue <guanziyue@gmail.com>
This commit is contained in:
Sivabalan Narayanan
2022-02-28 08:13:17 -05:00
committed by GitHub
parent 193215201c
commit 4a59876c8b
11 changed files with 340 additions and 61 deletions

View File

@@ -33,9 +33,14 @@ import java.util.Set;
public class WorkloadProfile implements Serializable {
/**
* Computed workload profile.
* Computed workload stats.
*/
protected final HashMap<String, WorkloadStat> partitionPathStatMap;
protected final HashMap<String, WorkloadStat> inputPartitionPathStatMap;
/**
* Execution/Output workload stats
*/
protected final HashMap<String, WorkloadStat> outputPartitionPathStatMap;
/**
* Global workloadStat.
@@ -47,13 +52,21 @@ public class WorkloadProfile implements Serializable {
*/
private WriteOperationType operationType;
private final boolean hasOutputWorkLoadStats;
public WorkloadProfile(Pair<HashMap<String, WorkloadStat>, WorkloadStat> profile) {
this.partitionPathStatMap = profile.getLeft();
this.globalStat = profile.getRight();
this(profile, false);
}
public WorkloadProfile(Pair<HashMap<String, WorkloadStat>, WorkloadStat> profile, WriteOperationType operationType) {
this(profile);
public WorkloadProfile(Pair<HashMap<String, WorkloadStat>, WorkloadStat> profile, boolean hasOutputWorkLoadStats) {
this.inputPartitionPathStatMap = profile.getLeft();
this.globalStat = profile.getRight();
this.outputPartitionPathStatMap = new HashMap<>();
this.hasOutputWorkLoadStats = hasOutputWorkLoadStats;
}
public WorkloadProfile(Pair<HashMap<String, WorkloadStat>, WorkloadStat> profile, WriteOperationType operationType, boolean hasOutputWorkLoadStats) {
this(profile, hasOutputWorkLoadStats);
this.operationType = operationType;
}
@@ -62,15 +75,37 @@ public class WorkloadProfile implements Serializable {
}
public Set<String> getPartitionPaths() {
return partitionPathStatMap.keySet();
return inputPartitionPathStatMap.keySet();
}
public HashMap<String, WorkloadStat> getPartitionPathStatMap() {
return partitionPathStatMap;
public Set<String> getOutputPartitionPaths() {
return hasOutputWorkLoadStats ? outputPartitionPathStatMap.keySet() : inputPartitionPathStatMap.keySet();
}
public HashMap<String, WorkloadStat> getInputPartitionPathStatMap() {
return inputPartitionPathStatMap;
}
public HashMap<String, WorkloadStat> getOutputPartitionPathStatMap() {
return outputPartitionPathStatMap;
}
public boolean hasOutputWorkLoadStats() {
return hasOutputWorkLoadStats;
}
public void updateOutputPartitionPathStatMap(String partitionPath, WorkloadStat workloadStat) {
if (hasOutputWorkLoadStats) {
outputPartitionPathStatMap.put(partitionPath, workloadStat);
}
}
public WorkloadStat getWorkloadStat(String partitionPath) {
return partitionPathStatMap.get(partitionPath);
return inputPartitionPathStatMap.get(partitionPath);
}
public WorkloadStat getOutputWorkloadStat(String partitionPath) {
return hasOutputWorkLoadStats ? outputPartitionPathStatMap.get(partitionPath) : inputPartitionPathStatMap.get(partitionPath);
}
public WriteOperationType getOperationType() {
@@ -81,7 +116,8 @@ public class WorkloadProfile implements Serializable {
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
sb.append("globalStat=").append(globalStat).append(", ");
sb.append("partitionStat=").append(partitionPathStatMap).append(", ");
sb.append("InputPartitionStat=").append(inputPartitionPathStatMap).append(", ");
sb.append("OutputPartitionStat=").append(outputPartitionPathStatMap).append(", ");
sb.append("operationType=").append(operationType);
sb.append('}');
return sb.toString();

View File

@@ -33,9 +33,12 @@ public class WorkloadStat implements Serializable {
private long numUpdates = 0L;
private HashMap<String, Pair<String, Long>> insertLocationToCount;
private HashMap<String, Pair<String, Long>> updateLocationToCount;
public WorkloadStat() {
insertLocationToCount = new HashMap<>();
updateLocationToCount = new HashMap<>();
}
@@ -43,6 +46,17 @@ public class WorkloadStat implements Serializable {
return this.numInserts += numInserts;
}
public long addInserts(HoodieRecordLocation location, long numInserts) {
long accNumInserts = 0;
if (insertLocationToCount.containsKey(location.getFileId())) {
accNumInserts = insertLocationToCount.get(location.getFileId()).getRight();
}
insertLocationToCount.put(
location.getFileId(),
Pair.of(location.getInstantTime(), numInserts + accNumInserts));
return this.numInserts += numInserts;
}
public long addUpdates(HoodieRecordLocation location, long numUpdates) {
long accNumUpdates = 0;
if (updateLocationToCount.containsKey(location.getFileId())) {
@@ -66,6 +80,10 @@ public class WorkloadStat implements Serializable {
return updateLocationToCount;
}
public HashMap<String, Pair<String, Long>> getInsertLocationToCount() {
return insertLocationToCount;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("WorkloadStat {");

View File

@@ -68,6 +68,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I, K, O, R>
extends BaseActionExecutor<T, I, K, O, R> {
@@ -108,22 +109,32 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
throws HoodieCommitException {
try {
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
profile.getPartitionPaths().forEach(path -> {
WorkloadStat partitionStat = profile.getWorkloadStat(path);
profile.getOutputPartitionPaths().forEach(path -> {
WorkloadStat partitionStat = profile.getOutputWorkloadStat(path);
HoodieWriteStat insertStat = new HoodieWriteStat();
insertStat.setNumInserts(partitionStat.getNumInserts());
insertStat.setFileId("");
insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
metadata.addWriteStat(path, insertStat);
partitionStat.getUpdateLocationToCount().forEach((key, value) -> {
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(key);
// TODO : Write baseCommitTime is possible here ?
writeStat.setPrevCommit(value.getKey());
writeStat.setNumUpdateWrites(value.getValue());
metadata.addWriteStat(path, writeStat);
});
Map<String, Pair<String, Long>> updateLocationMap = partitionStat.getUpdateLocationToCount();
Map<String, Pair<String, Long>> insertLocationMap = partitionStat.getInsertLocationToCount();
Stream.concat(updateLocationMap.keySet().stream(), insertLocationMap.keySet().stream())
.distinct()
.forEach(fileId -> {
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);
Pair<String, Long> updateLocation = updateLocationMap.get(fileId);
Pair<String, Long> insertLocation = insertLocationMap.get(fileId);
// TODO : Write baseCommitTime is possible here ?
writeStat.setPrevCommit(updateLocation != null ? updateLocation.getKey() : insertLocation.getKey());
if (updateLocation != null) {
writeStat.setNumUpdateWrites(updateLocation.getValue());
}
if (insertLocation != null) {
writeStat.setNumInserts(insertLocation.getValue());
}
metadata.addWriteStat(path, writeStat);
});
});
metadata.setOperationType(operationType);

View File

@@ -182,14 +182,30 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
.withOperationField(config.allowOperationMetadataField())
.withPartition(operation.getPartitionPath())
.build();
if (!scanner.iterator().hasNext()) {
scanner.close();
return new ArrayList<>();
}
Option<HoodieBaseFile> oldDataFileOpt =
operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
// Considering following scenario: if all log blocks in this fileSlice is rollback, it returns an empty scanner.
// But in this case, we need to give it a base file. Otherwise, it will lose base file in following fileSlice.
if (!scanner.iterator().hasNext()) {
if (!oldDataFileOpt.isPresent()) {
scanner.close();
return new ArrayList<>();
} else {
// TODO: we may directly rename original parquet file if there is not evolution/devolution of schema
/*
TaskContextSupplier taskContextSupplier = hoodieCopyOnWriteTable.getTaskContextSupplier();
String newFileName = FSUtils.makeDataFileName(instantTime,
FSUtils.makeWriteToken(taskContextSupplier.getPartitionIdSupplier().get(), taskContextSupplier.getStageIdSupplier().get(), taskContextSupplier.getAttemptIdSupplier().get()),
operation.getFileId(), hoodieCopyOnWriteTable.getBaseFileExtension());
Path oldFilePath = new Path(oldDataFileOpt.get().getPath());
Path newFilePath = new Path(oldFilePath.getParent(), newFileName);
FileUtil.copy(fs,oldFilePath, fs, newFilePath, false, fs.getConf());
*/
}
}
// Compacting is very similar to applying updates to existing file
Iterator<List<WriteStatus>> result;
// If the dataFile is present, perform updates else perform inserts into a new base file.

View File

@@ -199,7 +199,7 @@ public class RollbackUtils {
partitionRollbackRequests.add(
ListingBasedRollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath));
// append rollback blocks for updates
// append rollback blocks for updates and inserts as A.2 and B.2
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
partitionRollbackRequests
.addAll(generateAppendRollbackBlocksAction(partitionPath, instantToRollback, commitMetadata, table));