[HUDI-3559] Flink bucket index with COW table throws NoSuchElementException
Actually method FlinkWriteHelper#deduplicateRecords does not guarantee the records sequence, but there is a implicit constraint: all the records in one bucket should have the same bucket type(instant time here), the BucketStreamWriteFunction breaks the rule and fails to comply with this constraint. close apache/hudi#5018
This commit is contained in:
@@ -65,7 +65,17 @@ public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
|
||||
private String indexKeyFields;
|
||||
|
||||
private final HashMap<String, String> bucketToFileIDMap;
|
||||
/**
|
||||
* BucketID to file group mapping.
|
||||
*/
|
||||
private HashMap<String, String> bucketIndex;
|
||||
|
||||
/**
|
||||
* Incremental bucket index of the current checkpoint interval,
|
||||
* it is needed because the bucket type('I' or 'U') should be decided based on the committed files view,
|
||||
* all the records in one bucket should have the same bucket type.
|
||||
*/
|
||||
private HashMap<String, String> incBucketIndex;
|
||||
|
||||
/**
|
||||
* Constructs a BucketStreamWriteFunction.
|
||||
@@ -74,7 +84,6 @@ public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
*/
|
||||
public BucketStreamWriteFunction(Configuration config) {
|
||||
super(config);
|
||||
this.bucketToFileIDMap = new HashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -85,6 +94,8 @@ public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks();
|
||||
this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks();
|
||||
this.bucketIndex = new HashMap<>();
|
||||
this.incBucketIndex = new HashMap<>();
|
||||
bootstrapIndex();
|
||||
}
|
||||
|
||||
@@ -94,6 +105,13 @@ public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
this.table = this.writeClient.getHoodieTable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState() {
|
||||
super.snapshotState();
|
||||
this.bucketIndex.putAll(this.incBucketIndex);
|
||||
this.incBucketIndex.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(I i, ProcessFunction<I, Object>.Context context, Collector<Object> collector) throws Exception {
|
||||
HoodieRecord<?> record = (HoodieRecord<?>) i;
|
||||
@@ -103,12 +121,12 @@ public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, this.bucketNum);
|
||||
final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum);
|
||||
|
||||
if (bucketToFileIDMap.containsKey(partitionBucketId)) {
|
||||
location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId));
|
||||
if (bucketIndex.containsKey(partitionBucketId)) {
|
||||
location = new HoodieRecordLocation("U", bucketIndex.get(partitionBucketId));
|
||||
} else {
|
||||
String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum);
|
||||
location = new HoodieRecordLocation("I", newFileId);
|
||||
bucketToFileIDMap.put(partitionBucketId, newFileId);
|
||||
incBucketIndex.put(partitionBucketId, newFileId);
|
||||
}
|
||||
record.unseal();
|
||||
record.setCurrentLocation(location);
|
||||
@@ -154,12 +172,12 @@ public class BucketStreamWriteFunction<I> extends StreamWriteFunction<I> {
|
||||
if (bucketToLoad.contains(bucketNumber)) {
|
||||
String partitionBucketId = BucketIdentifier.partitionBucketIdStr(partitionPath, bucketNumber);
|
||||
LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID));
|
||||
if (bucketToFileIDMap.containsKey(partitionBucketId)) {
|
||||
if (bucketIndex.containsKey(partitionBucketId)) {
|
||||
throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found "
|
||||
+ "during the BucketStreamWriteFunction index bootstrap.", fileID, partitionBucketId));
|
||||
} else {
|
||||
LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId));
|
||||
bucketToFileIDMap.put(partitionBucketId, fileID);
|
||||
bucketIndex.put(partitionBucketId, fileID);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user