From 27fad5fe3d667e130476ebd627cb6be78d12d96a Mon Sep 17 00:00:00 2001 From: Apurva Telang Date: Wed, 8 Apr 2020 20:53:02 -0700 Subject: [PATCH 1/4] Adding namedtuples for DescribeConsumerGroup response; Adding Serialization of MemberData and MemberAssignment for the response --- kafka/admin/client.py | 51 ++++++++++++++++++++++++++-------- kafka/structs.py | 5 ++++ test/test_admin_integration.py | 19 ++++++++++++- 3 files changed, 62 insertions(+), 13 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index d0fa84560..d780e06c7 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -1,6 +1,6 @@ from __future__ import absolute_import -from collections import defaultdict +from collections import defaultdict, namedtuple import copy import logging import socket @@ -17,9 +17,11 @@ from kafka.protocol.admin import ( CreateTopicsRequest, DeleteTopicsRequest, DescribeConfigsRequest, AlterConfigsRequest, CreatePartitionsRequest, ListGroupsRequest, DescribeGroupsRequest, DescribeAclsRequest, CreateAclsRequest, DeleteAclsRequest) +from kafka.protocol.types import Array +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment, ConsumerProtocol from kafka.protocol.commit import GroupCoordinatorRequest, OffsetFetchRequest from kafka.protocol.metadata import MetadataRequest -from kafka.structs import TopicPartition, OffsetAndMetadata +from kafka.structs import TopicPartition, OffsetAndMetadata, MemberInformation, GroupInformation from kafka.admin.acl_resource import ACLOperation, ACLPermissionType, ACLFilter, ACL, ResourcePattern, ResourceType, \ ACLResourcePatternType from kafka.version import __version__ @@ -1000,22 +1002,47 @@ def _describe_consumer_groups_process_response(self, response): """Process a DescribeGroupsResponse into a group description.""" if response.API_VERSION <= 3: assert len(response.groups) == 1 - # TODO need to implement converting the response tuple into - # a more accessible interface like a namedtuple and then stop - # hardcoding tuple indices here. Several Java examples, - # including KafkaAdminClient.java - group_description = response.groups[0] - error_code = group_description[0] + for response_field, response_name in zip(response.SCHEMA.fields, response.SCHEMA.names): + if type(response_field) == Array: + described_groups = response.__dict__[response_name] + described_groups_field_schema = response_field.array_of + for described_group in described_groups: + described_group_information_list = [] + is_consumer_protocol_type = False + for (described_group_information, group_information_name, group_information_field) in zip(described_group, described_groups_field_schema.names, described_groups_field_schema.fields): + if group_information_name == 'protocol_type': + protocol_type = described_group_information + is_consumer_protocol_type = (protocol_type == ConsumerProtocol.PROTOCOL_TYPE or not protocol_type) + if type(group_information_field) == Array: + member_information_list = [] + member_schema = group_information_field.array_of + for members in described_group_information: + member_information = [] + for (member, member_field, member_name) in zip(members, member_schema.fields, member_schema.names): + if member_name == 'member_metadata' and is_consumer_protocol_type: + member_information.append(ConsumerProtocolMemberMetadata.decode(member)) + elif member_name == 'member_assignment' and is_consumer_protocol_type: + member_information.append(ConsumerProtocolMemberAssignment.decode(member)) + else: + member_information.append(member) + else: + member_info_tuple = MemberInformation._make(member_information) + member_information_list.append(member_info_tuple) + else: + described_group_information_list.append(member_information_list) + else: + described_group_information_list.append(described_group_information) + else: + if response.API_VERSION <=2: + described_group_information_list.append([]) + group_description = GroupInformation._make(described_group_information_list) + error_code = group_description.error_code error_type = Errors.for_code(error_code) # Java has the note: KAFKA-6789, we can retry based on the error code if error_type is not Errors.NoError: raise error_type( "DescribeGroupsResponse failed with response '{}'." .format(response)) - # TODO Java checks the group protocol type, and if consumer - # (ConsumerProtocol.PROTOCOL_TYPE) or empty string, it decodes - # the members' partition assignments... that hasn't yet been - # implemented here so just return the raw struct results else: raise NotImplementedError( "Support for DescribeGroupsResponse_v{} has not yet been added to KafkaAdminClient." diff --git a/kafka/structs.py b/kafka/structs.py index 9ab4f8bfa..c867cdca2 100644 --- a/kafka/structs.py +++ b/kafka/structs.py @@ -20,6 +20,11 @@ OffsetAndTimestamp = namedtuple("OffsetAndTimestamp", ["offset", "timestamp"]) +MemberInformation = namedtuple("MemberInformation", + ["member_id", "client_id", "client_host", "member_metadata", "member_assignment"]) + +GroupInformation = namedtuple("GroupInformation", + ["error_code", "group", "state", "protocol_type", "protocol", "members", "authorized_operations"]) # Define retry policy for async producer # Limit value: int >= 0, 0 means no retries diff --git a/test/test_admin_integration.py b/test/test_admin_integration.py index 37b140573..b49f88031 100644 --- a/test/test_admin_integration.py +++ b/test/test_admin_integration.py @@ -2,7 +2,7 @@ from test.testutil import env_kafka_version -from kafka.errors import NoError +from kafka.errors import (NoError, GroupCoordinatorNotAvailableError) from kafka.admin import ( ACLFilter, ACLOperation, ACLPermissionType, ResourcePattern, ResourceType, ACL, ConfigResource, ConfigResourceType) @@ -138,3 +138,20 @@ def test_describe_configs_invalid_broker_id_raises(kafka_admin_client): with pytest.raises(ValueError): configs = kafka_admin_client.describe_configs([ConfigResource(ConfigResourceType.BROKER, broker_id)]) + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason='Describe consumer group requires broker >=0.11') +def test_describe_consumer_group_does_not_exist(kafka_admin_client): + """Tests that the describe consumer group call fails if the group coordinator is not available + """ + with pytest.raises(GroupCoordinatorNotAvailableError): + group_description = kafka_admin_client.describe_consumer_groups(['test']) + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason='Describe consumer group requires broker >=0.11') +def test_describe_consumer_group_exists(kafka_admin_client, kafka_consumer_factory, topic): + """Tests that the describe consumer group call returns valid consumer group information + """ + consumer = kafka_consumer_factory(group_id='testgrp', auto_offset_reset='earliest') + consumer.poll(timeout_ms=20) + output = kafka_admin_client.describe_consumer_groups(['testgrp']) + assert output[0].group == 'testgrp' + assert output[0].members[0].member_metadata.subscription[0] == topic From 74379e433fc4bfc9659e0daa3ab04188dda0ca70 Mon Sep 17 00:00:00 2001 From: Apurva Telang Date: Sun, 26 Apr 2020 15:32:30 -0700 Subject: [PATCH 2/4] PR review changes: Sorted Imports in alpha order, made group description response processing more readable, enhanced tests --- kafka/admin/client.py | 62 +++++++++++----------- test/test_admin_integration.py | 95 +++++++++++++++++++++++++++++++--- 2 files changed, 119 insertions(+), 38 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index d780e06c7..c8d6acc1b 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -6,9 +6,11 @@ import socket from . import ConfigResourceType -from kafka.vendor import six +from kafka.admin.acl_resource import ACLOperation, ACLPermissionType, ACLFilter, ACL, ResourcePattern, ResourceType, \ + ACLResourcePatternType from kafka.client_async import KafkaClient, selectors +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment, ConsumerProtocol import kafka.errors as Errors from kafka.errors import ( IncompatibleBrokerVersion, KafkaConfigurationError, NotControllerError, @@ -17,13 +19,11 @@ from kafka.protocol.admin import ( CreateTopicsRequest, DeleteTopicsRequest, DescribeConfigsRequest, AlterConfigsRequest, CreatePartitionsRequest, ListGroupsRequest, DescribeGroupsRequest, DescribeAclsRequest, CreateAclsRequest, DeleteAclsRequest) -from kafka.protocol.types import Array -from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment, ConsumerProtocol from kafka.protocol.commit import GroupCoordinatorRequest, OffsetFetchRequest from kafka.protocol.metadata import MetadataRequest +from kafka.protocol.types import Array from kafka.structs import TopicPartition, OffsetAndMetadata, MemberInformation, GroupInformation -from kafka.admin.acl_resource import ACLOperation, ACLPermissionType, ACLFilter, ACL, ResourcePattern, ResourceType, \ - ACLResourcePatternType +from kafka.vendor import six from kafka.version import __version__ @@ -1003,39 +1003,39 @@ def _describe_consumer_groups_process_response(self, response): if response.API_VERSION <= 3: assert len(response.groups) == 1 for response_field, response_name in zip(response.SCHEMA.fields, response.SCHEMA.names): - if type(response_field) == Array: + if isinstance(response_field, Array): described_groups = response.__dict__[response_name] described_groups_field_schema = response_field.array_of - for described_group in described_groups: - described_group_information_list = [] - is_consumer_protocol_type = False - for (described_group_information, group_information_name, group_information_field) in zip(described_group, described_groups_field_schema.names, described_groups_field_schema.fields): - if group_information_name == 'protocol_type': - protocol_type = described_group_information - is_consumer_protocol_type = (protocol_type == ConsumerProtocol.PROTOCOL_TYPE or not protocol_type) - if type(group_information_field) == Array: - member_information_list = [] - member_schema = group_information_field.array_of - for members in described_group_information: - member_information = [] - for (member, member_field, member_name) in zip(members, member_schema.fields, member_schema.names): - if member_name == 'member_metadata' and is_consumer_protocol_type: + described_group = response.__dict__[response_name][0] + described_group_information_list = [] + protocol_type_is_consumer = False + for (described_group_information, group_information_name, group_information_field) in zip(described_group, described_groups_field_schema.names, described_groups_field_schema.fields): + if group_information_name == 'protocol_type': + protocol_type = described_group_information + protocol_type_is_consumer = (protocol_type == ConsumerProtocol.PROTOCOL_TYPE or not protocol_type) + if type(group_information_field) == Array: + member_information_list = [] + member_schema = group_information_field.array_of + for members in described_group_information: + member_information = [] + for (member, member_field, member_name) in zip(members, member_schema.fields, member_schema.names): + if protocol_type_is_consumer: + if member_name == 'member_metadata' and member: member_information.append(ConsumerProtocolMemberMetadata.decode(member)) - elif member_name == 'member_assignment' and is_consumer_protocol_type: + elif member_name == 'member_assignment' and member: member_information.append(ConsumerProtocolMemberAssignment.decode(member)) else: member_information.append(member) - else: - member_info_tuple = MemberInformation._make(member_information) - member_information_list.append(member_info_tuple) - else: - described_group_information_list.append(member_information_list) - else: - described_group_information_list.append(described_group_information) + member_info_tuple = MemberInformation._make(member_information) + member_information_list.append(member_info_tuple) + described_group_information_list.append(member_information_list) else: - if response.API_VERSION <=2: - described_group_information_list.append([]) - group_description = GroupInformation._make(described_group_information_list) + described_group_information_list.append(described_group_information) + # Version 3 of the DescribeGroups API introduced the "authorized_operations" field. This will cause the namedtuple to fail + # Therefore, appending a placeholder of None in it. + if response.API_VERSION <=2: + described_group_information_list.append(None) + group_description = GroupInformation._make(described_group_information_list) error_code = group_description.error_code error_type = Errors.for_code(error_code) # Java has the note: KAFKA-6789, we can retry based on the error code diff --git a/test/test_admin_integration.py b/test/test_admin_integration.py index b49f88031..dc04537d5 100644 --- a/test/test_admin_integration.py +++ b/test/test_admin_integration.py @@ -1,10 +1,13 @@ import pytest -from test.testutil import env_kafka_version +from logging import info +from test.testutil import env_kafka_version, random_string +from threading import Event, Thread +from time import time, sleep -from kafka.errors import (NoError, GroupCoordinatorNotAvailableError) from kafka.admin import ( ACLFilter, ACLOperation, ACLPermissionType, ResourcePattern, ResourceType, ACL, ConfigResource, ConfigResourceType) +from kafka.errors import (NoError, GroupCoordinatorNotAvailableError) @pytest.mark.skipif(env_kafka_version() < (0, 11), reason="ACL features require broker >=0.11") @@ -149,9 +152,87 @@ def test_describe_consumer_group_does_not_exist(kafka_admin_client): @pytest.mark.skipif(env_kafka_version() < (0, 11), reason='Describe consumer group requires broker >=0.11') def test_describe_consumer_group_exists(kafka_admin_client, kafka_consumer_factory, topic): """Tests that the describe consumer group call returns valid consumer group information + This test takes inspiration from the test 'test_group' in test_consumer_group.py. """ - consumer = kafka_consumer_factory(group_id='testgrp', auto_offset_reset='earliest') - consumer.poll(timeout_ms=20) - output = kafka_admin_client.describe_consumer_groups(['testgrp']) - assert output[0].group == 'testgrp' - assert output[0].members[0].member_metadata.subscription[0] == topic + consumers = {} + stop = {} + threads = {} + random_group_id = 'test-group-' + random_string(6) + group_id_list = [random_group_id, random_group_id + '_2'] + generations = {group_id_list[0]: set(), group_id_list[1]: set()} + def consumer_thread(i, group_id): + assert i not in consumers + assert i not in stop + stop[i] = Event() + consumers[i] = kafka_consumer_factory(group_id=group_id) + while not stop[i].is_set(): + consumers[i].poll(20) + consumers[i].close() + consumers[i] = None + stop[i] = None + + num_consumers = 3 + for i in range(num_consumers): + group_id = group_id_list[i % 2] + t = Thread(target=consumer_thread, args=(i, group_id,)) + t.start() + threads[i] = t + + try: + timeout = time() + 35 + while True: + for c in range(num_consumers): + + # Verify all consumers have been created + if c not in consumers: + break + + # Verify all consumers have an assignment + elif not consumers[c].assignment(): + break + + # If all consumers exist and have an assignment + else: + + info('All consumers have assignment... checking for stable group') + # Verify all consumers are in the same generation + # then log state and break while loop + + for consumer in consumers.values(): + generations[consumer.config['group_id']].add(consumer._coordinator._generation.generation_id) + + is_same_generation = any([len(consumer_generation) == 1 for consumer_generation in generations.values()]) + + # New generation assignment is not complete until + # coordinator.rejoining = False + rejoining = any([consumer._coordinator.rejoining + for consumer in list(consumers.values())]) + + if not rejoining and is_same_generation: + break + else: + sleep(1) + assert time() < timeout, "timeout waiting for assignments" + + info('Group stabilized; verifying assignment') + output = kafka_admin_client.describe_consumer_groups(group_id_list) + assert len(output) == 2 + consumer_groups = set() + for consumer_group in output: + assert(consumer_group.group in group_id_list) + if consumer_group.group == group_id_list[0]: + assert(len(consumer_group.members) == 2) + else: + assert(len(consumer_group.members) == 1) + for member in consumer_group.members: + assert(member.member_metadata.subscription[0] == topic) + assert(member.member_assignment.assignment[0][0] == topic) + consumer_groups.add(consumer_group.group) + assert(sorted(list(consumer_groups)) == group_id_list) + finally: + info('Shutting down %s consumers', num_consumers) + for c in range(num_consumers): + info('Stopping consumer %s', c) + stop[c].set() + threads[c].join() + threads[c] = None From 5e13d38aafd8d8f86ca8663b65610973d1552776 Mon Sep 17 00:00:00 2001 From: Apurva Telang Date: Fri, 1 May 2020 12:35:47 -0700 Subject: [PATCH 3/4] Change type to isinstance as per PR review --- kafka/admin/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index c8d6acc1b..e8720b82b 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -1013,7 +1013,7 @@ def _describe_consumer_groups_process_response(self, response): if group_information_name == 'protocol_type': protocol_type = described_group_information protocol_type_is_consumer = (protocol_type == ConsumerProtocol.PROTOCOL_TYPE or not protocol_type) - if type(group_information_field) == Array: + if isinstance(group_information_field, Array): member_information_list = [] member_schema = group_information_field.array_of for members in described_group_information: From c890b8a3f434d6c24964e3d8cf16b972e00ac297 Mon Sep 17 00:00:00 2001 From: Jeff Widman Date: Wed, 16 Sep 2020 17:31:23 -0700 Subject: [PATCH 4/4] revert the vendored import location change --- kafka/admin/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index e8720b82b..e820587f8 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -6,6 +6,7 @@ import socket from . import ConfigResourceType +from kafka.vendor import six from kafka.admin.acl_resource import ACLOperation, ACLPermissionType, ACLFilter, ACL, ResourcePattern, ResourceType, \ ACLResourcePatternType @@ -23,7 +24,6 @@ from kafka.protocol.metadata import MetadataRequest from kafka.protocol.types import Array from kafka.structs import TopicPartition, OffsetAndMetadata, MemberInformation, GroupInformation -from kafka.vendor import six from kafka.version import __version__