28
28
import org .apache .kafka .common .serialization .StringDeserializer ;
29
29
import org .apache .kafka .common .serialization .StringSerializer ;
30
30
import org .apache .kafka .common .utils .Utils ;
31
+ import org .apache .kafka .streams .GroupProtocol ;
31
32
import org .apache .kafka .streams .KafkaStreams ;
32
33
import org .apache .kafka .streams .KeyValue ;
33
34
import org .apache .kafka .streams .StreamsBuilder ;
50
51
import org .junit .jupiter .api .TestInfo ;
51
52
import org .junit .jupiter .api .Timeout ;
52
53
import org .junit .jupiter .params .ParameterizedTest ;
53
- import org .junit .jupiter .params .provider .ValueSource ;
54
+ import org .junit .jupiter .params .provider .Arguments ;
55
+ import org .junit .jupiter .params .provider .MethodSource ;
54
56
55
57
import java .io .File ;
56
58
import java .io .IOException ;
59
61
import java .util .Arrays ;
60
62
import java .util .Collections ;
61
63
import java .util .List ;
64
+ import java .util .Locale ;
62
65
import java .util .Objects ;
63
66
import java .util .Optional ;
64
67
import java .util .Properties ;
71
74
import java .util .regex .Pattern ;
72
75
import java .util .stream .Collectors ;
73
76
import java .util .stream .IntStream ;
77
+ import java .util .stream .Stream ;
74
78
75
79
import static org .apache .kafka .streams .KafkaStreams .State .ERROR ;
76
80
import static org .apache .kafka .streams .KafkaStreams .State .REBALANCING ;
@@ -121,7 +125,7 @@ public void before(final TestInfo testInfo) throws InterruptedException {
121
125
CLUSTER .createTopic (outputTopic , 1 , 1 );
122
126
}
123
127
124
- private Properties createStreamsConfig (final String topologyOptimization ) {
128
+ private Properties createStreamsConfig (final String topologyOptimization , final boolean useNewProtocol ) {
125
129
final Properties streamsConfiguration = new Properties ();
126
130
streamsConfiguration .put (StreamsConfig .APPLICATION_ID_CONFIG , applicationId );
127
131
streamsConfiguration .put (StreamsConfig .BOOTSTRAP_SERVERS_CONFIG , CLUSTER .bootstrapServers ());
@@ -131,9 +135,23 @@ private Properties createStreamsConfig(final String topologyOptimization) {
131
135
streamsConfiguration .put (StreamsConfig .DEFAULT_KEY_SERDE_CLASS_CONFIG , Serdes .Integer ().getClass ());
132
136
streamsConfiguration .put (StreamsConfig .DEFAULT_VALUE_SERDE_CLASS_CONFIG , Serdes .String ().getClass ());
133
137
streamsConfiguration .put (StreamsConfig .TOPOLOGY_OPTIMIZATION_CONFIG , topologyOptimization );
138
+
139
+ if (useNewProtocol ) {
140
+ streamsConfiguration .put (StreamsConfig .GROUP_PROTOCOL_CONFIG , GroupProtocol .STREAMS .name ().toLowerCase (Locale .getDefault ()));
141
+ }
142
+
134
143
return streamsConfiguration ;
135
144
}
136
145
146
+ private static Stream <Arguments > protocolAndOptimizationParameters () {
147
+ return Stream .of (
148
+ Arguments .of (StreamsConfig .OPTIMIZE , false ), // OPTIMIZE with CLASSIC protocol
149
+ Arguments .of (StreamsConfig .OPTIMIZE , true ), // OPTIMIZE with STREAMS protocol
150
+ Arguments .of (StreamsConfig .NO_OPTIMIZATION , false ), // NO_OPTIMIZATION with CLASSIC protocol
151
+ Arguments .of (StreamsConfig .NO_OPTIMIZATION , true ) // NO_OPTIMIZATION with STREAMS protocol
152
+ );
153
+ }
154
+
137
155
@ AfterEach
138
156
public void whenShuttingDown () throws IOException {
139
157
kafkaStreamsInstances .stream ()
@@ -144,8 +162,8 @@ public void whenShuttingDown() throws IOException {
144
162
}
145
163
146
164
@ ParameterizedTest
147
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
148
- public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining (final String topologyOptimization ) throws InterruptedException {
165
+ @ MethodSource ( "protocolAndOptimizationParameters" )
166
+ public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining (final String topologyOptimization , final boolean useNewProtocol ) throws InterruptedException {
149
167
final int topicBNumberOfPartitions = 6 ;
150
168
final String inputTopicRepartitionName = "join-repartition-test" ;
151
169
final AtomicReference <Throwable > expectedThrowable = new AtomicReference <>();
@@ -167,10 +185,12 @@ public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDo
167
185
.join (topicBStream , (value1 , value2 ) -> value2 , JoinWindows .ofTimeDifferenceWithNoGrace (Duration .ofSeconds (10 )))
168
186
.to (outputTopic );
169
187
170
- final Properties streamsConfiguration = createStreamsConfig (topologyOptimization );
188
+ final Properties streamsConfiguration = createStreamsConfig (topologyOptimization , useNewProtocol );
171
189
try (final KafkaStreams ks = new KafkaStreams (builder .build (streamsConfiguration ), streamsConfiguration )) {
172
190
ks .setUncaughtExceptionHandler (exception -> {
173
191
expectedThrowable .set (exception );
192
+ System .out .println (String .format ("[%s Protocol] Exception caught: %s" ,
193
+ useNewProtocol ? "STREAMS" : "CLASSIC" , exception .getMessage ()));
174
194
return SHUTDOWN_CLIENT ;
175
195
});
176
196
ks .start ();
@@ -186,8 +206,8 @@ public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDo
186
206
}
187
207
188
208
@ ParameterizedTest
189
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
190
- public void shouldDeductNumberOfPartitionsFromRepartitionOperation (final String topologyOptimization ) throws Exception {
209
+ @ MethodSource ( "protocolAndOptimizationParameters" )
210
+ public void shouldDeductNumberOfPartitionsFromRepartitionOperation (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
191
211
final String topicBMapperName = "topic-b-mapper" ;
192
212
final int topicBNumberOfPartitions = 6 ;
193
213
final String inputTopicRepartitionName = "join-repartition-test" ;
@@ -220,7 +240,7 @@ public void shouldDeductNumberOfPartitionsFromRepartitionOperation(final String
220
240
.join (topicBStream , (value1 , value2 ) -> value2 , JoinWindows .of (Duration .ofSeconds (10 )))
221
241
.to (outputTopic );
222
242
223
- final Properties streamsConfiguration = createStreamsConfig (topologyOptimization );
243
+ final Properties streamsConfiguration = createStreamsConfig (topologyOptimization , useNewProtocol );
224
244
builder .build (streamsConfiguration );
225
245
226
246
startStreams (builder , streamsConfiguration );
@@ -239,8 +259,8 @@ public void shouldDeductNumberOfPartitionsFromRepartitionOperation(final String
239
259
}
240
260
241
261
@ ParameterizedTest
242
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
243
- public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation (final String topologyOptimization ) throws Exception {
262
+ @ MethodSource ( "protocolAndOptimizationParameters" )
263
+ public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
244
264
final String topicBRepartitionedName = "topic-b-scale-up" ;
245
265
final String inputTopicRepartitionedName = "input-topic-scale-up" ;
246
266
@@ -278,7 +298,7 @@ public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartit
278
298
.join (topicBStream , (value1 , value2 ) -> value2 , JoinWindows .of (Duration .ofSeconds (10 )))
279
299
.to (outputTopic );
280
300
281
- startStreams (builder , createStreamsConfig (topologyOptimization ));
301
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
282
302
283
303
assertEquals (4 , getNumberOfPartitionsForTopic (toRepartitionTopicName (topicBRepartitionedName )));
284
304
assertEquals (4 , getNumberOfPartitionsForTopic (toRepartitionTopicName (inputTopicRepartitionedName )));
@@ -291,8 +311,8 @@ public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartit
291
311
}
292
312
293
313
@ ParameterizedTest
294
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
295
- public void shouldRepartitionToMultiplePartitions (final String topologyOptimization ) throws Exception {
314
+ @ MethodSource ( "protocolAndOptimizationParameters" )
315
+ public void shouldRepartitionToMultiplePartitions (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
296
316
final String repartitionName = "broadcasting-partitioner-test" ;
297
317
final long timestamp = System .currentTimeMillis ();
298
318
final AtomicInteger partitionerInvocation = new AtomicInteger (0 );
@@ -334,7 +354,7 @@ public Optional<Set<Integer>> partitions(final String topic, final Integer key,
334
354
.repartition (repartitioned )
335
355
.to (broadcastingOutputTopic );
336
356
337
- startStreams (builder , createStreamsConfig (topologyOptimization ));
357
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
338
358
339
359
final String topic = toRepartitionTopicName (repartitionName );
340
360
@@ -360,8 +380,8 @@ public Optional<Set<Integer>> partitions(final String topic, final Integer key,
360
380
361
381
362
382
@ ParameterizedTest
363
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
364
- public void shouldUseStreamPartitionerForRepartitionOperation (final String topologyOptimization ) throws Exception {
383
+ @ MethodSource ( "protocolAndOptimizationParameters" )
384
+ public void shouldUseStreamPartitionerForRepartitionOperation (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
365
385
final int partition = 1 ;
366
386
final String repartitionName = "partitioner-test" ;
367
387
final long timestamp = System .currentTimeMillis ();
@@ -387,7 +407,7 @@ public void shouldUseStreamPartitionerForRepartitionOperation(final String topol
387
407
.repartition (repartitioned )
388
408
.to (outputTopic );
389
409
390
- startStreams (builder , createStreamsConfig (topologyOptimization ));
410
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
391
411
392
412
final String topic = toRepartitionTopicName (repartitionName );
393
413
@@ -402,8 +422,8 @@ public void shouldUseStreamPartitionerForRepartitionOperation(final String topol
402
422
}
403
423
404
424
@ ParameterizedTest
405
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
406
- public void shouldPerformSelectKeyWithRepartitionOperation (final String topologyOptimization ) throws Exception {
425
+ @ MethodSource ( "protocolAndOptimizationParameters" )
426
+ public void shouldPerformSelectKeyWithRepartitionOperation (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
407
427
final long timestamp = System .currentTimeMillis ();
408
428
409
429
sendEvents (
@@ -421,7 +441,7 @@ public void shouldPerformSelectKeyWithRepartitionOperation(final String topology
421
441
.repartition ()
422
442
.to (outputTopic );
423
443
424
- startStreams (builder , createStreamsConfig (topologyOptimization ));
444
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
425
445
426
446
validateReceivedMessages (
427
447
new IntegerDeserializer (),
@@ -438,8 +458,8 @@ public void shouldPerformSelectKeyWithRepartitionOperation(final String topology
438
458
}
439
459
440
460
@ ParameterizedTest
441
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
442
- public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed (final String topologyOptimization ) throws Exception {
461
+ @ MethodSource ( "protocolAndOptimizationParameters" )
462
+ public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
443
463
final String repartitionName = "dummy" ;
444
464
final long timestamp = System .currentTimeMillis ();
445
465
@@ -457,7 +477,7 @@ public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed(fi
457
477
.repartition (Repartitioned .as (repartitionName ))
458
478
.to (outputTopic );
459
479
460
- startStreams (builder , createStreamsConfig (topologyOptimization ));
480
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
461
481
462
482
validateReceivedMessages (
463
483
new IntegerDeserializer (),
@@ -475,8 +495,8 @@ public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed(fi
475
495
}
476
496
477
497
@ ParameterizedTest
478
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
479
- public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector (final String topologyOptimization ) throws Exception {
498
+ @ MethodSource ( "protocolAndOptimizationParameters" )
499
+ public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
480
500
final String repartitionedName = "new-key" ;
481
501
final long timestamp = System .currentTimeMillis ();
482
502
@@ -501,7 +521,7 @@ public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKey
501
521
.toStream ()
502
522
.to (outputTopic );
503
523
504
- startStreams (builder , createStreamsConfig (topologyOptimization ));
524
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
505
525
506
526
validateReceivedMessages (
507
527
new StringDeserializer (),
@@ -521,8 +541,8 @@ public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKey
521
541
}
522
542
523
543
@ ParameterizedTest
524
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
525
- public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions (final String topologyOptimization ) throws Exception {
544
+ @ MethodSource ( "protocolAndOptimizationParameters" )
545
+ public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
526
546
final String repartitionName = "new-partitions" ;
527
547
final long timestamp = System .currentTimeMillis ();
528
548
@@ -543,7 +563,7 @@ public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions(final St
543
563
.toStream ()
544
564
.to (outputTopic );
545
565
546
- startStreams (builder , createStreamsConfig (topologyOptimization ));
566
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
547
567
548
568
validateReceivedMessages (
549
569
new IntegerDeserializer (),
@@ -561,8 +581,8 @@ public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions(final St
561
581
}
562
582
563
583
@ ParameterizedTest
564
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
565
- public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified (final String topologyOptimization ) throws Exception {
584
+ @ MethodSource ( "protocolAndOptimizationParameters" )
585
+ public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
566
586
final String repartitionName = "new-topic" ;
567
587
final long timestamp = System .currentTimeMillis ();
568
588
@@ -583,7 +603,7 @@ public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNum
583
603
.toStream ()
584
604
.to (outputTopic );
585
605
586
- startStreams (builder , createStreamsConfig (topologyOptimization ));
606
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
587
607
588
608
validateReceivedMessages (
589
609
new IntegerDeserializer (),
@@ -601,8 +621,8 @@ public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNum
601
621
}
602
622
603
623
@ ParameterizedTest
604
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
605
- public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey (final String topologyOptimization ) throws Exception {
624
+ @ MethodSource ( "protocolAndOptimizationParameters" )
625
+ public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
606
626
final String repartitionName = "new-partitions" ;
607
627
final long timestamp = System .currentTimeMillis ();
608
628
@@ -629,7 +649,7 @@ public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupB
629
649
.toStream ()
630
650
.to (outputTopic );
631
651
632
- startStreams (builder , createStreamsConfig (topologyOptimization ));
652
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
633
653
634
654
final String topology = builder .build ().describe ().toString ();
635
655
@@ -647,8 +667,8 @@ public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupB
647
667
}
648
668
649
669
@ ParameterizedTest
650
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
651
- public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified (final String topologyOptimization ) throws Exception {
670
+ @ MethodSource ( "protocolAndOptimizationParameters" )
671
+ public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
652
672
final long timestamp = System .currentTimeMillis ();
653
673
654
674
sendEvents (
@@ -666,7 +686,7 @@ public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified(final String to
666
686
.repartition (Repartitioned .with (Serdes .String (), Serdes .String ()))
667
687
.to (outputTopic );
668
688
669
- startStreams (builder , createStreamsConfig (topologyOptimization ));
689
+ startStreams (builder , createStreamsConfig (topologyOptimization , useNewProtocol ));
670
690
671
691
validateReceivedMessages (
672
692
new StringDeserializer (),
@@ -683,8 +703,8 @@ public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified(final String to
683
703
}
684
704
685
705
@ ParameterizedTest
686
- @ ValueSource ( strings = { StreamsConfig . OPTIMIZE , StreamsConfig . NO_OPTIMIZATION } )
687
- public void shouldGoThroughRebalancingCorrectly (final String topologyOptimization ) throws Exception {
706
+ @ MethodSource ( "protocolAndOptimizationParameters" )
707
+ public void shouldGoThroughRebalancingCorrectly (final String topologyOptimization , final boolean useNewProtocol ) throws Exception {
688
708
final String repartitionName = "rebalancing-test" ;
689
709
final long timestamp = System .currentTimeMillis ();
690
710
@@ -711,7 +731,7 @@ public void shouldGoThroughRebalancingCorrectly(final String topologyOptimizatio
711
731
.toStream ()
712
732
.to (outputTopic );
713
733
714
- final Properties streamsConfiguration = createStreamsConfig (topologyOptimization );
734
+ final Properties streamsConfiguration = createStreamsConfig (topologyOptimization , useNewProtocol );
715
735
startStreams (builder , streamsConfiguration );
716
736
final Properties streamsToCloseConfigs = new Properties ();
717
737
streamsToCloseConfigs .putAll (streamsConfiguration );
0 commit comments