[HUDI-2245] BucketAssigner generates the fileId evenly to avoid data skew (#3362)
This commit is contained in:
@@ -123,6 +123,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
this.bucketAssigner = BucketAssigners.create(
|
||||
getRuntimeContext().getIndexOfThisSubtask(),
|
||||
getRuntimeContext().getMaxNumberOfParallelSubtasks(),
|
||||
getRuntimeContext().getNumberOfParallelSubtasks(),
|
||||
WriteOperationType.isOverwrite(WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION))),
|
||||
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE)),
|
||||
|
||||
@@ -28,14 +28,16 @@ import org.apache.hudi.table.action.commit.BucketType;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
|
||||
import org.apache.flink.util.Preconditions;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Bucket assigner that assigns the data buffer of one checkpoint into buckets.
|
||||
@@ -57,6 +59,11 @@ public class BucketAssigner implements AutoCloseable {
|
||||
*/
|
||||
private final int taskID;
|
||||
|
||||
/**
|
||||
* The max parallelism.
|
||||
*/
|
||||
private final int maxParallelism;
|
||||
|
||||
/**
|
||||
* Number of tasks.
|
||||
*/
|
||||
@@ -89,10 +96,12 @@ public class BucketAssigner implements AutoCloseable {
|
||||
|
||||
public BucketAssigner(
|
||||
int taskID,
|
||||
int maxParallelism,
|
||||
int numTasks,
|
||||
WriteProfile profile,
|
||||
HoodieWriteConfig config) {
|
||||
this.taskID = taskID;
|
||||
this.maxParallelism = maxParallelism;
|
||||
this.numTasks = numTasks;
|
||||
this.config = config;
|
||||
this.writeProfile = profile;
|
||||
@@ -148,7 +157,7 @@ public class BucketAssigner implements AutoCloseable {
|
||||
return bucketInfoMap.get(key);
|
||||
}
|
||||
}
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.INSERT, FSUtils.createNewFileIdPfx(), partitionPath);
|
||||
BucketInfo bucketInfo = new BucketInfo(BucketType.INSERT, createFileIdOfThisTask(), partitionPath);
|
||||
final String key = StreamerUtil.generateBucketKey(partitionPath, bucketInfo.getFileIdPrefix());
|
||||
bucketInfoMap.put(key, bucketInfo);
|
||||
NewFileAssignState newFileAssignState = new NewFileAssignState(bucketInfo.getFileIdPrefix(), writeProfile.getRecordsPerBucket());
|
||||
@@ -186,13 +195,26 @@ public class BucketAssigner implements AutoCloseable {
|
||||
return this.writeProfile.getTable();
|
||||
}
|
||||
|
||||
private List<SmallFile> smallFilesOfThisTask(List<SmallFile> smallFiles) {
|
||||
// computes the small files to write inserts for this task.
|
||||
List<SmallFile> smallFilesOfThisTask = new ArrayList<>();
|
||||
for (int i = taskID; i < smallFiles.size(); i += numTasks) {
|
||||
smallFilesOfThisTask.add(smallFiles.get(i));
|
||||
private boolean fileIdOfThisTask(String fileId) {
|
||||
// the file id can shuffle to this task
|
||||
return KeyGroupRangeAssignment.assignKeyToParallelOperator(fileId, maxParallelism, numTasks) == taskID;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String createFileIdOfThisTask() {
|
||||
String newFileIdPfx = FSUtils.createNewFileIdPfx();
|
||||
while (!fileIdOfThisTask(newFileIdPfx)) {
|
||||
newFileIdPfx = FSUtils.createNewFileIdPfx();
|
||||
}
|
||||
return smallFilesOfThisTask;
|
||||
return newFileIdPfx;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public List<SmallFile> smallFilesOfThisTask(List<SmallFile> smallFiles) {
|
||||
// computes the small files to write inserts for this task.
|
||||
return smallFiles.stream()
|
||||
.filter(smallFile -> fileIdOfThisTask(smallFile.location.getFileId()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public void close() {
|
||||
|
||||
@@ -29,21 +29,24 @@ import org.apache.hudi.sink.partitioner.profile.WriteProfiles;
|
||||
*/
|
||||
public abstract class BucketAssigners {
|
||||
|
||||
private BucketAssigners() {}
|
||||
private BucketAssigners() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a {@code BucketAssigner}.
|
||||
*
|
||||
* @param taskID The task ID
|
||||
* @param numTasks The number of tasks
|
||||
* @param overwrite Whether the write operation is OVERWRITE
|
||||
* @param tableType The table type
|
||||
* @param context The engine context
|
||||
* @param config The configuration
|
||||
* @param taskID The task ID
|
||||
* @param maxParallelism The max parallelism
|
||||
* @param numTasks The number of tasks
|
||||
* @param overwrite Whether the write operation is OVERWRITE
|
||||
* @param tableType The table type
|
||||
* @param context The engine context
|
||||
* @param config The configuration
|
||||
* @return the bucket assigner instance
|
||||
*/
|
||||
public static BucketAssigner create(
|
||||
int taskID,
|
||||
int maxParallelism,
|
||||
int numTasks,
|
||||
boolean overwrite,
|
||||
HoodieTableType tableType,
|
||||
@@ -51,6 +54,6 @@ public abstract class BucketAssigners {
|
||||
HoodieWriteConfig config) {
|
||||
boolean delta = tableType.equals(HoodieTableType.MERGE_ON_READ);
|
||||
WriteProfile writeProfile = WriteProfiles.singleton(overwrite, delta, config, context);
|
||||
return new BucketAssigner(taskID, numTasks, writeProfile, config);
|
||||
return new BucketAssigner(taskID, maxParallelism, numTasks, writeProfile, config);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user