From f99cd95d76c73d5a49049415129079999d54f5b8 Mon Sep 17 00:00:00 2001 From: Pranav Rathi <4427674+pranavrth@users.noreply.github.com> Date: Thu, 17 Jul 2025 12:35:57 +0530 Subject: [PATCH] Added online upgrade and downgrade test --- tests/common/__init__.py | 25 ++-- tests/integration/cluster_fixture.py | 15 +- .../test_consumer_upgrade_downgrade.py | 131 ++++++++++++++++++ 3 files changed, 159 insertions(+), 12 deletions(-) create mode 100644 tests/integration/consumer/test_consumer_upgrade_downgrade.py diff --git a/tests/common/__init__.py b/tests/common/__init__.py index 3d9ec5c7a..5834604c1 100644 --- a/tests/common/__init__.py +++ b/tests/common/__init__.py @@ -55,22 +55,25 @@ def use_group_protocol_consumer(): @staticmethod def update_conf_group_protocol(conf=None): - if conf is not None and 'group.id' in conf and TestUtils.use_group_protocol_consumer(): + if TestUtils.can_upgrade_group_protocol_to_consumer(conf): conf['group.protocol'] = 'consumer' + @staticmethod + def can_upgrade_group_protocol_to_consumer(conf): + return conf is not None and 'group.id' in conf and 'group.protocol' not in conf and TestUtils.use_group_protocol_consumer() + @staticmethod def remove_forbidden_conf_group_protocol_consumer(conf): - if conf is None: + if conf is None or not TestUtils.use_group_protocol_consumer() or conf.get('group.protocol', 'consumer') != 'consumer': return - if TestUtils.use_group_protocol_consumer(): - forbidden_conf_properties = ["session.timeout.ms", - "partition.assignment.strategy", - "heartbeat.interval.ms", - "group.protocol.type"] - for prop in forbidden_conf_properties: - if prop in conf: - print("Skipping setting forbidden configuration {prop} for `CONSUMER` protocol") - del conf[prop] + forbidden_conf_properties = ["session.timeout.ms", + "partition.assignment.strategy", + "heartbeat.interval.ms", + "group.protocol.type"] + for prop in forbidden_conf_properties: + if prop in conf: + print(f"Skipping setting forbidden configuration {prop} for `CONSUMER` protocol") + del conf[prop] class TestConsumer(Consumer): diff --git a/tests/integration/cluster_fixture.py b/tests/integration/cluster_fixture.py index 0d441ca1d..e6bf06a84 100644 --- a/tests/integration/cluster_fixture.py +++ b/tests/integration/cluster_fixture.py @@ -233,6 +233,19 @@ def create_topic(self, prefix, conf=None, **create_topic_kwargs): future_topic.get(name).result() return name + + def delete_topic(self, topic): + """ + Deletes a topic with this cluster. + + :param str topic: topic name + """ + future = self.admin().delete_topics([topic]) + try: + future.get(topic).result() + print("Topic {} deleted".format(topic)) + except Exception as e: + print("Failed to delete topic {}: {}".format(topic, e)) def create_topic_and_wait_propogation(self, prefix, conf=None, **create_topic_kwargs): """ @@ -273,7 +286,7 @@ def seed_topic(self, topic, value_source=None, key_source=None, header_source=No value_source = ['test-data{}'.format(i) for i in range(0, 100)] if key_source is None: - key_source = [None] + key_source = ['test-key{}'.format(i) for i in range(0, 100)] KafkaClusterFixture._produce(self._producer, topic, value_source, key_source, header_source) diff --git a/tests/integration/consumer/test_consumer_upgrade_downgrade.py b/tests/integration/consumer/test_consumer_upgrade_downgrade.py new file mode 100644 index 000000000..4397e3c4f --- /dev/null +++ b/tests/integration/consumer/test_consumer_upgrade_downgrade.py @@ -0,0 +1,131 @@ +#!/usr/bin/env python +# -*- coding: utf-8 -*- +# +# Copyright 2025 Confluent Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limit + +import pytest +from confluent_kafka import ConsumerGroupType, IsolationLevel, KafkaException, TopicPartition +from confluent_kafka.admin import OffsetSpec +from tests.common import TestUtils + +topic_prefix = "test_consumer_upgrade_downgrade_" +number_of_partitions = 10 + + +def get_group_protocol_type(a, group_id): + futureMap = a.describe_consumer_groups([group_id]) + try: + future = futureMap[group_id] + g = future.result() + return g.type + except KafkaException as e: + print("Error while describing group id '{}': {}".format(group_id, e)) + except Exception: + raise + + +def list_offsets(a, topic, no_of_partitions): + topic_partition_offsets = {} + for partition in range(no_of_partitions): + topic_partition = TopicPartition(topic, partition) + topic_partition_offsets[topic_partition] = OffsetSpec.latest() + futmap = a.list_offsets(topic_partition_offsets, isolation_level=IsolationLevel.READ_COMMITTED, request_timeout=30) + for partition, fut in futmap.items(): + try: + result = fut.result() + print("Topicname : {} Partition_Index : {} Offset : {} Timestamp : {}" + .format(partition.topic, partition.partition, result.offset, + result.timestamp)) + except KafkaException as e: + print("Topicname : {} Partition_Index : {} Error : {}" + .format(partition.topic, partition.partition, e)) + + +# def produce_messages(producer, topic, partitions, num_messages): +# for i in range(num_messages): +# key = "key-{}".format(i) +# value = "value-{}".format(i) +# partition = i % partitions +# producer.produce(topic, key=key, value=value, partition=partition) +# producer.flush() + + +def check_consumer(kafka_cluster, consumers, admin_client, topic, expected_protocol): + total_msg_read = 0 + while len(consumers[-1].assignment()) != number_of_partitions // len(consumers): + for consumer in consumers: + consumer.poll(0.1) + + for consumer in consumers: + assert len(consumer.assignment()) == number_of_partitions // len(consumers) + + assert get_group_protocol_type(admin_client, topic) == expected_protocol + + # Produce some messages to the topic + kafka_cluster.seed_topic(topic) + list_offsets(admin_client, topic, number_of_partitions) + + while total_msg_read < 100: + for consumer in consumers: + # Poll for messages + msg = consumer.poll(0.1) + if msg is not None: + total_msg_read += 1 + + assert total_msg_read == 100, "Expected to read 100 messages, but read {}".format(total_msg_read) + + +def perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, partition_assignment_strategy): + """ + Test consumer upgrade and downgrade. + """ + topic = kafka_cluster.create_topic_and_wait_propogation(topic_prefix, + { + "num_partitions": number_of_partitions + }) + admin_client = kafka_cluster.admin() + + # Create a consumer with the latest version + consumer_conf = {'group.id': topic, + 'auto.offset.reset': 'earliest', + 'group.protocol': 'classic'} + consumer_conf['partition.assignment.strategy'] = partition_assignment_strategy + consumer = kafka_cluster.consumer(consumer_conf) + assert consumer is not None + consumer.subscribe([topic]) + check_consumer(kafka_cluster, [consumer], admin_client, topic, ConsumerGroupType.CLASSIC) + del consumer_conf['partition.assignment.strategy'] + + # Now simulate an upgrade by creating a new consumer with 'consumer' protocol + consumer_conf['group.protocol'] = 'consumer' + consumer2 = kafka_cluster.consumer(consumer_conf) + assert consumer2 is not None + consumer2.subscribe([topic]) + check_consumer(kafka_cluster, [consumer, consumer2], admin_client, topic, ConsumerGroupType.CONSUMER) + + # Now simulate a downgrade by deleting the second consumer and keeping only 'classic' consumer + consumer2.close() + check_consumer(kafka_cluster, [consumer], admin_client, topic, ConsumerGroupType.CLASSIC) + + consumer.close() + kafka_cluster.delete_topic(topic) + + +@pytest.mark.skipif(not TestUtils.use_group_protocol_consumer(), + reason="Skipping test as group protocol consumer is not enabled") +def test_consumer_upgrade_downgrade(kafka_cluster): + perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, 'roundrobin') + perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, 'range') + perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, 'cooperative-sticky')