@@ -92,12 +92,13 @@ public class Pipelines {
9292 *
9393 * <p>The bulk insert should be run in batch execution mode.
9494 *
95- * @param conf The configuration
96- * @param rowType The input row type
97- * @param dataStream The input data stream
95+ * @param conf The configuration
96+ * @param defaultParallelism The default parallelism
97+ * @param rowType The input row type
98+ * @param dataStream The input data stream
9899 * @return the bulk insert data stream sink
99100 */
100- public static DataStreamSink <Object > bulkInsert (Configuration conf , RowType rowType , DataStream <RowData > dataStream ) {
101+ public static DataStreamSink <Object > bulkInsert (Configuration conf , int defaultParallelism , RowType rowType , DataStream <RowData > dataStream ) {
101102 WriteOperatorFactory <RowData > operatorFactory = BulkInsertWriteOperator .getFactory (conf , rowType );
102103 if (OptionsResolver .isBucketIndexType (conf )) {
103104 String indexKeys = conf .getString (FlinkOptions .INDEX_KEY_FIELD );
@@ -111,18 +112,18 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
111112 Map <String , String > bucketIdToFileId = new HashMap <>();
112113 dataStream = dataStream .partitionCustom (partitioner , keyGen ::getHoodieKey )
113114 .map (record -> BucketBulkInsertWriterHelper .rowWithFileId (bucketIdToFileId , keyGen , record , indexKeys , numBuckets ), typeInfo )
114- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS )); // same parallelism as write task to avoid shuffle
115+ .setParallelism (getWriteTasks ( conf , defaultParallelism )); // same parallelism as write task to avoid shuffle
115116 if (conf .getBoolean (FlinkOptions .WRITE_BULK_INSERT_SORT_INPUT )) {
116117 SortOperatorGen sortOperatorGen = BucketBulkInsertWriterHelper .getFileIdSorterGen (rowTypeWithFileId );
117118 dataStream = dataStream .transform ("file_sorter" , typeInfo , sortOperatorGen .createSortOperator ())
118- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS )); // same parallelism as write task to avoid shuffle
119+ .setParallelism (getWriteTasks ( conf , defaultParallelism )); // same parallelism as write task to avoid shuffle
119120 ExecNodeUtil .setManagedMemoryWeight (dataStream .getTransformation (),
120121 conf .getInteger (FlinkOptions .WRITE_SORT_MEMORY ) * 1024L * 1024L );
121122 }
122123 return dataStream
123124 .transform (opIdentifier ("bucket_bulk_insert" , conf ), TypeInformation .of (Object .class ), operatorFactory )
124125 .uid ("uid_bucket_bulk_insert" + conf .getString (FlinkOptions .TABLE_NAME ))
125- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS ))
126+ .setParallelism (getWriteTasks ( conf , defaultParallelism ))
126127 .addSink (DummySink .INSTANCE )
127128 .name ("dummy" );
128129 }
@@ -136,7 +137,7 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
136137 // use #partitionCustom instead of #keyBy to avoid duplicate sort operations,
137138 // see BatchExecutionUtils#applyBatchExecutionSettings for details.
138139 Partitioner <String > partitioner = (key , channels ) -> KeyGroupRangeAssignment .assignKeyToParallelOperator (key ,
139- KeyGroupRangeAssignment .computeDefaultMaxParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS )), channels );
140+ KeyGroupRangeAssignment .computeDefaultMaxParallelism (getWriteTasks ( conf , defaultParallelism )), channels );
140141 dataStream = dataStream .partitionCustom (partitioner , rowDataKeyGen ::getPartitionPath );
141142 }
142143 if (conf .getBoolean (FlinkOptions .WRITE_BULK_INSERT_SORT_INPUT )) {
@@ -146,7 +147,7 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
146147 .transform ("partition_key_sorter" ,
147148 TypeInformation .of (RowData .class ),
148149 sortOperatorGen .createSortOperator ())
149- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS ));
150+ .setParallelism (getWriteTasks ( conf , defaultParallelism ));
150151 ExecNodeUtil .setManagedMemoryWeight (dataStream .getTransformation (),
151152 conf .getInteger (FlinkOptions .WRITE_SORT_MEMORY ) * 1024L * 1024L );
152153 }
@@ -156,7 +157,7 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
156157 TypeInformation .of (Object .class ),
157158 operatorFactory )
158159 // follow the parallelism of upstream operators to avoid shuffle
159- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS ))
160+ .setParallelism (getWriteTasks ( conf , defaultParallelism ))
160161 .addSink (DummySink .INSTANCE )
161162 .name ("dummy" );
162163 }
@@ -177,14 +178,16 @@ public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowT
177178 * <p>The write task switches to new file handle each time it receives a record
178179 * from the different partition path, so there may be many small files.
179180 *
180- * @param conf The configuration
181- * @param rowType The input row type
182- * @param dataStream The input data stream
183- * @param bounded Whether the input stream is bounded
181+ * @param conf The configuration
182+ * @param defaultParallelism The default parallelism
183+ * @param rowType The input row type
184+ * @param dataStream The input data stream
185+ * @param bounded Whether the input stream is bounded
184186 * @return the appending data stream sink
185187 */
186188 public static DataStream <Object > append (
187189 Configuration conf ,
190+ int defaultParallelism ,
188191 RowType rowType ,
189192 DataStream <RowData > dataStream ,
190193 boolean bounded ) {
@@ -198,7 +201,7 @@ public static DataStream<Object> append(
198201 return dataStream
199202 .transform (opIdentifier ("hoodie_append_write" , conf ), TypeInformation .of (Object .class ), operatorFactory )
200203 .uid ("uid_hoodie_stream_write" + conf .getString (FlinkOptions .TABLE_NAME ))
201- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS ));
204+ .setParallelism (getWriteTasks ( conf , defaultParallelism ));
202205 }
203206
204207 /**
@@ -319,7 +322,7 @@ public static DataStream<HoodieRecord> rowDataToHoodieRecord(Configuration conf,
319322 * @param dataStream The input data stream
320323 * @return the stream write data stream pipeline
321324 */
322- public static DataStream <Object > hoodieStreamWrite (Configuration conf , int defaultParallelism , DataStream <HoodieRecord > dataStream ) {
325+ public static DataStream <Object > hoodieStreamWrite (Configuration conf , int defaultParallelism ,DataStream <HoodieRecord > dataStream ) {
323326 if (OptionsResolver .isBucketIndexType (conf )) {
324327 WriteOperatorFactory <HoodieRecord > operatorFactory = BucketStreamWriteOperator .getFactory (conf );
325328 int bucketNum = conf .getInteger (FlinkOptions .BUCKET_INDEX_NUM_BUCKETS );
@@ -328,7 +331,7 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defau
328331 return dataStream .partitionCustom (partitioner , HoodieRecord ::getKey )
329332 .transform (opIdentifier ("bucket_write" , conf ), TypeInformation .of (Object .class ), operatorFactory )
330333 .uid ("uid_bucket_write" + conf .getString (FlinkOptions .TABLE_NAME ))
331- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS ));
334+ .setParallelism (getWriteTasks ( conf , defaultParallelism ));
332335 } else {
333336 WriteOperatorFactory <HoodieRecord > operatorFactory = StreamWriteOperator .getFactory (conf );
334337 return dataStream
@@ -344,7 +347,7 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defau
344347 .keyBy (record -> record .getCurrentLocation ().getFileId ())
345348 .transform (opIdentifier ("stream_write" , conf ), TypeInformation .of (Object .class ), operatorFactory )
346349 .uid ("uid_stream_write" + conf .getString (FlinkOptions .TABLE_NAME ))
347- .setParallelism (conf . getInteger ( FlinkOptions . WRITE_TASKS ));
350+ .setParallelism (getWriteTasks ( conf , defaultParallelism ));
348351 }
349352 }
350353
@@ -364,11 +367,13 @@ public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defau
364367 * Note: both the compaction plan generation task and commission task are singleton.
365368 * </pre>
366369 *
367- * @param conf The configuration
368- * @param dataStream The input data stream
370+ * @param conf The configuration
371+ * @param defaultParallelism The default parallelism
372+ * @param dataStream The input data stream
369373 * @return the compaction pipeline
370374 */
371- public static DataStreamSink <CompactionCommitEvent > compact (Configuration conf , DataStream <Object > dataStream ) {
375+ public static DataStreamSink <CompactionCommitEvent > compact (Configuration conf , int defaultParallelism , DataStream <Object > dataStream ) {
376+ int compactionTasks = getWriteTasks (conf , defaultParallelism ) * conf .getInteger (FlinkOptions .COMPACTION_DELTA_COMMITS ) / 5 ;
372377 return dataStream .transform ("compact_plan_generate" ,
373378 TypeInformation .of (CompactionPlanEvent .class ),
374379 new CompactionPlanOperator (conf ))
@@ -379,7 +384,8 @@ public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf,
379384 .transform ("compact_task" ,
380385 TypeInformation .of (CompactionCommitEvent .class ),
381386 new ProcessOperator <>(new CompactFunction (conf )))
382- .setParallelism (conf .getInteger (FlinkOptions .COMPACTION_TASKS ))
387+ .setParallelism (conf .getOptional (FlinkOptions .COMPACTION_TASKS )
388+ .orElse (Math .max (compactionTasks , 1 )))
383389 .addSink (new CompactionCommitSink (conf ))
384390 .name ("compact_commit" )
385391 .setParallelism (1 ); // compaction commit should be singleton
@@ -401,12 +407,13 @@ public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf,
401407 * Note: both the clustering plan generation task and commission task are singleton.
402408 * </pre>
403409 *
404- * @param conf The configuration
405- * @param rowType The input row type
406- * @param dataStream The input data stream
410+ * @param conf The configuration
411+ * @param defaultParallelism The default parallelism
412+ * @param rowType The input row type
413+ * @param dataStream The input data stream
407414 * @return the clustering pipeline
408415 */
409- public static DataStreamSink <ClusteringCommitEvent > cluster (Configuration conf , RowType rowType , DataStream <Object > dataStream ) {
416+ public static DataStreamSink <ClusteringCommitEvent > cluster (Configuration conf , int defaultParallelism , RowType rowType , DataStream <Object > dataStream ) {
410417 return dataStream .transform ("cluster_plan_generate" ,
411418 TypeInformation .of (ClusteringPlanEvent .class ),
412419 new ClusteringPlanOperator (conf ))
@@ -419,7 +426,7 @@ public static DataStreamSink<ClusteringCommitEvent> cluster(Configuration conf,
419426 .transform ("clustering_task" ,
420427 TypeInformation .of (ClusteringCommitEvent .class ),
421428 new ClusteringOperator (conf , rowType ))
422- .setParallelism (conf .getInteger (FlinkOptions .CLUSTERING_TASKS ))
429+ .setParallelism (conf .getOptional (FlinkOptions .CLUSTERING_TASKS ). orElse ( defaultParallelism ))
423430 .addSink (new ClusteringCommitSink (conf ))
424431 .name ("clustering_commit" )
425432 .setParallelism (1 ); // compaction commit should be singleton
@@ -439,6 +446,16 @@ public static String opIdentifier(String operatorN, Configuration conf) {
439446 return operatorN + ": " + conf .getString (FlinkOptions .TABLE_NAME );
440447 }
441448
449+ /**
450+ * Gets parallelism of tasks that do actual write.
451+ * @param conf The configuration
452+ * @param defaultParallelism The default parallelism
453+ * @return the parallelism of tasks that do actual write
454+ */
455+ private static int getWriteTasks (Configuration conf , int defaultParallelism ) {
456+ return conf .getOptional (FlinkOptions .WRITE_TASKS ).orElse (defaultParallelism );
457+ }
458+
442459 /**
443460 * Dummy sink that does nothing.
444461 */
0 commit comments