18
18
19
19
import org .apache .kafka .clients .consumer .internals .events .StreamsOnAllTasksLostCallbackCompletedEvent ;
20
20
import org .apache .kafka .clients .consumer .internals .events .StreamsOnTasksAssignedCallbackCompletedEvent ;
21
+ import org .apache .kafka .clients .consumer .internals .events .StreamsOnTasksRevokedCallbackCompletedEvent ;
21
22
import org .apache .kafka .common .KafkaException ;
23
+ import org .apache .kafka .common .MetricName ;
22
24
import org .apache .kafka .common .TopicPartition ;
23
25
import org .apache .kafka .common .message .StreamsGroupHeartbeatResponseData ;
24
26
import org .apache .kafka .common .metrics .Metrics ;
41
43
import java .util .List ;
42
44
import java .util .Optional ;
43
45
import java .util .Set ;
46
+ import java .util .SortedSet ;
47
+ import java .util .TreeSet ;
44
48
import java .util .concurrent .CompletableFuture ;
45
49
import java .util .concurrent .ExecutionException ;
46
50
import java .util .function .Supplier ;
51
+ import java .util .stream .Collectors ;
47
52
53
+ import static org .apache .kafka .clients .consumer .internals .ConsumerUtils .CONSUMER_METRIC_GROUP_PREFIX ;
54
+ import static org .apache .kafka .clients .consumer .internals .ConsumerUtils .COORDINATOR_METRICS_SUFFIX ;
48
55
import static org .apache .kafka .common .utils .Utils .mkEntry ;
49
56
import static org .apache .kafka .common .utils .Utils .mkMap ;
50
57
import static org .junit .jupiter .api .Assertions .assertEquals ;
@@ -73,8 +80,8 @@ public class StreamsMembershipManagerTest {
73
80
private static final int PARTITION_0 = 0 ;
74
81
private static final int PARTITION_1 = 1 ;
75
82
76
- private Time time = new MockTime (0 );
77
- private Metrics metrics = new Metrics (time );
83
+ private final Time time = new MockTime (0 );
84
+ private final Metrics metrics = new Metrics (time );
78
85
79
86
private StreamsMembershipManager membershipManager ;
80
87
@@ -1075,6 +1082,54 @@ public void testOnHeartbeatRequestGeneratedWhenInLeavingAndPollTimerExpired() {
1075
1082
verifyInStateStale (membershipManager );
1076
1083
}
1077
1084
1085
+ @ Test
1086
+ public void testOnHeartbeatFailureAfterLeaveRequestGenerated () {
1087
+ final CompletableFuture <Void > onAllTasksLostCallbackExecuted = new CompletableFuture <>();
1088
+ when (streamsRebalanceEventsProcessor .requestOnAllTasksLostCallbackInvocation ())
1089
+ .thenReturn (onAllTasksLostCallbackExecuted );
1090
+ joining ();
1091
+ final CompletableFuture <Void > groupLeft = leaving (onAllTasksLostCallbackExecuted );
1092
+ membershipManager .onHeartbeatRequestGenerated ();
1093
+ assertFalse (groupLeft .isDone ());
1094
+
1095
+ membershipManager .onHeartbeatFailure (true );
1096
+
1097
+ assertTrue (groupLeft .isDone ());
1098
+ }
1099
+
1100
+ @ Test
1101
+ public void testOnHeartbeatFatalFailure () {
1102
+ testOnHeartbeatFailure (false );
1103
+ }
1104
+
1105
+ @ Test
1106
+ public void testOnHeartbeatRetriableFailure () {
1107
+ testOnHeartbeatFailure (true );
1108
+ }
1109
+
1110
+ private void testOnHeartbeatFailure (boolean retriable ) {
1111
+ final MetricName failedRebalanceTotalMetricName = metrics .metricName (
1112
+ "failed-rebalance-total" ,
1113
+ CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX
1114
+ );
1115
+ setupStreamsAssignmentInterfaceWithOneSubtopologyOneSourceTopic (SUB_TOPOLOGY_ID_0 , TOPIC_0 );
1116
+ final Set <StreamsRebalanceData .TaskId > activeTasks =
1117
+ Set .of (new StreamsRebalanceData .TaskId (SUB_TOPOLOGY_ID_0 , PARTITION_0 ));
1118
+ final CompletableFuture <Void > onTasksAssignedCallbackExecuted = new CompletableFuture <>();
1119
+ when (streamsRebalanceEventsProcessor .requestOnTasksAssignedCallbackInvocation (makeTaskAssignment (activeTasks , Collections .emptySet (), Collections .emptySet ())))
1120
+ .thenReturn (onTasksAssignedCallbackExecuted );
1121
+ joining ();
1122
+ time .sleep (1 );
1123
+ reconcile (makeHeartbeatResponseWithActiveTasks (SUB_TOPOLOGY_ID_0 , List .of (PARTITION_0 )));
1124
+ final double failedRebalancesTotalBefore = (double ) metrics .metric (failedRebalanceTotalMetricName ).metricValue ();
1125
+ assertEquals (0L , failedRebalancesTotalBefore );
1126
+
1127
+ membershipManager .onHeartbeatFailure (retriable );
1128
+
1129
+ final double failedRebalancesTotalAfter = (double ) metrics .metric (failedRebalanceTotalMetricName ).metricValue ();
1130
+ assertEquals (retriable ? 0L : 1L , failedRebalancesTotalAfter );
1131
+ }
1132
+
1078
1133
@ Test
1079
1134
public void testOnFencedWhenInJoining () {
1080
1135
joining ();
@@ -1292,6 +1347,65 @@ public void testOnTasksAssignedCallbackCompleted() {
1292
1347
assertFalse (future .isCompletedExceptionally ());
1293
1348
}
1294
1349
1350
+ @ Test
1351
+ public void testOnTasksAssignedCallbackCompletedWhenCallbackFails () {
1352
+ final String errorMessage = "KABOOM!" ;
1353
+ final CompletableFuture <Void > future = new CompletableFuture <>();
1354
+ final StreamsOnTasksAssignedCallbackCompletedEvent event = new StreamsOnTasksAssignedCallbackCompletedEvent (
1355
+ future ,
1356
+ Optional .of (new KafkaException (errorMessage ))
1357
+ );
1358
+
1359
+ membershipManager .onTasksAssignedCallbackCompleted (event );
1360
+
1361
+ assertTrue (future .isDone ());
1362
+ assertFalse (future .isCancelled ());
1363
+ assertTrue (future .isCompletedExceptionally ());
1364
+ final ExecutionException executionException = assertThrows (ExecutionException .class , future ::get );
1365
+ assertInstanceOf (KafkaException .class , executionException .getCause ());
1366
+ assertEquals (errorMessage , executionException .getCause ().getMessage ());
1367
+
1368
+ final SortedSet <StreamsRebalanceData .TaskId > activeTasksToAssign = new TreeSet <>();
1369
+ activeTasksToAssign .add (new StreamsRebalanceData .TaskId (SUB_TOPOLOGY_ID_0 , PARTITION_0 ));
1370
+ System .out .println (activeTasksToAssign .stream ()
1371
+ .map (StreamsRebalanceData .TaskId ::toString )
1372
+ .collect (Collectors .joining (", " )));
1373
+ }
1374
+
1375
+ @ Test
1376
+ public void testOnTasksRevokedCallbackCompleted () {
1377
+ final CompletableFuture <Void > future = new CompletableFuture <>();
1378
+ final StreamsOnTasksRevokedCallbackCompletedEvent event = new StreamsOnTasksRevokedCallbackCompletedEvent (
1379
+ future ,
1380
+ Optional .empty ()
1381
+ );
1382
+
1383
+ membershipManager .onTasksRevokedCallbackCompleted (event );
1384
+
1385
+ assertTrue (future .isDone ());
1386
+ assertFalse (future .isCancelled ());
1387
+ assertFalse (future .isCompletedExceptionally ());
1388
+ }
1389
+
1390
+ @ Test
1391
+ public void testOnTasksRevokedCallbackCompletedWhenCallbackFails () {
1392
+ final String errorMessage = "KABOOM!" ;
1393
+ final CompletableFuture <Void > future = new CompletableFuture <>();
1394
+ final StreamsOnTasksRevokedCallbackCompletedEvent event = new StreamsOnTasksRevokedCallbackCompletedEvent (
1395
+ future ,
1396
+ Optional .of (new KafkaException (errorMessage ))
1397
+ );
1398
+
1399
+ membershipManager .onTasksRevokedCallbackCompleted (event );
1400
+
1401
+ assertTrue (future .isDone ());
1402
+ assertFalse (future .isCancelled ());
1403
+ assertTrue (future .isCompletedExceptionally ());
1404
+ final ExecutionException executionException = assertThrows (ExecutionException .class , future ::get );
1405
+ assertInstanceOf (KafkaException .class , executionException .getCause ());
1406
+ assertEquals (errorMessage , executionException .getCause ().getMessage ());
1407
+ }
1408
+
1295
1409
@ Test
1296
1410
public void testOnAllTasksLostCallbackCompleted () {
1297
1411
final CompletableFuture <Void > future = new CompletableFuture <>();
@@ -1308,7 +1422,7 @@ public void testOnAllTasksLostCallbackCompleted() {
1308
1422
}
1309
1423
1310
1424
@ Test
1311
- public void testOnTasksAssignedCallbackCompletedWhenCallbackFails () {
1425
+ public void testOnAllTasksLostCallbackCompletedWhenCallbackFails () {
1312
1426
final String errorMessage = "KABOOM!" ;
1313
1427
final CompletableFuture <Void > future = new CompletableFuture <>();
1314
1428
final StreamsOnAllTasksLostCallbackCompletedEvent event = new StreamsOnAllTasksLostCallbackCompletedEvent (
@@ -1326,6 +1440,24 @@ public void testOnTasksAssignedCallbackCompletedWhenCallbackFails() {
1326
1440
assertEquals (errorMessage , executionException .getCause ().getMessage ());
1327
1441
}
1328
1442
1443
+ @ Test
1444
+ public void testMaybeRejoinStaleMember () {
1445
+ final CompletableFuture <Void > onAllTasksLostCallbackExecuted = new CompletableFuture <>();
1446
+ when (streamsRebalanceEventsProcessor .requestOnAllTasksLostCallbackInvocation ())
1447
+ .thenReturn (onAllTasksLostCallbackExecuted );
1448
+ joining ();
1449
+ membershipManager .onPollTimerExpired ();
1450
+ membershipManager .onHeartbeatRequestGenerated ();
1451
+ verifyInStateStale (membershipManager );
1452
+
1453
+ membershipManager .maybeRejoinStaleMember ();
1454
+
1455
+ verifyInStateStale (membershipManager );
1456
+ onAllTasksLostCallbackExecuted .complete (null );
1457
+ verifyInStateJoining (membershipManager );
1458
+ assertEquals (StreamsGroupHeartbeatRequest .JOIN_GROUP_MEMBER_EPOCH , membershipManager .memberEpoch ());
1459
+ }
1460
+
1329
1461
private void verifyThatNoTasksHaveBeenRevoked () {
1330
1462
verify (streamsRebalanceEventsProcessor , never ()).requestOnTasksRevokedCallbackInvocation (any ());
1331
1463
verify (subscriptionState , never ()).markPendingRevocation (any ());
0 commit comments