31
31
import org .apache .kafka .clients .consumer .internals .NetworkClientDelegate ;
32
32
import org .apache .kafka .clients .consumer .internals .OffsetsRequestManager ;
33
33
import org .apache .kafka .clients .consumer .internals .RequestManagers ;
34
+ import org .apache .kafka .clients .consumer .internals .StreamsGroupHeartbeatRequestManager ;
35
+ import org .apache .kafka .clients .consumer .internals .StreamsMembershipManager ;
34
36
import org .apache .kafka .clients .consumer .internals .SubscriptionState ;
35
37
import org .apache .kafka .clients .consumer .internals .TopicMetadataRequestManager ;
36
38
import org .apache .kafka .common .Cluster ;
37
39
import org .apache .kafka .common .KafkaException ;
38
40
import org .apache .kafka .common .TopicPartition ;
41
+ import org .apache .kafka .common .utils .LogCaptureAppender ;
39
42
import org .apache .kafka .common .utils .LogContext ;
40
43
import org .apache .kafka .common .utils .MockTime ;
41
44
import org .apache .kafka .common .utils .Time ;
42
45
46
+ import org .apache .logging .log4j .Level ;
43
47
import org .junit .jupiter .api .Test ;
44
48
import org .junit .jupiter .params .ParameterizedTest ;
45
49
import org .junit .jupiter .params .provider .Arguments ;
@@ -86,6 +90,8 @@ public class ApplicationEventProcessorTest {
86
90
private final OffsetsRequestManager offsetsRequestManager = mock (OffsetsRequestManager .class );
87
91
private SubscriptionState subscriptionState = mock (SubscriptionState .class );
88
92
private final ConsumerMetadata metadata = mock (ConsumerMetadata .class );
93
+ private final StreamsGroupHeartbeatRequestManager streamsGroupHeartbeatRequestManager = mock (StreamsGroupHeartbeatRequestManager .class );
94
+ private final StreamsMembershipManager streamsMembershipManager = mock (StreamsMembershipManager .class );
89
95
private ApplicationEventProcessor processor ;
90
96
91
97
private void setupProcessor (boolean withGroupId ) {
@@ -109,6 +115,27 @@ private void setupProcessor(boolean withGroupId) {
109
115
);
110
116
}
111
117
118
+ private void setupStreamProcessor (boolean withGroupId ) {
119
+ RequestManagers requestManagers = new RequestManagers (
120
+ new LogContext (),
121
+ offsetsRequestManager ,
122
+ mock (TopicMetadataRequestManager .class ),
123
+ mock (FetchRequestManager .class ),
124
+ withGroupId ? Optional .of (mock (CoordinatorRequestManager .class )) : Optional .empty (),
125
+ withGroupId ? Optional .of (commitRequestManager ) : Optional .empty (),
126
+ withGroupId ? Optional .of (heartbeatRequestManager ) : Optional .empty (),
127
+ Optional .empty (),
128
+ withGroupId ? Optional .of (streamsGroupHeartbeatRequestManager ) : Optional .empty (),
129
+ withGroupId ? Optional .of (streamsMembershipManager ) : Optional .empty ()
130
+ );
131
+ processor = new ApplicationEventProcessor (
132
+ new LogContext (),
133
+ requestManagers ,
134
+ metadata ,
135
+ subscriptionState
136
+ );
137
+ }
138
+
112
139
@ Test
113
140
public void testPrepClosingCommitEvents () {
114
141
setupProcessor (true );
@@ -556,6 +583,78 @@ public void testAsyncCommitEventWithException() {
556
583
assertFutureThrows (IllegalStateException .class , event .future ());
557
584
}
558
585
586
+ @ Test
587
+ public void testStreamsOnTasksRevokedCallbackCompletedEvent () {
588
+ setupStreamProcessor (true );
589
+ StreamsOnTasksRevokedCallbackCompletedEvent event =
590
+ new StreamsOnTasksRevokedCallbackCompletedEvent (new CompletableFuture <>(), Optional .empty ());
591
+ processor .process (event );
592
+ verify (streamsMembershipManager ).onTasksRevokedCallbackCompleted (event );
593
+ }
594
+
595
+ @ Test
596
+ public void testStreamsOnTasksRevokedCallbackCompletedEventWithoutStreamsMembershipManager () {
597
+ setupStreamProcessor (false );
598
+ StreamsOnTasksRevokedCallbackCompletedEvent event =
599
+ new StreamsOnTasksRevokedCallbackCompletedEvent (new CompletableFuture <>(), Optional .empty ());
600
+ try (final LogCaptureAppender logAppender = LogCaptureAppender .createAndRegister ()) {
601
+ logAppender .setClassLogger (ApplicationEventProcessor .class , Level .WARN );
602
+ processor .process (event );
603
+ assertTrue (logAppender .getMessages ().stream ().anyMatch (e ->
604
+ e .contains ("An internal error occurred; the Streams membership manager was not present, so the notification " +
605
+ "of the onTasksRevoked callback execution could not be sent" )));
606
+ verify (streamsMembershipManager , never ()).onTasksRevokedCallbackCompleted (event );
607
+ }
608
+ }
609
+
610
+ @ Test
611
+ public void testStreamsOnTasksAssignedCallbackCompletedEvent () {
612
+ setupStreamProcessor (true );
613
+ StreamsOnTasksAssignedCallbackCompletedEvent event =
614
+ new StreamsOnTasksAssignedCallbackCompletedEvent (new CompletableFuture <>(), Optional .empty ());
615
+ processor .process (event );
616
+ verify (streamsMembershipManager ).onTasksAssignedCallbackCompleted (event );
617
+ }
618
+
619
+ @ Test
620
+ public void testStreamsOnTasksAssignedCallbackCompletedEventWithoutStreamsMembershipManager () {
621
+ setupStreamProcessor (false );
622
+ StreamsOnTasksAssignedCallbackCompletedEvent event =
623
+ new StreamsOnTasksAssignedCallbackCompletedEvent (new CompletableFuture <>(), Optional .empty ());
624
+ try (final LogCaptureAppender logAppender = LogCaptureAppender .createAndRegister ()) {
625
+ logAppender .setClassLogger (ApplicationEventProcessor .class , Level .WARN );
626
+ processor .process (event );
627
+ assertTrue (logAppender .getMessages ().stream ().anyMatch (e ->
628
+ e .contains ("An internal error occurred; the Streams membership manager was not present, so the notification " +
629
+ "of the onTasksAssigned callback execution could not be sent" )));
630
+ verify (streamsMembershipManager , never ()).onTasksAssignedCallbackCompleted (event );
631
+ }
632
+ }
633
+
634
+ @ Test
635
+ public void testStreamsOnAllTasksLostCallbackCompletedEvent () {
636
+ setupStreamProcessor (true );
637
+ StreamsOnAllTasksLostCallbackCompletedEvent event =
638
+ new StreamsOnAllTasksLostCallbackCompletedEvent (new CompletableFuture <>(), Optional .empty ());
639
+ processor .process (event );
640
+ verify (streamsMembershipManager ).onAllTasksLostCallbackCompleted (event );
641
+ }
642
+
643
+ @ Test
644
+ public void testStreamsOnAllTasksLostCallbackCompletedEventWithoutStreamsMembershipManager () {
645
+ setupStreamProcessor (false );
646
+ StreamsOnAllTasksLostCallbackCompletedEvent event =
647
+ new StreamsOnAllTasksLostCallbackCompletedEvent (new CompletableFuture <>(), Optional .empty ());
648
+ try (final LogCaptureAppender logAppender = LogCaptureAppender .createAndRegister ()) {
649
+ logAppender .setClassLogger (ApplicationEventProcessor .class , Level .WARN );
650
+ processor .process (event );
651
+ assertTrue (logAppender .getMessages ().stream ().anyMatch (e ->
652
+ e .contains ("An internal error occurred; the Streams membership manager was not present, so the notification " +
653
+ "of the onAllTasksLost callback execution could not be sent" )));
654
+ verify (streamsMembershipManager , never ()).onAllTasksLostCallbackCompleted (event );
655
+ }
656
+ }
657
+
559
658
private List <NetworkClientDelegate .UnsentRequest > mockCommitResults () {
560
659
return Collections .singletonList (mock (NetworkClientDelegate .UnsentRequest .class ));
561
660
}
0 commit comments