KAFKA-19202: Enable KIP-1071 in streams_smoke_test.py (#19560)
CI / build (push) Waiting to run Details

Enables KIP-1071 (`group.protocol=streams`) in the first streams  system
test `streams_smoke_test.py`.

All tests using KIP-1071 cannot use `KafkaTest` anymore, since  we need
to customize the broker configuration. The corresponding  functionality
is added to `BaseStreamsTest`, which all streams  tests will have to
extend from now on.

There are some left-overs from ZK in the tests that I copied   from
'KafkaTest'. They need to be cleaned up, but this should  be done in a
separate PR.
This commit is contained in:
Lucas Brutschy 2025-04-29 13:35:19 +02:00 committed by GitHub
parent 019459e950
commit 5c63b4569b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 53 additions and 19 deletions

View File

@ -81,6 +81,8 @@ SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR ="share.coordinator.state.topic
SHARE_COORDINATOR_STATE_TOPIC_MIN_ISR = "share.coordinator.state.topic.min.isr" SHARE_COORDINATOR_STATE_TOPIC_MIN_ISR = "share.coordinator.state.topic.min.isr"
SHARE_GROUP_ENABLE = "group.share.enable" SHARE_GROUP_ENABLE = "group.share.enable"
UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable"
""" """
From KafkaConfig.scala From KafkaConfig.scala

View File

@ -206,7 +206,8 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
consumer_group_migration_policy=None, consumer_group_migration_policy=None,
dynamicRaftQuorum=False, dynamicRaftQuorum=False,
use_transactions_v2=False, use_transactions_v2=False,
use_share_groups=None use_share_groups=None,
use_streams_groups=False
): ):
""" """
:param context: test context :param context: test context
@ -271,6 +272,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
:param dynamicRaftQuorum: When true, controller_quorum_bootstrap_servers, and bootstraps the first controller using the standalone flag :param dynamicRaftQuorum: When true, controller_quorum_bootstrap_servers, and bootstraps the first controller using the standalone flag
:param use_transactions_v2: When true, uses transaction.version=2 which utilizes the new transaction protocol introduced in KIP-890 :param use_transactions_v2: When true, uses transaction.version=2 which utilizes the new transaction protocol introduced in KIP-890
:param use_share_groups: When true, enables the use of share groups introduced in KIP-932 :param use_share_groups: When true, enables the use of share groups introduced in KIP-932
:param use_streams_groups: When true, enables the use of streams groups introduced in KIP-1071
""" """
self.zk = zk self.zk = zk
@ -296,6 +298,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
# Assign the determined value. # Assign the determined value.
self.use_transactions_v2 = use_transactions_v2 self.use_transactions_v2 = use_transactions_v2
self.use_share_groups = use_share_groups self.use_share_groups = use_share_groups
self.use_streams_groups = use_streams_groups
# Set consumer_group_migration_policy based on context and arguments. # Set consumer_group_migration_policy based on context and arguments.
if consumer_group_migration_policy is None: if consumer_group_migration_policy is None:
@ -776,9 +779,17 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
for prop in self.per_node_server_prop_overrides.get(self.idx(node), []): for prop in self.per_node_server_prop_overrides.get(self.idx(node), []):
override_configs[prop[0]] = prop[1] override_configs[prop[0]] = prop[1]
enabledProtocols = 'classic,consumer'
if self.use_share_groups is not None and self.use_share_groups is True: if self.use_share_groups is not None and self.use_share_groups is True:
override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups) override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups)
override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer,share' enabledProtocols += ',share'
if self.use_streams_groups is True:
override_configs[config_property.UNSTABLE_API_VERSIONS_ENABLE] = str(True)
enabledProtocols += ',streams'
override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = enabledProtocols
#update template configs with test override configs #update template configs with test override configs
configs.update(override_configs) configs.update(override_configs)

View File

@ -320,13 +320,14 @@ class StreamsTestBaseService(KafkaPathResolverMixin, JmxMixin, Service):
class StreamsSmokeTestBaseService(StreamsTestBaseService): class StreamsSmokeTestBaseService(StreamsTestBaseService):
"""Base class for Streams Smoke Test services providing some common settings and functionality""" """Base class for Streams Smoke Test services providing some common settings and functionality"""
def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', num_threads = 3, replication_factor = 3): def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', group_protocol = 'classic', num_threads = 3, replication_factor = 3):
super(StreamsSmokeTestBaseService, self).__init__(test_context, super(StreamsSmokeTestBaseService, self).__init__(test_context,
kafka, kafka,
"org.apache.kafka.streams.tests.StreamsSmokeTest", "org.apache.kafka.streams.tests.StreamsSmokeTest",
command) command)
self.NUM_THREADS = num_threads self.NUM_THREADS = num_threads
self.PROCESSING_GUARANTEE = processing_guarantee self.PROCESSING_GUARANTEE = processing_guarantee
self.GROUP_PROTOCOL = group_protocol
self.KAFKA_STREAMS_VERSION = "" self.KAFKA_STREAMS_VERSION = ""
self.UPGRADE_FROM = None self.UPGRADE_FROM = None
self.REPLICATION_FACTOR = replication_factor self.REPLICATION_FACTOR = replication_factor
@ -341,6 +342,7 @@ class StreamsSmokeTestBaseService(StreamsTestBaseService):
properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT, properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT,
streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(), streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(),
streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE, streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE,
streams_property.GROUP_PROTOCOL: self.GROUP_PROTOCOL,
streams_property.NUM_THREADS: self.NUM_THREADS, streams_property.NUM_THREADS: self.NUM_THREADS,
"replication.factor": self.REPLICATION_FACTOR, "replication.factor": self.REPLICATION_FACTOR,
"num.standby.replicas": 2, "num.standby.replicas": 2,
@ -436,8 +438,8 @@ class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService):
return cmd return cmd
class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService):
def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3, replication_factor = 3): def __init__(self, test_context, kafka, processing_guarantee, group_protocol = 'classic', num_threads = 3, replication_factor = 3):
super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, num_threads, replication_factor) super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, group_protocol, num_threads, replication_factor)
class StreamsEosTestDriverService(StreamsEosTestBaseService): class StreamsEosTestDriverService(StreamsEosTestBaseService):
def __init__(self, test_context, kafka): def __init__(self, test_context, kafka):

View File

@ -21,3 +21,4 @@ STATE_DIR = "state.dir"
KAFKA_SERVERS = "bootstrap.servers" KAFKA_SERVERS = "bootstrap.servers"
NUM_THREADS = "num.stream.threads" NUM_THREADS = "num.stream.threads"
PROCESSING_GUARANTEE = "processing.guarantee" PROCESSING_GUARANTEE = "processing.guarantee"
GROUP_PROTOCOL = "group.protocol"

View File

@ -13,13 +13,14 @@
# See the License for the specific language governing permissions and # See the License for the specific language governing permissions and
# limitations under the License. # limitations under the License.
from ducktape.tests.test import Test
from ducktape.utils.util import wait_until from ducktape.utils.util import wait_until
from kafkatest.services.verifiable_consumer import VerifiableConsumer from kafkatest.services.verifiable_consumer import VerifiableConsumer
from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.services.verifiable_producer import VerifiableProducer
from kafkatest.tests.kafka_test import KafkaTest from kafkatest.services.kafka import KafkaService
class BaseStreamsTest(KafkaTest): class BaseStreamsTest(Test):
""" """
Helper class that contains methods for producing and consuming Helper class that contains methods for producing and consuming
messages and verification of results from log files messages and verification of results from log files
@ -27,8 +28,24 @@ class BaseStreamsTest(KafkaTest):
Extends KafkaTest which manages setting up Kafka Cluster and Zookeeper Extends KafkaTest which manages setting up Kafka Cluster and Zookeeper
see tests/kafkatest/tests/kafka_test.py for more info see tests/kafkatest/tests/kafka_test.py for more info
""" """
def __init__(self, test_context, topics, num_controllers=1, num_brokers=3): def __init__(self, test_context, topics, num_controllers=1, num_brokers=3):
super(BaseStreamsTest, self).__init__(test_context, num_controllers, num_brokers, topics) self.num_controllers = num_controllers
self.num_brokers = num_brokers
self.topics = topics
self.kafka = KafkaService(
test_context, self.num_brokers,
None, topics=self.topics,
controller_num_nodes_override=self.num_controllers,
use_streams_groups=True,
server_prop_overrides=[
[ "group.streams.min.session.timeout.ms", "10000" ], # Need to up the lower bound
[ "group.streams.session.timeout.ms", "10000" ] # As in classic groups, set this to 10s
]
)
def setUp(self):
self.kafka.start()
def get_consumer(self, client_id, topic, num_messages): def get_consumer(self, client_id, topic, num_messages):
return VerifiableConsumer(self.test_context, return VerifiableConsumer(self.test_context,

View File

@ -41,8 +41,8 @@ class StreamsBrokerDownResilience(BaseStreamsTest):
num_brokers=1) num_brokers=1)
def setUp(self): def setUp(self):
if self.zk: # do not start kafka
self.zk.start() pass
@cluster(num_nodes=7) @cluster(num_nodes=7)
@matrix(metadata_quorum=[quorum.combined_kraft]) @matrix(metadata_quorum=[quorum.combined_kraft])

View File

@ -17,16 +17,16 @@ from ducktape.mark import matrix
from ducktape.mark.resource import cluster from ducktape.mark.resource import cluster
from kafkatest.services.kafka import quorum from kafkatest.services.kafka import quorum
from kafkatest.tests.kafka_test import KafkaTest
from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService
from kafkatest.tests.streams.base_streams_test import BaseStreamsTest
class StreamsSmokeTest(KafkaTest): class StreamsSmokeTest(BaseStreamsTest):
""" """
Simple test of Kafka Streams. Simple test of Kafka Streams.
""" """
def __init__(self, test_context): def __init__(self, test_context):
super(StreamsSmokeTest, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ super(StreamsSmokeTest, self).__init__(test_context, topics={
'echo' : { 'partitions': 5, 'replication-factor': 1 }, 'echo' : { 'partitions': 5, 'replication-factor': 1 },
'data' : { 'partitions': 5, 'replication-factor': 1 }, 'data' : { 'partitions': 5, 'replication-factor': 1 },
'min' : { 'partitions': 5, 'replication-factor': 1 }, 'min' : { 'partitions': 5, 'replication-factor': 1 },
@ -49,11 +49,12 @@ class StreamsSmokeTest(KafkaTest):
@cluster(num_nodes=8) @cluster(num_nodes=8)
@matrix(processing_guarantee=['exactly_once_v2', 'at_least_once'], @matrix(processing_guarantee=['exactly_once_v2', 'at_least_once'],
crash=[True, False], crash=[True, False],
metadata_quorum=[quorum.combined_kraft]) metadata_quorum=[quorum.combined_kraft],
def test_streams(self, processing_guarantee, crash, metadata_quorum): group_protocol=["classic", "streams"])
processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee) def test_streams(self, processing_guarantee, crash, metadata_quorum, group_protocol):
processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee) processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee) processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
with processor1.node.account.monitor_log(processor1.STDOUT_FILE) as monitor1: with processor1.node.account.monitor_log(processor1.STDOUT_FILE) as monitor1:
processor1.start() processor1.start()