60
60
import static org .junit .jupiter .api .Assertions .assertThrows ;
61
61
import static org .junit .jupiter .api .Assertions .assertTrue ;
62
62
import static org .mockito .ArgumentMatchers .any ;
63
+ import static org .mockito .ArgumentMatchers .argThat ;
63
64
import static org .mockito .Mockito .never ;
64
65
import static org .mockito .Mockito .times ;
65
66
import static org .mockito .Mockito .verify ;
@@ -185,8 +186,8 @@ public void testReconcilingEmptyToSingleActiveTask() {
185
186
186
187
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
187
188
188
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
189
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
189
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
190
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
190
191
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
191
192
onTasksAssignedCallbackExecuted .complete (null );
192
193
verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
@@ -221,8 +222,8 @@ public void testReconcilingActiveTaskToDifferentActiveTask() {
221
222
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
222
223
223
224
final Set <TopicPartition > expectedPartitionsToRevoke = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
224
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
225
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
225
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
226
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
226
227
verifyInStateReconcilingBeforeOnTaskRevokedCallbackExecuted (
227
228
expectedPartitionsToRevoke ,
228
229
expectedFullPartitionsToAssign ,
@@ -231,7 +232,7 @@ public void testReconcilingActiveTaskToDifferentActiveTask() {
231
232
onTasksRevokedCallbackExecuted .complete (null );
232
233
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
233
234
onTasksAssignedCallbackExecuted .complete (null );
234
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
235
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
235
236
}
236
237
237
238
@ Test
@@ -259,14 +260,14 @@ public void testReconcilingSingleActiveTaskToAdditionalActiveTask() {
259
260
260
261
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 , PARTITION_1 )));
261
262
262
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (
263
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (
263
264
new TopicPartition (TOPIC_0 , PARTITION_0 ),
264
265
new TopicPartition (TOPIC_0 , PARTITION_1 )
265
266
);
266
- final Collection <TopicPartition > expectedNewPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
267
+ final Set <TopicPartition > expectedNewPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
267
268
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
268
269
onTasksAssignedCallbackExecuted .complete (null );
269
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
270
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
270
271
verifyThatNoTasksHaveBeenRevoked ();
271
272
}
272
273
@@ -302,8 +303,8 @@ public void testReconcilingMultipleActiveTaskToSingleActiveTask() {
302
303
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
303
304
304
305
final Set <TopicPartition > expectedPartitionsToRevoke = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
305
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
306
- final Collection <TopicPartition > expectedNewPartitionsToAssign = Collections .emptySet ();
306
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
307
+ final Set <TopicPartition > expectedNewPartitionsToAssign = Collections .emptySet ();
307
308
verifyInStateReconcilingBeforeOnTaskRevokedCallbackExecuted (
308
309
expectedPartitionsToRevoke ,
309
310
expectedFullPartitionsToAssign ,
@@ -312,7 +313,7 @@ public void testReconcilingMultipleActiveTaskToSingleActiveTask() {
312
313
onTasksRevokedCallbackExecuted .complete (null );
313
314
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
314
315
onTasksAssignedCallbackExecuted .complete (null );
315
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
316
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
316
317
}
317
318
318
319
@ Test
@@ -338,14 +339,14 @@ public void testReconcilingEmptyToMultipleActiveTaskOfDifferentSubtopologies() {
338
339
SUBTOPOLOGY_ID_1 , List .of (PARTITION_0 ))
339
340
);
340
341
341
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (
342
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (
342
343
new TopicPartition (TOPIC_0 , PARTITION_0 ),
343
344
new TopicPartition (TOPIC_1 , PARTITION_0 )
344
345
);
345
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
346
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
346
347
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
347
348
onTasksAssignedCallbackExecuted .complete (null );
348
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
349
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
349
350
verifyThatNoTasksHaveBeenRevoked ();
350
351
}
351
352
@@ -382,8 +383,8 @@ public void testReconcilingActiveTaskToStandbyTask() {
382
383
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
383
384
384
385
final Set <TopicPartition > expectedPartitionsToRevoke = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
385
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
386
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
386
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
387
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
387
388
verifyInStateReconcilingBeforeOnTaskRevokedCallbackExecuted (
388
389
expectedPartitionsToRevoke ,
389
390
expectedFullPartitionsToAssign ,
@@ -392,7 +393,7 @@ public void testReconcilingActiveTaskToStandbyTask() {
392
393
onTasksRevokedCallbackExecuted .complete (null );
393
394
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
394
395
onTasksAssignedCallbackExecuted .complete (null );
395
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
396
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
396
397
}
397
398
398
399
@ Test
@@ -428,8 +429,8 @@ public void testReconcilingActiveTaskToWarmupTask() {
428
429
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
429
430
430
431
final Set <TopicPartition > expectedPartitionsToRevoke = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
431
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
432
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
432
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
433
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
433
434
verifyInStateReconcilingBeforeOnTaskRevokedCallbackExecuted (
434
435
expectedPartitionsToRevoke ,
435
436
expectedFullPartitionsToAssign ,
@@ -438,7 +439,7 @@ public void testReconcilingActiveTaskToWarmupTask() {
438
439
onTasksRevokedCallbackExecuted .complete (null );
439
440
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
440
441
onTasksAssignedCallbackExecuted .complete (null );
441
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
442
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
442
443
}
443
444
444
445
@ Test
@@ -455,11 +456,11 @@ public void testReconcilingEmptyToSingleStandbyTask() {
455
456
456
457
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
457
458
458
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
459
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
459
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
460
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
460
461
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
461
462
onTasksAssignedCallbackExecuted .complete (null );
462
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
463
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
463
464
verifyThatNoTasksHaveBeenRevoked ();
464
465
}
465
466
@@ -487,14 +488,15 @@ public void testReconcilingStandbyTaskToDifferentStandbyTask() {
487
488
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
488
489
acknowledging (onTasksAssignedCallbackExecutedSetup );
489
490
Mockito .reset (subscriptionState );
491
+ Mockito .reset (memberStateListener );
490
492
491
493
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
492
494
493
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
494
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
495
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
496
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
495
497
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
496
498
onTasksAssignedCallbackExecuted .complete (null );
497
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
499
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
498
500
verifyThatNoTasksHaveBeenRevoked ();
499
501
}
500
502
@@ -523,14 +525,15 @@ public void testReconcilingSingleStandbyTaskToAdditionalStandbyTask() {
523
525
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
524
526
acknowledging (onTasksAssignedCallbackExecutedSetup );
525
527
Mockito .reset (subscriptionState );
528
+ Mockito .reset (memberStateListener );
526
529
527
530
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 , PARTITION_1 )));
528
531
529
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
530
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
532
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
533
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
531
534
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
532
535
onTasksAssignedCallbackExecuted .complete (null );
533
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
536
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
534
537
verifyThatNoTasksHaveBeenRevoked ();
535
538
}
536
539
@@ -559,14 +562,15 @@ public void testReconcilingMultipleStandbyTaskToSingleStandbyTask() {
559
562
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 , PARTITION_1 )));
560
563
acknowledging (onTasksAssignedCallbackExecutedSetup );
561
564
Mockito .reset (subscriptionState );
565
+ Mockito .reset (memberStateListener );
562
566
563
567
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
564
568
565
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
566
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
569
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
570
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
567
571
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
568
572
onTasksAssignedCallbackExecuted .complete (null );
569
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
573
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
570
574
verifyThatNoTasksHaveBeenRevoked ();
571
575
}
572
576
@@ -601,11 +605,11 @@ public void testReconcilingStandbyTaskToActiveTask() {
601
605
602
606
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
603
607
604
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
605
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
608
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
609
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
606
610
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
607
611
onTasksAssignedCallbackExecuted .complete (null );
608
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
612
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
609
613
verifyThatNoTasksHaveBeenRevoked ();
610
614
}
611
615
@@ -631,14 +635,15 @@ public void testReconcilingStandbyTaskToWarmupTask() {
631
635
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
632
636
acknowledging (onTasksAssignedCallbackExecutedSetup );
633
637
Mockito .reset (subscriptionState );
638
+ Mockito .reset (memberStateListener );
634
639
635
640
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
636
641
637
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
638
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
642
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
643
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
639
644
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
640
645
onTasksAssignedCallbackExecuted .complete (null );
641
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
646
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
642
647
verifyThatNoTasksHaveBeenRevoked ();
643
648
}
644
649
@@ -656,11 +661,11 @@ public void testReconcilingEmptyToSingleWarmupTask() {
656
661
657
662
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
658
663
659
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
660
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
664
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
665
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
661
666
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
662
667
onTasksAssignedCallbackExecuted .complete (null );
663
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
668
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
664
669
verifyThatNoTasksHaveBeenRevoked ();
665
670
}
666
671
@@ -688,14 +693,15 @@ public void testReconcilingWarmupTaskToDifferentWarmupTask() {
688
693
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
689
694
acknowledging (onTasksAssignedCallbackExecutedSetup );
690
695
Mockito .reset (subscriptionState );
696
+ Mockito .reset (memberStateListener );
691
697
692
698
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
693
699
694
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
695
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
700
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
701
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
696
702
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
697
703
onTasksAssignedCallbackExecuted .complete (null );
698
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
704
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
699
705
verifyThatNoTasksHaveBeenRevoked ();
700
706
}
701
707
@@ -724,14 +730,15 @@ public void testReconcilingSingleWarmupTaskToAdditionalWarmupTask() {
724
730
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
725
731
acknowledging (onTasksAssignedCallbackExecutedSetup );
726
732
Mockito .reset (subscriptionState );
733
+ Mockito .reset (memberStateListener );
727
734
728
735
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 , PARTITION_1 )));
729
736
730
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
731
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
737
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
738
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
732
739
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
733
740
onTasksAssignedCallbackExecuted .complete (null );
734
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
741
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
735
742
verifyThatNoTasksHaveBeenRevoked ();
736
743
}
737
744
@@ -760,14 +767,15 @@ public void testReconcilingMultipleWarmupTaskToSingleWarmupTask() {
760
767
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 , PARTITION_1 )));
761
768
acknowledging (onTasksAssignedCallbackExecutedSetup );
762
769
Mockito .reset (subscriptionState );
770
+ Mockito .reset (memberStateListener );
763
771
764
772
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
765
773
766
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
767
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
774
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
775
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
768
776
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
769
777
onTasksAssignedCallbackExecuted .complete (null );
770
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
778
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
771
779
verifyThatNoTasksHaveBeenRevoked ();
772
780
}
773
781
@@ -802,11 +810,11 @@ public void testReconcilingWarmupTaskToActiveTask() {
802
810
803
811
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
804
812
805
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
806
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
813
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
814
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
807
815
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
808
816
onTasksAssignedCallbackExecuted .complete (null );
809
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
817
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
810
818
verifyThatNoTasksHaveBeenRevoked ();
811
819
}
812
820
@@ -834,14 +842,15 @@ public void testReconcilingWarmupTaskToStandbyTask() {
834
842
reconcile (makeHeartbeatResponseWithWarmupTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
835
843
acknowledging (onTasksAssignedCallbackExecutedSetup );
836
844
Mockito .reset (subscriptionState );
845
+ Mockito .reset (memberStateListener );
837
846
838
847
reconcile (makeHeartbeatResponseWithStandbyTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
839
848
840
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
841
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
849
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Collections .emptySet ();
850
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
842
851
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
843
852
onTasksAssignedCallbackExecuted .complete (null );
844
- verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedNewPartitionsToAssign );
853
+ verifyInStateAcknowledgingAfterOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign );
845
854
verifyThatNoTasksHaveBeenRevoked ();
846
855
}
847
856
@@ -857,8 +866,8 @@ public void testReconcilingAndAssignmentCallbackFails() {
857
866
858
867
reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
859
868
860
- final Collection <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
861
- final Collection <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
869
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
870
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
862
871
verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
863
872
864
873
onTasksAssignedCallbackExecuted .completeExceptionally (new RuntimeException ("KABOOM!" ));
@@ -867,6 +876,54 @@ public void testReconcilingAndAssignmentCallbackFails() {
867
876
verify (subscriptionState , never ()).enablePartitionsAwaitingCallback (any ());
868
877
}
869
878
879
+ @ Test
880
+ public void testReconcilingAndRevocationCallbackFails () {
881
+ setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic (SUBTOPOLOGY_ID_0 , TOPIC_0 );
882
+ final CompletableFuture <Void > onTasksAssignedCallbackExecutedSetup = new CompletableFuture <>();
883
+ final CompletableFuture <Void > onTasksRevokedCallbackExecuted = new CompletableFuture <>();
884
+ final Set <StreamsRebalanceData .TaskId > activeTasksSetup = Set .of (
885
+ new StreamsRebalanceData .TaskId (SUBTOPOLOGY_ID_0 , PARTITION_0 )
886
+ );
887
+ final Set <StreamsRebalanceData .TaskId > activeTasks = Set .of (
888
+ new StreamsRebalanceData .TaskId (SUBTOPOLOGY_ID_0 , PARTITION_1 )
889
+ );
890
+ when (streamsRebalanceEventsProcessor .requestOnTasksAssignedCallbackInvocation (makeTaskAssignment (activeTasksSetup , Collections .emptySet (), Collections .emptySet ())))
891
+ .thenReturn (onTasksAssignedCallbackExecutedSetup );
892
+ when (streamsRebalanceEventsProcessor .requestOnTasksRevokedCallbackInvocation (activeTasksSetup ))
893
+ .thenReturn (onTasksRevokedCallbackExecuted );
894
+ when (subscriptionState .assignedPartitions ())
895
+ .thenReturn (Collections .emptySet ())
896
+ .thenReturn (Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 )));
897
+ joining ();
898
+ reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_0 )));
899
+ acknowledging (onTasksAssignedCallbackExecutedSetup );
900
+
901
+ reconcile (makeHeartbeatResponseWithActiveTasks (SUBTOPOLOGY_ID_0 , List .of (PARTITION_1 )));
902
+
903
+ final Set <TopicPartition > partitionsToAssignAtSetup = Set .of (new TopicPartition (TOPIC_0 , PARTITION_0 ));
904
+ final Set <TopicPartition > expectedPartitionsToRevoke = partitionsToAssignAtSetup ;
905
+ final Set <TopicPartition > expectedFullPartitionsToAssign = Set .of (new TopicPartition (TOPIC_0 , PARTITION_1 ));
906
+ final Set <TopicPartition > expectedNewPartitionsToAssign = expectedFullPartitionsToAssign ;
907
+ verifyInStateReconcilingBeforeOnTaskRevokedCallbackExecuted (
908
+ expectedPartitionsToRevoke ,
909
+ expectedFullPartitionsToAssign ,
910
+ expectedNewPartitionsToAssign
911
+ );
912
+
913
+ onTasksRevokedCallbackExecuted .completeExceptionally (new RuntimeException ("KABOOM!" ));
914
+
915
+ verify (subscriptionState ).markPendingRevocation (expectedPartitionsToRevoke );
916
+ verify (subscriptionState , never ()).assignFromSubscribedAwaitingCallback (expectedFullPartitionsToAssign , expectedNewPartitionsToAssign );
917
+ verify (memberStateListener , never ()).onGroupAssignmentUpdated (expectedFullPartitionsToAssign );
918
+ verify (subscriptionState , never ())
919
+ .enablePartitionsAwaitingCallback (argThat (a -> !a .equals (partitionsToAssignAtSetup )));
920
+ verifyInStateReconciling (membershipManager );
921
+ verify (streamsRebalanceEventsProcessor , never ()).requestOnTasksAssignedCallbackInvocation (
922
+ makeTaskAssignment (activeTasks , Collections .emptySet (), Collections .emptySet ())
923
+ );
924
+ verifyInStateReconciling (membershipManager );
925
+ }
926
+
870
927
@ Test
871
928
public void testLeaveGroupWhenNotInGroup () {
872
929
testLeaveGroupWhenNotInGroup (membershipManager ::leaveGroup );
@@ -885,6 +942,7 @@ private void testLeaveGroupWhenNotInGroup(final Supplier<CompletableFuture<Void>
885
942
assertFalse (future .isCancelled ());
886
943
assertFalse (future .isCompletedExceptionally ());
887
944
verify (subscriptionState ).unsubscribe ();
945
+ verify (memberStateListener ).onGroupAssignmentUpdated (Collections .emptySet ());
888
946
verifyInStateUnsubscribed (membershipManager );
889
947
}
890
948
@@ -912,6 +970,7 @@ private void testLeaveGroupOnCloseWhenNotInGroupAndFenced(final Supplier<Complet
912
970
assertFalse (future .isCompletedExceptionally ());
913
971
verify (subscriptionState ).unsubscribe ();
914
972
verify (subscriptionState ).assignFromSubscribed (Collections .emptySet ());
973
+ verify (memberStateListener , times (2 )).onGroupAssignmentUpdated (Collections .emptySet ());
915
974
verifyInStateUnsubscribed (membershipManager );
916
975
}
917
976
@@ -940,6 +999,7 @@ public void testLeaveGroupWhenInGroupWithAssignment() {
940
999
final CompletableFuture <Void > onGroupLeftOnCloseBeforeRevocationCallback = membershipManager .leaveGroupOnClose ();
941
1000
assertEquals (onGroupLeft , onGroupLeftOnCloseBeforeRevocationCallback );
942
1001
onTasksRevokedCallbackExecuted .complete (null );
1002
+ verify (memberStateListener ).onGroupAssignmentUpdated (Collections .emptySet ());
943
1003
verify (subscriptionState ).unsubscribe ();
944
1004
assertFalse (onGroupLeft .isDone ());
945
1005
verifyInStateLeaving (membershipManager );
@@ -970,6 +1030,7 @@ public void testLeaveGroupOnCloseWhenInGroupWithAssignment() {
970
1030
assertFalse (onGroupLeft .isDone ());
971
1031
verifyInStateLeaving (membershipManager );
972
1032
verify (subscriptionState ).unsubscribe ();
1033
+ verify (memberStateListener ).onGroupAssignmentUpdated (Collections .emptySet ());
973
1034
verify (streamsRebalanceEventsProcessor , never ()).requestOnTasksRevokedCallbackInvocation (any ());
974
1035
final CompletableFuture <Void > onGroupLeftBeforeHeartbeatRequestGenerated = membershipManager .leaveGroup ();
975
1036
assertEquals (onGroupLeft , onGroupLeftBeforeHeartbeatRequestGenerated );
@@ -1199,6 +1260,10 @@ public void testOnHeartbeatRequestGeneratedWhenInLeavingAndPollTimerExpired() {
1199
1260
membershipManager .onHeartbeatRequestGenerated ();
1200
1261
1201
1262
verifyInStateStale (membershipManager );
1263
+ verify (subscriptionState , never ()).assignFromSubscribed (Collections .emptySet ());
1264
+ onAllTasksLostCallbackExecuted .complete (null );
1265
+ verify (subscriptionState ).assignFromSubscribed (Collections .emptySet ());
1266
+ verify (memberStateListener ).onGroupAssignmentUpdated (Collections .emptySet ());
1202
1267
}
1203
1268
1204
1269
@ Test
@@ -1313,6 +1378,7 @@ private void testOnFencedWhenInJoiningOrReconcilingOrAcknowledgingOrStable() {
1313
1378
verify (subscriptionState , never ()).assignFromSubscribed (Collections .emptySet ());
1314
1379
onAllTasksLostCallbackExecuted .complete (null );
1315
1380
verify (subscriptionState ).assignFromSubscribed (Collections .emptySet ());
1381
+ verify (memberStateListener ).onGroupAssignmentUpdated (Collections .emptySet ());
1316
1382
verifyInStateJoining (membershipManager );
1317
1383
}
1318
1384
@@ -1354,6 +1420,7 @@ public void testTransitionToFatalWhenInPrepareLeaving() {
1354
1420
joining ();
1355
1421
1356
1422
testTransitionToFatalWhenInPrepareLeavingOrLeaving (prepareLeaving ());
1423
+
1357
1424
verify (memberStateListener ).onMemberEpochUpdated (Optional .empty (), membershipManager .memberId ());
1358
1425
}
1359
1426
@@ -1452,6 +1519,7 @@ private void testTransitionToFatalWhenInJoiningOrReconcilingOrAcknowledgingOrSta
1452
1519
verify (subscriptionState ).assignFromSubscribed (Collections .emptySet ());
1453
1520
verifyInStateFatal (membershipManager );
1454
1521
verify (memberStateListener ).onMemberEpochUpdated (Optional .empty (), membershipManager .memberId ());
1522
+ verify (memberStateListener ).onGroupAssignmentUpdated (Collections .emptySet ());
1455
1523
}
1456
1524
1457
1525
@ Test
@@ -1619,16 +1687,17 @@ private void verifyThatNoTasksHaveBeenRevoked() {
1619
1687
}
1620
1688
1621
1689
private void verifyInStateReconcilingBeforeOnTaskRevokedCallbackExecuted (Set <TopicPartition > expectedPartitionsToRevoke ,
1622
- Collection <TopicPartition > expectedAllPartitionsToAssign ,
1623
- Collection <TopicPartition > expectedNewPartitionsToAssign ) {
1690
+ Set <TopicPartition > expectedAllPartitionsToAssign ,
1691
+ Set <TopicPartition > expectedNewPartitionsToAssign ) {
1624
1692
verify (subscriptionState ).markPendingRevocation (expectedPartitionsToRevoke );
1625
1693
verify (subscriptionState , never ()).assignFromSubscribedAwaitingCallback (expectedAllPartitionsToAssign , expectedNewPartitionsToAssign );
1626
1694
verifyInStateReconciling (membershipManager );
1627
1695
}
1628
1696
1629
- private void verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (Collection <TopicPartition > expectedAllPartitionsToAssign ,
1630
- Collection <TopicPartition > expectedNewPartitionsToAssign ) {
1697
+ private void verifyInStateReconcilingBeforeOnTaskAssignedCallbackExecuted (Set <TopicPartition > expectedAllPartitionsToAssign ,
1698
+ Set <TopicPartition > expectedNewPartitionsToAssign ) {
1631
1699
verify (subscriptionState ).assignFromSubscribedAwaitingCallback (expectedAllPartitionsToAssign , expectedNewPartitionsToAssign );
1700
+ verify (memberStateListener ).onGroupAssignmentUpdated (expectedAllPartitionsToAssign );
1632
1701
verify (subscriptionState , never ()).enablePartitionsAwaitingCallback (expectedNewPartitionsToAssign );
1633
1702
verifyInStateReconciling (membershipManager );
1634
1703
}
0 commit comments