Skip to content

Commit ada336f

Browse files
committed
Part 3 of 4 converting streams system tests to KRaft
1 parent e27d0df commit ada336f

File tree

4 files changed

+18
-82
lines changed

4 files changed

+18
-82
lines changed

tests/kafkatest/tests/streams/streams_broker_compatibility_test.py

Lines changed: 12 additions & 66 deletions
Original file line numberDiff line numberDiff line change
@@ -18,34 +18,27 @@
1818
from ducktape.mark.resource import cluster
1919
from ducktape.tests.test import Test
2020
from ducktape.utils.util import wait_until
21-
from kafkatest.services.kafka import KafkaService
21+
from kafkatest.services.kafka import KafkaService, quorum
2222
from kafkatest.services.streams import StreamsBrokerCompatibilityService
2323
from kafkatest.services.verifiable_consumer import VerifiableConsumer
24-
from kafkatest.services.zookeeper import ZookeeperService
25-
from kafkatest.version import LATEST_0_11_0, LATEST_0_10_2, LATEST_0_10_1, LATEST_0_10_0, LATEST_1_0, LATEST_1_1, \
26-
LATEST_2_0, LATEST_2_1, LATEST_2_2, LATEST_2_3, LATEST_2_4, LATEST_2_5, LATEST_2_6, LATEST_2_7, LATEST_2_8, \
24+
from kafkatest.version import LATEST_2_8, \
2725
LATEST_3_0, LATEST_3_1, LATEST_3_2, LATEST_3_3, LATEST_3_4, LATEST_3_5, LATEST_3_6, LATEST_3_7, LATEST_3_8, KafkaVersion
2826

2927

3028
class StreamsBrokerCompatibility(Test):
3129
"""
3230
These tests validates that
33-
- Streams works for older brokers 0.11 (or newer)
34-
- Streams w/ EOS-alpha works for older brokers 0.11 (or newer)
3531
- Streams w/ EOS-v2 works for older brokers 2.5 (or newer)
36-
- Streams fails fast for older brokers 0.10.0, 0.10.2, and 0.10.1
37-
- Streams w/ EOS-v2 fails fast for older brokers 2.4 or older
3832
"""
3933

4034
input = "brokerCompatibilitySourceTopic"
4135
output = "brokerCompatibilitySinkTopic"
4236

4337
def __init__(self, test_context):
4438
super(StreamsBrokerCompatibility, self).__init__(test_context=test_context)
45-
self.zk = ZookeeperService(test_context, num_nodes=1)
4639
self.kafka = KafkaService(test_context,
4740
num_nodes=1,
48-
zk=self.zk,
41+
zk=None,
4942
topics={
5043
self.input: {'partitions': 1, 'replication-factor': 1},
5144
self.output: {'partitions': 1, 'replication-factor': 1}
@@ -60,18 +53,15 @@ def __init__(self, test_context):
6053
self.output,
6154
"stream-broker-compatibility-verify-consumer")
6255

63-
def setUp(self):
64-
self.zk.start()
65-
6656

6757
@cluster(num_nodes=4)
68-
@matrix(broker_version=[str(LATEST_0_11_0),str(LATEST_1_0),str(LATEST_1_1),str(LATEST_2_0),
69-
str(LATEST_2_1),str(LATEST_2_2),str(LATEST_2_3),str(LATEST_2_4),
70-
str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8),
58+
@matrix(broker_version=[str(LATEST_2_8),
7159
str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3),
7260
str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7),
73-
str(LATEST_3_8)])
74-
def test_compatible_brokers_eos_disabled(self, broker_version):
61+
str(LATEST_3_8)],
62+
metadata_quorum=[quorum.combined_kraft]
63+
)
64+
def test_compatible_brokers_eos_disabled(self, broker_version, metadata_quorum):
7565
self.kafka.set_version(KafkaVersion(broker_version))
7666
self.kafka.start()
7767

@@ -88,11 +78,12 @@ def test_compatible_brokers_eos_disabled(self, broker_version):
8878
self.kafka.stop()
8979

9080
@cluster(num_nodes=4)
91-
@matrix(broker_version=[str(LATEST_2_5),str(LATEST_2_6),str(LATEST_2_7),str(LATEST_2_8),
81+
@matrix(broker_version=[str(LATEST_2_8),
9282
str(LATEST_3_0),str(LATEST_3_1),str(LATEST_3_2),str(LATEST_3_3),
9383
str(LATEST_3_4),str(LATEST_3_5),str(LATEST_3_6),str(LATEST_3_7),
94-
str(LATEST_3_8)])
95-
def test_compatible_brokers_eos_v2_enabled(self, broker_version):
84+
str(LATEST_3_8)],
85+
metadata_quorum=[quorum.combined_kraft])
86+
def test_compatible_brokers_eos_v2_enabled(self, broker_version, metadata_quorum):
9687
self.kafka.set_version(KafkaVersion(broker_version))
9788
self.kafka.start()
9889

@@ -107,48 +98,3 @@ def test_compatible_brokers_eos_v2_enabled(self, broker_version):
10798

10899
self.consumer.stop()
109100
self.kafka.stop()
110-
111-
@cluster(num_nodes=4)
112-
@parametrize(broker_version=str(LATEST_0_10_2))
113-
@parametrize(broker_version=str(LATEST_0_10_1))
114-
@parametrize(broker_version=str(LATEST_0_10_0))
115-
def test_fail_fast_on_incompatible_brokers(self, broker_version):
116-
self.kafka.set_version(KafkaVersion(broker_version))
117-
self.kafka.start()
118-
119-
processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, "at_least_once")
120-
121-
with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor:
122-
processor.start()
123-
monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException',
124-
timeout_sec=60,
125-
err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException " + str(processor.node.account))
126-
127-
self.kafka.stop()
128-
129-
@cluster(num_nodes=4)
130-
@parametrize(broker_version=str(LATEST_2_4))
131-
@parametrize(broker_version=str(LATEST_2_3))
132-
@parametrize(broker_version=str(LATEST_2_2))
133-
@parametrize(broker_version=str(LATEST_2_1))
134-
@parametrize(broker_version=str(LATEST_2_0))
135-
@parametrize(broker_version=str(LATEST_1_1))
136-
@parametrize(broker_version=str(LATEST_1_0))
137-
@parametrize(broker_version=str(LATEST_0_11_0))
138-
def test_fail_fast_on_incompatible_brokers_if_eos_v2_enabled(self, broker_version):
139-
self.kafka.set_version(KafkaVersion(broker_version))
140-
self.kafka.start()
141-
142-
processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka, "exactly_once_v2")
143-
144-
with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor:
145-
with processor.node.account.monitor_log(processor.LOG_FILE) as log:
146-
processor.start()
147-
log.wait_until('Shutting down because the Kafka cluster seems to be on a too old version. Setting processing\.guarantee="exactly_once_v2" requires broker version 2\.5 or higher\.',
148-
timeout_sec=60,
149-
err_msg="Never saw 'Shutting down because the Kafka cluster seems to be on a too old version. Setting `processing.guarantee=\"exactly_once_v2\"` requires broker version 2.5 or higher.' log message " + str(processor.node.account))
150-
monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException',
151-
timeout_sec=60,
152-
err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException' error message " + str(processor.node.account))
153-
154-
self.kafka.stop()

tests/kafkatest/tests/streams/streams_optimized_test.py

Lines changed: 3 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121
from kafkatest.services.streams import StreamsOptimizedUpgradeTestService
2222
from kafkatest.services.streams import StreamsResetter
2323
from kafkatest.services.verifiable_producer import VerifiableProducer
24-
from kafkatest.services.zookeeper import ZookeeperService
2524
from kafkatest.tests.streams.utils import stop_processors
2625

2726
class StreamsOptimizedTest(Test):
@@ -46,13 +45,8 @@ def __init__(self, test_context):
4645
self.join_topic: {'partitions': 6}
4746
}
4847

49-
self.zookeeper = (
50-
ZookeeperService(self.test_context, 1)
51-
if quorum.for_test(self.test_context) == quorum.zk
52-
else None
53-
)
54-
self.kafka = KafkaService(self.test_context, num_nodes=3,
55-
zk=self.zookeeper, topics=self.topics, controller_num_nodes_override=1)
48+
self.kafka = KafkaService(self.test_context, num_nodes=3, controller_num_nodes_override=1,
49+
zk=None, topics=self.topics)
5650

5751
self.producer = VerifiableProducer(self.test_context,
5852
1,
@@ -62,10 +56,8 @@ def __init__(self, test_context):
6256
acks=1)
6357

6458
@cluster(num_nodes=9)
65-
@matrix(metadata_quorum=[quorum.isolated_kraft])
59+
@matrix(metadata_quorum=[quorum.combined_kraft])
6660
def test_upgrade_optimized_topology(self, metadata_quorum):
67-
if self.zookeeper:
68-
self.zookeeper.start()
6961
self.kafka.start()
7062

7163
processor1 = StreamsOptimizedUpgradeTestService(self.test_context, self.kafka)
@@ -111,8 +103,6 @@ def test_upgrade_optimized_topology(self, metadata_quorum):
111103
self.logger.info("teardown")
112104
self.producer.stop()
113105
self.kafka.stop()
114-
if self.zookeeper:
115-
self.zookeeper.stop()
116106

117107
def reset_application(self):
118108
resetter = StreamsResetter(self.test_context, self.kafka, topic = self.input_topic, applicationId = 'StreamsOptimizedTest')

tests/kafkatest/tests/streams/streams_relational_smoke_test.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -86,8 +86,8 @@ def __init__(self, test_context):
8686

8787
@cluster(num_nodes=8)
8888
@matrix(crash=[False, True],
89-
processing_guarantee=['exactly_once', 'exactly_once_v2'],
90-
metadata_quorum=[quorum.isolated_kraft])
89+
processing_guarantee=['exactly_once_v2'],
90+
metadata_quorum=[quorum.combined_kraft])
9191
def test_streams(self, crash, processing_guarantee, metadata_quorum):
9292
driver = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "driver", "ignored", "ignored")
9393

tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ def __init__(self, test_context):
3333
self.driver = StreamsSmokeTestShutdownDeadlockService(test_context, self.kafka)
3434

3535
@cluster(num_nodes=3)
36-
@matrix(metadata_quorum=[quorum.isolated_kraft])
36+
@matrix(metadata_quorum=[quorum.combined_kraft])
3737
def test_shutdown_wont_deadlock(self, metadata_quorum):
3838
"""
3939
Start ShutdownDeadLockTest, wait for upt to 1 minute, and check that the process exited.

0 commit comments

Comments
 (0)