liufangqi commented on code in PR #5997:
URL: https://github.com/apache/hudi/pull/5997#discussion_r913468569


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java:
##########
@@ -330,17 +330,23 @@ public static DataStream<Object> 
hoodieStreamWrite(Configuration conf, int defau
           .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
     } else {
       WriteOperatorFactory<HoodieRecord> operatorFactory = 
StreamWriteOperator.getFactory(conf);
-      return dataStream
-          // Key-by record key, to avoid multiple subtasks write to a bucket 
at the same time
-          .keyBy(HoodieRecord::getRecordKey)
-          .transform(
-              "bucket_assigner",
-              TypeInformation.of(HoodieRecord.class),
-              new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
-          .uid("uid_bucket_assigner_" + 
conf.getString(FlinkOptions.TABLE_NAME))
-          
.setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism))
-          // shuffle by fileId(bucket id)
-          .keyBy(record -> record.getCurrentLocation().getFileId())
+
+      DataStream<HoodieRecord> bucketDataStream = dataStream
+              // Key-by record key, to avoid multiple subtasks write to a 
bucket at the same time
+              .keyBy(HoodieRecord::getRecordKey)
+              .transform(
+                      "bucket_assigner",
+                      TypeInformation.of(HoodieRecord.class),
+                      new KeyedProcessOperator<>(new 
BucketAssignFunction<>(conf)))
+              .uid("uid_bucket_assigner_" + 
conf.getString(FlinkOptions.TABLE_NAME))
+              
.setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism));
+
+      bucketDataStream = 
conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism) ==
+              conf.getInteger(FlinkOptions.WRITE_TASKS) ? bucketDataStream : 
bucketDataStream
+              // shuffle by fileId(bucket id)
+              .keyBy(record -> record.getCurrentLocation().getFileId());

Review Comment:
   > Why the data skew can happen ? The fileId is shuffled right ?
   
   @danny0405  As the https://issues.apache.org/jira/browse/HUDI-4338 saying, 
when we get the case that the bucket assinger nums == write operator nums == n. 
Often there is n buckets / file ids need to shuffle, we can not promise that 
every wirte opeartor task can get one bucket / file id.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to