@@ -91,7 +91,7 @@ public class PhysicalPlanGenerator {
91
91
92
92
private final List <Pipeline > pipelines ;
93
93
94
- private final IdGenerator idGenerator = new IdGenerator ();
94
+ private final IdGenerator taskGroupIdGenerator = new IdGenerator ();
95
95
96
96
private final JobImmutableInformation jobImmutableInformation ;
97
97
@@ -274,15 +274,14 @@ private List<PhysicalVertex> getCommitterTask(
274
274
}
275
275
// if sinkAggregatedCommitter is empty, don't create task.
276
276
if (sinkAggregatedCommitter .isPresent ()) {
277
- long taskGroupID = idGenerator .getNextId ();
278
- long taskTypeId = idGenerator .getNextId ();
277
+ long taskGroupID = taskGroupIdGenerator .getNextId ();
279
278
TaskGroupLocation taskGroupLocation =
280
279
new TaskGroupLocation (
281
280
jobImmutableInformation .getJobId (),
282
281
pipelineIndex ,
283
282
taskGroupID );
284
283
TaskLocation taskLocation =
285
- new TaskLocation (taskGroupLocation , taskTypeId , 0 );
284
+ new TaskLocation (taskGroupLocation , 0 , 0 );
286
285
SinkAggregatedCommitterTask <?, ?> t =
287
286
new SinkAggregatedCommitterTask (
288
287
jobImmutableInformation .getJobId (),
@@ -342,17 +341,15 @@ private List<PhysicalVertex> getShuffleTask(
342
341
if (shuffleStrategy instanceof ShuffleMultipleRowStrategy ) {
343
342
ShuffleMultipleRowStrategy shuffleMultipleRowStrategy =
344
343
(ShuffleMultipleRowStrategy ) shuffleStrategy ;
344
+ AtomicInteger atomicInteger = new AtomicInteger (0 );
345
345
for (Flow nextFlow : flow .getNext ()) {
346
346
PhysicalExecutionFlow sinkFlow =
347
347
(PhysicalExecutionFlow ) nextFlow ;
348
348
SinkAction sinkAction = (SinkAction ) sinkFlow .getAction ();
349
349
String sinkTableId =
350
350
sinkAction .getConfig ().getTablePath ().toString ();
351
351
352
- long taskIDPrefix = idGenerator .getNextId ();
353
- long taskGroupIDPrefix = idGenerator .getNextId ();
354
- int parallelismIndex = 0 ;
355
-
352
+ int parallelismIndex = atomicInteger .getAndIncrement ();
356
353
ShuffleStrategy shuffleStrategyOfSinkFlow =
357
354
shuffleMultipleRowStrategy
358
355
.toBuilder ()
@@ -363,7 +360,6 @@ private List<PhysicalVertex> getShuffleTask(
363
360
.toBuilder ()
364
361
.shuffleStrategy (shuffleStrategyOfSinkFlow )
365
362
.build ();
366
- long shuffleActionId = idGenerator .getNextId ();
367
363
String shuffleActionName =
368
364
String .format (
369
365
"%s -> %s -> %s" ,
@@ -372,7 +368,7 @@ private List<PhysicalVertex> getShuffleTask(
372
368
sinkAction .getName ());
373
369
ShuffleAction shuffleActionOfSinkFlow =
374
370
new ShuffleAction (
375
- shuffleActionId ,
371
+ parallelismIndex ,
376
372
shuffleActionName ,
377
373
shuffleConfigOfSinkFlow );
378
374
shuffleActionOfSinkFlow .setParallelism (1 );
@@ -382,19 +378,15 @@ private List<PhysicalVertex> getShuffleTask(
382
378
Collections .singletonList (sinkFlow ));
383
379
setFlowConfig (shuffleFlow );
384
380
385
- long taskGroupID =
386
- mixIDPrefixAndIndex (
387
- taskGroupIDPrefix , parallelismIndex );
381
+ long taskGroupID = taskGroupIdGenerator .getNextId ();
388
382
TaskGroupLocation taskGroupLocation =
389
383
new TaskGroupLocation (
390
384
jobImmutableInformation .getJobId (),
391
385
pipelineIndex ,
392
386
taskGroupID );
393
387
TaskLocation taskLocation =
394
388
new TaskLocation (
395
- taskGroupLocation ,
396
- taskIDPrefix ,
397
- parallelismIndex );
389
+ taskGroupLocation , 0 , parallelismIndex );
398
390
SeaTunnelTask seaTunnelTask =
399
391
new TransformSeaTunnelTask (
400
392
jobImmutableInformation .getJobId (),
@@ -428,17 +420,15 @@ private List<PhysicalVertex> getShuffleTask(
428
420
runningJobStateTimestampsIMap ));
429
421
}
430
422
} else {
431
- long taskIDPrefix = idGenerator .getNextId ();
432
- long taskGroupIDPrefix = idGenerator .getNextId ();
433
423
for (int i = 0 ; i < flow .getAction ().getParallelism (); i ++) {
434
- long taskGroupID = mixIDPrefixAndIndex ( taskGroupIDPrefix , i );
424
+ long taskGroupID = taskGroupIdGenerator . getNextId ( );
435
425
TaskGroupLocation taskGroupLocation =
436
426
new TaskGroupLocation (
437
427
jobImmutableInformation .getJobId (),
438
428
pipelineIndex ,
439
429
taskGroupID );
440
430
TaskLocation taskLocation =
441
- new TaskLocation (taskGroupLocation , taskIDPrefix , i );
431
+ new TaskLocation (taskGroupLocation , 0 , i );
442
432
setFlowConfig (flow );
443
433
SeaTunnelTask seaTunnelTask =
444
434
new TransformSeaTunnelTask (
@@ -483,15 +473,13 @@ private List<PhysicalVertex> getEnumeratorTask(
483
473
return sources .stream ()
484
474
.map (
485
475
sourceAction -> {
486
- long taskGroupID = idGenerator .getNextId ();
487
- long taskTypeId = idGenerator .getNextId ();
476
+ long taskGroupID = taskGroupIdGenerator .getNextId ();
488
477
TaskGroupLocation taskGroupLocation =
489
478
new TaskGroupLocation (
490
479
jobImmutableInformation .getJobId (),
491
480
pipelineIndex ,
492
481
taskGroupID );
493
- TaskLocation taskLocation =
494
- new TaskLocation (taskGroupLocation , taskTypeId , 0 );
482
+ TaskLocation taskLocation = new TaskLocation (taskGroupLocation , 0 , 0 );
495
483
SourceSplitEnumeratorTask <?> t =
496
484
new SourceSplitEnumeratorTask <>(
497
485
jobImmutableInformation .getJobId (),
@@ -541,32 +529,25 @@ private List<PhysicalVertex> getSourceTask(
541
529
if (sourceWithSink (flow )) {
542
530
flows .addAll (splitSinkFromFlow (flow ));
543
531
}
544
- long taskGroupIDPrefix = idGenerator .getNextId ();
545
- Map <Long , Long > flowTaskIDPrefixMap = new HashMap <>();
546
532
for (int i = 0 ; i < flow .getAction ().getParallelism (); i ++) {
533
+ long taskGroupId = taskGroupIdGenerator .getNextId ();
547
534
int finalParallelismIndex = i ;
548
- long taskGroupID = mixIDPrefixAndIndex (taskGroupIDPrefix , i );
549
535
TaskGroupLocation taskGroupLocation =
550
536
new TaskGroupLocation (
551
537
jobImmutableInformation .getJobId (),
552
538
pipelineIndex ,
553
- taskGroupID );
539
+ taskGroupId );
540
+ AtomicInteger taskInTaskGroupIndex = new AtomicInteger (0 );
554
541
List <SeaTunnelTask > taskList =
555
542
flows .stream ()
556
543
.map (
557
544
f -> {
558
545
setFlowConfig (f );
559
- long taskIDPrefix =
560
- flowTaskIDPrefixMap
561
- .computeIfAbsent (
562
- f .getFlowID (),
563
- id ->
564
- idGenerator
565
- .getNextId ());
566
546
final TaskLocation taskLocation =
567
547
new TaskLocation (
568
548
taskGroupLocation ,
569
- taskIDPrefix ,
549
+ taskInTaskGroupIndex
550
+ .getAndIncrement (),
570
551
finalParallelismIndex );
571
552
if (f
572
553
instanceof
@@ -768,10 +749,6 @@ private static boolean sourceWithSink(PhysicalExecutionFlow<?, ?> flow) {
768
749
.contains (true );
769
750
}
770
751
771
- private long mixIDPrefixAndIndex (long idPrefix , int index ) {
772
- return idPrefix * 10000 + index ;
773
- }
774
-
775
752
private List <Flow > getNextWrapper (List <ExecutionEdge > edges , Action start ) {
776
753
List <Action > actions =
777
754
edges .stream ()
0 commit comments