|
| 1 | +#!/usr/bin/env python |
| 2 | +# -*- coding: utf-8 -*- |
| 3 | +# |
| 4 | +# Copyright 2025 Confluent Inc. |
| 5 | +# |
| 6 | +# Licensed under the Apache License, Version 2.0 (the "License"); |
| 7 | +# you may not use this file except in compliance with the License. |
| 8 | +# You may obtain a copy of the License at |
| 9 | +# |
| 10 | +# http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | +# |
| 12 | +# Unless required by applicable law or agreed to in writing, software |
| 13 | +# distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | +# See the License for the specific language governing permissions and |
| 16 | +# limitations under the License. |
| 17 | + |
| 18 | +import pytest |
| 19 | +from enum import Enum |
| 20 | +from confluent_kafka import ConsumerGroupType, KafkaException |
| 21 | +from tests.common import TestUtils |
| 22 | + |
| 23 | +topic_prefix = "test_consumer_upgrade_downgrade_" |
| 24 | +number_of_partitions = 10 |
| 25 | + |
| 26 | + |
| 27 | +def get_group_protocol_type(a, group_id): |
| 28 | + futureMap = a.describe_consumer_groups([group_id]) |
| 29 | + try: |
| 30 | + future = futureMap[group_id] |
| 31 | + g = future.result() |
| 32 | + return g.type |
| 33 | + except KafkaException as e: |
| 34 | + print("Error while describing group id '{}': {}".format(group_id, e)) |
| 35 | + except Exception: |
| 36 | + raise |
| 37 | + |
| 38 | + |
| 39 | +def check_consumer(kafka_cluster, consumers, admin_client, group_id, topic, expected_protocol): |
| 40 | + no_of_messages = 100 |
| 41 | + total_msg_read = 0 |
| 42 | + expected_partitions_per_consumer = number_of_partitions // len(consumers) |
| 43 | + while len(consumers[-1].assignment()) != expected_partitions_per_consumer: |
| 44 | + for consumer in consumers: |
| 45 | + consumer.poll(0.1) |
| 46 | + |
| 47 | + all_assignments = set() |
| 48 | + for consumer in consumers: |
| 49 | + assignment = consumer.assignment() |
| 50 | + all_assignments.update(assignment) |
| 51 | + assert len(assignment) == expected_partitions_per_consumer |
| 52 | + assert len(all_assignments) == number_of_partitions |
| 53 | + |
| 54 | + assert get_group_protocol_type(admin_client, group_id) == expected_protocol |
| 55 | + |
| 56 | + # Produce some messages to the topic |
| 57 | + test_data = ['test-data{}'.format(i) for i in range(0, no_of_messages)] |
| 58 | + test_keys = ['test-key{}'.format(i) for i in range(0, no_of_messages)] # we want each partition to have data |
| 59 | + kafka_cluster.seed_topic(topic, test_data, test_keys) |
| 60 | + |
| 61 | + while total_msg_read < no_of_messages: |
| 62 | + for consumer in consumers: |
| 63 | + # Poll for messages |
| 64 | + msg = consumer.poll(0.1) |
| 65 | + if msg is not None: |
| 66 | + total_msg_read += 1 |
| 67 | + |
| 68 | + assert total_msg_read == no_of_messages, f"Expected to read {no_of_messages} messages, but read {total_msg_read}" |
| 69 | + |
| 70 | + |
| 71 | +class Operation(Enum): |
| 72 | + ADD = 0 |
| 73 | + REMOVE = 1 |
| 74 | + |
| 75 | + |
| 76 | +def perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy( |
| 77 | + kafka_cluster, partition_assignment_strategy): |
| 78 | + """ |
| 79 | + Test consumer upgrade and downgrade. |
| 80 | + """ |
| 81 | + topic_name_prefix = f"{topic_prefix}_{partition_assignment_strategy}" |
| 82 | + topic = kafka_cluster.create_topic_and_wait_propogation(topic_name_prefix, |
| 83 | + { |
| 84 | + "num_partitions": number_of_partitions |
| 85 | + }) |
| 86 | + admin_client = kafka_cluster.admin() |
| 87 | + |
| 88 | + consumer_conf = {'group.id': topic, |
| 89 | + 'auto.offset.reset': 'earliest'} |
| 90 | + consumer_conf_classic = { |
| 91 | + 'group.protocol': 'classic', |
| 92 | + 'partition.assignment.strategy': partition_assignment_strategy, |
| 93 | + **consumer_conf |
| 94 | + } |
| 95 | + consumer_conf_consumer = { |
| 96 | + 'group.protocol': 'consumer', |
| 97 | + **consumer_conf |
| 98 | + } |
| 99 | + |
| 100 | + test_scenarios = [(Operation.ADD, consumer_conf_classic, ConsumerGroupType.CLASSIC), |
| 101 | + (Operation.ADD, consumer_conf_consumer, ConsumerGroupType.CONSUMER), |
| 102 | + (Operation.REMOVE, None, ConsumerGroupType.CONSUMER), |
| 103 | + (Operation.ADD, consumer_conf_classic, ConsumerGroupType.CONSUMER), |
| 104 | + (Operation.REMOVE, None, ConsumerGroupType.CLASSIC)] |
| 105 | + consumers = [] |
| 106 | + |
| 107 | + for operation, conf, expected_protocol in test_scenarios: |
| 108 | + if operation == Operation.ADD: |
| 109 | + consumer = kafka_cluster.consumer(conf) |
| 110 | + assert consumer is not None |
| 111 | + consumer.subscribe([topic]) |
| 112 | + consumers.append(consumer) |
| 113 | + elif operation == Operation.REMOVE: |
| 114 | + consumer_to_remove = consumers.pop(0) |
| 115 | + consumer_to_remove.close() |
| 116 | + check_consumer(kafka_cluster, consumers, admin_client, topic, topic, expected_protocol) |
| 117 | + |
| 118 | + assert len(consumers) == 1 |
| 119 | + consumers[0].close() |
| 120 | + kafka_cluster.delete_topic(topic) |
| 121 | + |
| 122 | + |
| 123 | +@pytest.mark.skipif(not TestUtils.use_group_protocol_consumer(), |
| 124 | + reason="Skipping test as group protocol consumer is not enabled") |
| 125 | +def test_consumer_upgrade_downgrade(kafka_cluster): |
| 126 | + perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, 'roundrobin') |
| 127 | + perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, 'range') |
| 128 | + perform_consumer_upgrade_downgrade_test_with_partition_assignment_strategy(kafka_cluster, 'cooperative-sticky') |
0 commit comments