[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:
committed by
GitHub
parent
193215201c
commit
4a59876c8b
@@ -91,27 +91,27 @@ public abstract class BaseJavaCommitActionExecutor<T extends HoodieRecordPayload
|
||||
public HoodieWriteMetadata<List<WriteStatus>> execute(List<HoodieRecord<T>> inputRecords) {
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = new HoodieWriteMetadata<>();
|
||||
|
||||
WorkloadProfile profile = null;
|
||||
WorkloadProfile workloadProfile = null;
|
||||
if (isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(buildProfile(inputRecords));
|
||||
LOG.info("Workload profile :" + profile);
|
||||
try {
|
||||
saveWorkloadProfileMetadataToInflight(profile, instantTime);
|
||||
} catch (Exception e) {
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime);
|
||||
try {
|
||||
if (!metaClient.getFs().exists(new Path(metaClient.getMetaPath(), inflightInstant.getFileName()))) {
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", e);
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Check file exists failed");
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", ex);
|
||||
}
|
||||
}
|
||||
workloadProfile = new WorkloadProfile(buildProfile(inputRecords), table.getIndex().canIndexLogFiles());
|
||||
LOG.info("Input workload profile :" + workloadProfile);
|
||||
}
|
||||
|
||||
final Partitioner partitioner = getPartitioner(profile);
|
||||
final Partitioner partitioner = getPartitioner(workloadProfile);
|
||||
try {
|
||||
saveWorkloadProfileMetadataToInflight(workloadProfile, instantTime);
|
||||
} catch (Exception e) {
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime);
|
||||
try {
|
||||
if (!metaClient.getFs().exists(new Path(metaClient.getMetaPath(), inflightInstant.getFileName()))) {
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", e);
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Check file exists failed");
|
||||
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", ex);
|
||||
}
|
||||
}
|
||||
Map<Integer, List<HoodieRecord<T>>> partitionedRecords = partition(inputRecords, partitioner);
|
||||
|
||||
List<WriteStatus> writeStatuses = new LinkedList<>();
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
@@ -64,9 +65,9 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
*/
|
||||
private int totalBuckets = 0;
|
||||
/**
|
||||
* Stat for the current workload. Helps in determining inserts, upserts etc.
|
||||
* Stat for the input and output workload. Describe the workload before and after being assigned buckets.
|
||||
*/
|
||||
private WorkloadProfile profile;
|
||||
private WorkloadProfile workloadProfile;
|
||||
/**
|
||||
* Helps decide which bucket an incoming update should go to.
|
||||
*/
|
||||
@@ -84,16 +85,16 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
public JavaUpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
|
||||
public JavaUpsertPartitioner(WorkloadProfile workloadProfile, HoodieEngineContext context, HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
updateLocationToBucket = new HashMap<>();
|
||||
partitionPathToInsertBucketInfos = new HashMap<>();
|
||||
bucketInfoMap = new HashMap<>();
|
||||
this.profile = profile;
|
||||
this.workloadProfile = workloadProfile;
|
||||
this.table = table;
|
||||
this.config = config;
|
||||
assignUpdates(profile);
|
||||
assignInserts(profile, context);
|
||||
assignUpdates(workloadProfile);
|
||||
assignInserts(workloadProfile, context);
|
||||
|
||||
LOG.info("Total Buckets :" + totalBuckets + ", buckets info => " + bucketInfoMap + ", \n"
|
||||
+ "Partition to insert buckets => " + partitionPathToInsertBucketInfos + ", \n"
|
||||
@@ -102,11 +103,19 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
|
||||
private void assignUpdates(WorkloadProfile profile) {
|
||||
// each update location gets a partition
|
||||
Set<Map.Entry<String, WorkloadStat>> partitionStatEntries = profile.getPartitionPathStatMap().entrySet();
|
||||
Set<Map.Entry<String, WorkloadStat>> partitionStatEntries = profile.getInputPartitionPathStatMap().entrySet();
|
||||
for (Map.Entry<String, WorkloadStat> partitionStat : partitionStatEntries) {
|
||||
WorkloadStat outputWorkloadStats = profile.getOutputPartitionPathStatMap().getOrDefault(partitionStat.getKey(), new WorkloadStat());
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry :
|
||||
partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
|
||||
addUpdateBucket(partitionStat.getKey(), updateLocEntry.getKey());
|
||||
if (profile.hasOutputWorkLoadStats()) {
|
||||
HoodieRecordLocation hoodieRecordLocation = new HoodieRecordLocation(updateLocEntry.getValue().getKey(), updateLocEntry.getKey());
|
||||
outputWorkloadStats.addUpdates(hoodieRecordLocation, updateLocEntry.getValue().getValue());
|
||||
}
|
||||
}
|
||||
if (profile.hasOutputWorkLoadStats()) {
|
||||
profile.updateOutputPartitionPathStatMap(partitionStat.getKey(), outputWorkloadStats);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -133,6 +142,7 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
|
||||
for (String partitionPath : partitionPaths) {
|
||||
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
|
||||
WorkloadStat outputWorkloadStats = profile.getOutputPartitionPathStatMap().getOrDefault(partitionPath, new WorkloadStat());
|
||||
if (pStat.getNumInserts() > 0) {
|
||||
|
||||
List<SmallFile> smallFiles = partitionSmallFilesMap.getOrDefault(partitionPath, new ArrayList<>());
|
||||
@@ -158,6 +168,9 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
bucket = addUpdateBucket(partitionPath, smallFile.location.getFileId());
|
||||
LOG.info("Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
|
||||
}
|
||||
if (profile.hasOutputWorkLoadStats()) {
|
||||
outputWorkloadStats.addInserts(smallFile.location, recordsToAppend);
|
||||
}
|
||||
bucketNumbers.add(bucket);
|
||||
recordsPerBucket.add(recordsToAppend);
|
||||
totalUnassignedInserts -= recordsToAppend;
|
||||
@@ -183,6 +196,9 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
}
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.INSERT, FSUtils.createNewFileIdPfx(), partitionPath);
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
if (profile.hasOutputWorkLoadStats()) {
|
||||
outputWorkloadStats.addInserts(new HoodieRecordLocation(HoodieWriteStat.NULL_COMMIT, bucketInfo.getFileIdPrefix()), recordsPerBucket.get(recordsPerBucket.size() - 1));
|
||||
}
|
||||
totalBuckets++;
|
||||
}
|
||||
}
|
||||
@@ -200,6 +216,9 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
LOG.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
|
||||
partitionPathToInsertBucketInfos.put(partitionPath, insertBuckets);
|
||||
}
|
||||
if (profile.hasOutputWorkLoadStats()) {
|
||||
profile.updateOutputPartitionPathStatMap(partitionPath, outputWorkloadStats);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -271,7 +290,7 @@ public class JavaUpsertPartitioner<T extends HoodieRecordPayload<T>> implements
|
||||
String partitionPath = keyLocation.getLeft().getPartitionPath();
|
||||
List<InsertBucketCumulativeWeightPair> targetBuckets = partitionPathToInsertBucketInfos.get(partitionPath);
|
||||
// pick the target bucket to use based on the weights.
|
||||
final long totalInserts = Math.max(1, profile.getWorkloadStat(partitionPath).getNumInserts());
|
||||
final long totalInserts = Math.max(1, workloadProfile.getWorkloadStat(partitionPath).getNumInserts());
|
||||
final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", keyLocation.getLeft().getRecordKey());
|
||||
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
|
||||
|
||||
|
||||
Reference in New Issue
Block a user