@@ -209,72 +209,73 @@ DataStream<TaskResult> append(DataStream<Trigger> trigger) {
209209
210210 // Collect all data files
211211 SingleOutputStreamOperator <MetadataTablePlanner .SplitInfo > splits =
212- trigger
213- . process (
214- new MetadataTablePlanner (
215- taskName (),
216- index (),
217- tableLoader (),
218- FILE_PATH_SCAN_CONTEXT ,
219- MetadataTableType . ALL_FILES ,
220- planningWorkerPoolSize ))
221- . name ( operatorName ( PLANNER_TASK_NAME ))
222- . uid ( PLANNER_TASK_NAME + uidSuffix ( ))
223- . slotSharingGroup ( slotSharingGroup ())
224- .forceNonParallel ();
212+ setSlotSharingGroup (
213+ trigger
214+ . process (
215+ new MetadataTablePlanner (
216+ taskName (),
217+ index (),
218+ tableLoader () ,
219+ FILE_PATH_SCAN_CONTEXT ,
220+ MetadataTableType . ALL_FILES ,
221+ planningWorkerPoolSize ))
222+ . name ( operatorName ( PLANNER_TASK_NAME ))
223+ . uid ( PLANNER_TASK_NAME + uidSuffix ())
224+ .forceNonParallel () );
225225
226226 // Read the records and get all data files
227227 SingleOutputStreamOperator <String > tableDataFiles =
228- splits
229- . rebalance ()
230- . process (
231- new FileNameReader (
232- taskName (),
233- index (),
234- tableLoader (),
235- FILE_PATH_SCHEMA ,
236- FILE_PATH_SCAN_CONTEXT ,
237- MetadataTableType . ALL_FILES ))
238- . name ( operatorName ( READER_TASK_NAME ))
239- . uid ( READER_TASK_NAME + uidSuffix ( ))
240- . slotSharingGroup ( slotSharingGroup ())
241- .setParallelism (parallelism ());
228+ setSlotSharingGroup (
229+ splits
230+ . rebalance ()
231+ . process (
232+ new FileNameReader (
233+ taskName (),
234+ index (),
235+ tableLoader () ,
236+ FILE_PATH_SCHEMA ,
237+ FILE_PATH_SCAN_CONTEXT ,
238+ MetadataTableType . ALL_FILES ))
239+ . name ( operatorName ( READER_TASK_NAME ))
240+ . uid ( READER_TASK_NAME + uidSuffix ())
241+ .setParallelism (parallelism () ));
242242
243243 // Collect all meta data files
244244 SingleOutputStreamOperator <String > tableMetadataFiles =
245- trigger
246- . process ( new ListMetadataFiles ( taskName (), index (), tableLoader ()))
247- . name ( operatorName ( METADATA_FILES_TASK_NAME ))
248- . uid ( METADATA_FILES_TASK_NAME + uidSuffix ( ))
249- . slotSharingGroup ( slotSharingGroup ())
250- .forceNonParallel ();
245+ setSlotSharingGroup (
246+ trigger
247+ . process ( new ListMetadataFiles ( taskName (), index (), tableLoader () ))
248+ . name ( operatorName ( METADATA_FILES_TASK_NAME ))
249+ . uid ( METADATA_FILES_TASK_NAME + uidSuffix ())
250+ .forceNonParallel () );
251251
252252 // List the all file system files
253253 SingleOutputStreamOperator <String > allFsFiles =
254- trigger
255- . process (
256- new ListFileSystemFiles (
257- taskName (),
258- index (),
259- tableLoader (),
260- location ,
261- minAge . toMillis () ,
262- usePrefixListing ))
263- . name ( operatorName ( FILESYSTEM_FILES_TASK_NAME ))
264- . uid ( FILESYSTEM_FILES_TASK_NAME + uidSuffix ( ))
265- . slotSharingGroup ( slotSharingGroup ())
266- .forceNonParallel ();
254+ setSlotSharingGroup (
255+ trigger
256+ . process (
257+ new ListFileSystemFiles (
258+ taskName (),
259+ index (),
260+ tableLoader () ,
261+ location ,
262+ minAge . toMillis (),
263+ usePrefixListing ))
264+ . name ( operatorName ( FILESYSTEM_FILES_TASK_NAME ))
265+ . uid ( FILESYSTEM_FILES_TASK_NAME + uidSuffix ())
266+ .forceNonParallel () );
267267
268268 SingleOutputStreamOperator <String > filesToDelete =
269- tableMetadataFiles
270- .union (tableDataFiles )
271- .keyBy (new FileUriKeySelector (equalSchemes , equalAuthorities ))
272- .connect (allFsFiles .keyBy (new FileUriKeySelector (equalSchemes , equalAuthorities )))
273- .process (new OrphanFilesDetector (prefixMismatchMode , equalSchemes , equalAuthorities ))
274- .slotSharingGroup (slotSharingGroup ())
275- .name (operatorName (FILTER_FILES_TASK_NAME ))
276- .uid (FILTER_FILES_TASK_NAME + uidSuffix ())
277- .setParallelism (parallelism ());
269+ setSlotSharingGroup (
270+ tableMetadataFiles
271+ .union (tableDataFiles )
272+ .keyBy (new FileUriKeySelector (equalSchemes , equalAuthorities ))
273+ .connect (allFsFiles .keyBy (new FileUriKeySelector (equalSchemes , equalAuthorities )))
274+ .process (
275+ new OrphanFilesDetector (prefixMismatchMode , equalSchemes , equalAuthorities ))
276+ .name (operatorName (FILTER_FILES_TASK_NAME ))
277+ .uid (FILTER_FILES_TASK_NAME + uidSuffix ())
278+ .setParallelism (parallelism ()));
278279
279280 DataStream <Exception > errorStream =
280281 tableMetadataFiles
@@ -287,38 +288,38 @@ DataStream<TaskResult> append(DataStream<Trigger> trigger) {
287288
288289 // Stop deleting the files if there is an error
289290 SingleOutputStreamOperator <String > filesOrSkip =
290- filesToDelete
291- . connect ( errorStream )
292- . transform (
293- operatorName ( SKIP_ON_ERROR_TASK_NAME ),
294- TypeInformation . of ( String . class ),
295- new SkipOnError ())
296- . uid ( SKIP_ON_ERROR_TASK_NAME + uidSuffix ())
297- . slotSharingGroup ( slotSharingGroup ())
298- .forceNonParallel ();
291+ setSlotSharingGroup (
292+ filesToDelete
293+ . connect ( errorStream )
294+ . transform (
295+ operatorName ( SKIP_ON_ERROR_TASK_NAME ),
296+ TypeInformation . of ( String . class ),
297+ new SkipOnError ())
298+ . uid ( SKIP_ON_ERROR_TASK_NAME + uidSuffix ())
299+ .forceNonParallel () );
299300
300301 // delete the files
301- filesOrSkip
302- . rebalance ()
303- . transform (
304- operatorName ( DELETE_FILES_TASK_NAME ),
305- TypeInformation . of ( Void . class ),
306- new DeleteFilesProcessor (
307- tableLoader (). loadTable (), taskName (), index (), deleteBatchSize ))
308- . uid ( DELETE_FILES_TASK_NAME + uidSuffix ( ))
309- . slotSharingGroup ( slotSharingGroup ())
310- .setParallelism (parallelism ());
302+ setSlotSharingGroup (
303+ filesOrSkip
304+ . rebalance ()
305+ . transform (
306+ operatorName ( DELETE_FILES_TASK_NAME ),
307+ TypeInformation . of ( Void . class ),
308+ new DeleteFilesProcessor (
309+ tableLoader (). loadTable (), taskName (), index (), deleteBatchSize ))
310+ . uid ( DELETE_FILES_TASK_NAME + uidSuffix ())
311+ .setParallelism (parallelism () ));
311312
312313 // Ignore the file deletion result and return the DataStream<TaskResult> directly
313- return trigger
314- . connect ( errorStream )
315- . transform (
316- operatorName ( AGGREGATOR_TASK_NAME ),
317- TypeInformation . of ( TaskResult . class ),
318- new TaskResultAggregator ( tableName (), taskName (), index ()))
319- . uid ( AGGREGATOR_TASK_NAME + uidSuffix ( ))
320- . slotSharingGroup ( slotSharingGroup ())
321- .forceNonParallel ();
314+ return setSlotSharingGroup (
315+ trigger
316+ . connect ( errorStream )
317+ . transform (
318+ operatorName ( AGGREGATOR_TASK_NAME ),
319+ TypeInformation . of ( TaskResult . class ),
320+ new TaskResultAggregator ( tableName (), taskName (), index () ))
321+ . uid ( AGGREGATOR_TASK_NAME + uidSuffix ())
322+ .forceNonParallel () );
322323 }
323324 }
324325
0 commit comments