From 179c8ed954a988ca8d29fa81c76a12cd2527914e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 09:48:52 -0700 Subject: [PATCH 01/51] Add send_request() and send_requests() to KafkaAdminClient (#2649) --- kafka/admin/client.py | 410 +++++++-------------- test/integration/test_admin_integration.py | 3 - 2 files changed, 136 insertions(+), 277 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 8490fdb46..c5d8f8636 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -2,6 +2,7 @@ from collections import defaultdict import copy +import itertools import logging import socket import time @@ -255,12 +256,7 @@ def _refresh_controller_id(self, timeout_ms=30000): if 1 <= version <= 6: timeout_at = time.time() + timeout_ms / 1000 while time.time() < timeout_at: - request = MetadataRequest[version]() - future = self._send_request_to_node(self._client.least_loaded_node(), request) - - self._wait_for_futures([future]) - - response = future.value + response = self.send_request(MetadataRequest[version]()) controller_id = response.controller_id if controller_id == -1: log.warning("Controller ID not available, got -1") @@ -281,7 +277,7 @@ def _refresh_controller_id(self, timeout_ms=30000): "Kafka Admin interface cannot determine the controller using MetadataRequest_v{}." .format(version)) - def _find_coordinator_id_send_request(self, group_id): + def _find_coordinator_id_request(self, group_id): """Send a FindCoordinatorRequest to a broker. Arguments: @@ -289,18 +285,14 @@ def _find_coordinator_id_send_request(self, group_id): name as a string. Returns: - A message future + FindCoordinatorRequest """ version = self._client.api_version(FindCoordinatorRequest, max_version=2) if version <= 0: request = FindCoordinatorRequest[version](group_id) elif version <= 2: request = FindCoordinatorRequest[version](group_id, 0) - else: - raise NotImplementedError( - "Support for FindCoordinatorRequest_v{} has not yet been added to KafkaAdminClient." - .format(version)) - return self._send_request_to_node(self._client.least_loaded_node(), request) + return request def _find_coordinator_id_process_response(self, response): """Process a FindCoordinatorResponse. @@ -335,16 +327,9 @@ def _find_coordinator_ids(self, group_ids): A dict of {group_id: node_id} where node_id is the id of the broker that is the coordinator for the corresponding group. """ - groups_futures = { - group_id: self._find_coordinator_id_send_request(group_id) - for group_id in group_ids - } - self._wait_for_futures(groups_futures.values()) - groups_coordinators = { - group_id: self._find_coordinator_id_process_response(future.value) - for group_id, future in groups_futures.items() - } - return groups_coordinators + requests = [(self._find_coordinator_id_request(group_id), None) for group_id in group_ids] + coordinator_ids = self.send_requests(requests, response_fn=self._find_coordinator_id_process_response) + return dict(zip(group_ids, coordinator_ids)) def _send_request_to_node(self, node_id, request, wakeup=True): """Send a Kafka protocol message to a specific broker. @@ -366,6 +351,40 @@ def _send_request_to_node(self, node_id, request, wakeup=True): return Future().failure(e) return self._client.send(node_id, request, wakeup) + def _wait_for_futures(self, futures): + """Block until all futures complete. If any fail, raise the encountered exception. + + Arguments: + futures: A list of Future objects awaiting results. + + Raises: + The first encountered exception if a future fails. + """ + while not all(future.succeeded() for future in futures): + for future in futures: + self._client.poll(future=future) + + if future.failed(): + raise future.exception # pylint: disable-msg=raising-bad-type + + def send_request(self, request, node_id=None): + if node_id is None: + node_id = self._client.least_loaded_node() + self._client.await_ready(node_id) + future = self._client.send(node_id, request) + self._wait_for_futures([future]) # raises exception on failure + return future.value + + def send_requests(self, requests_and_node_ids, response_fn=lambda x: x): + futures = [] + for request, node_id in requests_and_node_ids: + if node_id is None: + node_id = self._client.least_loaded_node() + self._client.await_ready(node_id) + futures.append(self._client.send(node_id, request)) + self._wait_for_futures(futures) + return [response_fn(future.value) for future in futures] + def _send_request_to_controller(self, request): """Send a Kafka protocol message to the cluster controller. @@ -380,11 +399,7 @@ def _send_request_to_controller(self, request): tries = 2 # in case our cached self._controller_id is outdated while tries: tries -= 1 - future = self._send_request_to_node(self._controller_id, request) - - self._wait_for_futures([future]) - - response = future.value + response = self.send_request(request, node_id=self._controller_id) # In Java, the error field name is inconsistent: # - CreateTopicsResponse / CreatePartitionsResponse uses topic_errors # - DeleteTopicsResponse uses topic_error_codes @@ -499,10 +514,6 @@ def create_topics(self, new_topics, timeout_ms=None, validate_only=False): timeout=timeout_ms, validate_only=validate_only ) - else: - raise NotImplementedError( - "Support for CreateTopics v{} has not yet been added to KafkaAdminClient." - .format(version)) # TODO convert structs to a more pythonic interface # TODO raise exceptions if errors return self._send_request_to_controller(request) @@ -522,18 +533,12 @@ def delete_topics(self, topics, timeout_ms=None): """ version = self._client.api_version(DeleteTopicsRequest, max_version=3) timeout_ms = self._validate_timeout(timeout_ms) - if version <= 3: - request = DeleteTopicsRequest[version]( + return self._send_request_to_controller( + DeleteTopicsRequest[version]( topics=topics, timeout=timeout_ms ) - response = self._send_request_to_controller(request) - else: - raise NotImplementedError( - "Support for DeleteTopics v{} has not yet been added to KafkaAdminClient." - .format(version)) - return response - + ) def _get_cluster_metadata(self, topics=None, auto_topic_creation=False): """ @@ -554,12 +559,7 @@ def _get_cluster_metadata(self, topics=None, auto_topic_creation=False): allow_auto_topic_creation=auto_topic_creation ) - future = self._send_request_to_node( - self._client.least_loaded_node(), - request - ) - self._wait_for_futures([future]) - return future.value + return self.send_request(request) def list_topics(self): """Retrieve a list of all topic names in the cluster. @@ -677,16 +677,7 @@ def describe_acls(self, acl_filter): permission_type=acl_filter.permission_type ) - else: - raise NotImplementedError( - "Support for DescribeAcls v{} has not yet been added to KafkaAdmin." - .format(version) - ) - - future = self._send_request_to_node(self._client.least_loaded_node(), request) - self._wait_for_futures([future]) - response = future.value - + response = self.send_request(request) error_type = Errors.for_code(response.error_code) if error_type is not Errors.NoError: # optionally we could retry if error_type.retriable @@ -799,16 +790,7 @@ def create_acls(self, acls): request = CreateAclsRequest[version]( creations=[self._convert_create_acls_resource_request_v1(acl) for acl in acls] ) - else: - raise NotImplementedError( - "Support for CreateAcls v{} has not yet been added to KafkaAdmin." - .format(version) - ) - - future = self._send_request_to_node(self._client.least_loaded_node(), request) - self._wait_for_futures([future]) - response = future.value - + response = self.send_request(request) return self._convert_create_acls_response_to_acls(acls, response) @staticmethod @@ -922,16 +904,7 @@ def delete_acls(self, acl_filters): request = DeleteAclsRequest[version]( filters=[self._convert_delete_acls_resource_request_v1(acl) for acl in acl_filters] ) - else: - raise NotImplementedError( - "Support for DeleteAcls v{} has not yet been added to KafkaAdmin." - .format(version) - ) - - future = self._send_request_to_node(self._client.least_loaded_node(), request) - self._wait_for_futures([future]) - response = future.value - + response = self.send_request(request) return self._convert_delete_acls_response_to_matching_acls(acl_filters, response) @staticmethod @@ -966,7 +939,7 @@ def describe_configs(self, config_resources, include_synonyms=False): supported by all versions. Default: False. Returns: - Appropriate version of DescribeConfigsResponse class. + List of DescribeConfigsResponses. """ # Break up requests by type - a broker config request must be sent to the specific broker. @@ -980,58 +953,36 @@ def describe_configs(self, config_resources, include_synonyms=False): else: topic_resources.append(self._convert_describe_config_resource_request(config_resource)) - futures = [] version = self._client.api_version(DescribeConfigsRequest, max_version=2) - if version == 0: - if include_synonyms: - raise IncompatibleBrokerVersion( - "include_synonyms requires DescribeConfigsRequest >= v1, which is not supported by Kafka {}." - .format(self.config['api_version'])) - - if len(broker_resources) > 0: - for broker_resource in broker_resources: - try: - broker_id = int(broker_resource[1]) - except ValueError: - raise ValueError("Broker resource names must be an integer or a string represented integer") - - futures.append(self._send_request_to_node( - broker_id, - DescribeConfigsRequest[version](resources=[broker_resource]) - )) - - if len(topic_resources) > 0: - futures.append(self._send_request_to_node( - self._client.least_loaded_node(), - DescribeConfigsRequest[version](resources=topic_resources) - )) + if include_synonyms and version == 0: + raise IncompatibleBrokerVersion( + "include_synonyms requires DescribeConfigsRequest >= v1, which is not supported by Kafka {}." + .format(self.config['api_version'])) - elif version <= 2: - if len(broker_resources) > 0: - for broker_resource in broker_resources: - try: - broker_id = int(broker_resource[1]) - except ValueError: - raise ValueError("Broker resource names must be an integer or a string represented integer") - - futures.append(self._send_request_to_node( - broker_id, - DescribeConfigsRequest[version]( - resources=[broker_resource], - include_synonyms=include_synonyms) - )) - - if len(topic_resources) > 0: - futures.append(self._send_request_to_node( - self._client.least_loaded_node(), - DescribeConfigsRequest[version](resources=topic_resources, include_synonyms=include_synonyms) - )) - else: - raise NotImplementedError( - "Support for DescribeConfigs v{} has not yet been added to KafkaAdminClient.".format(version)) + requests = [] + if len(broker_resources) > 0: + for broker_resource in broker_resources: + try: + broker_id = int(broker_resource[1]) + except ValueError: + raise ValueError("Broker resource names must be an integer or a string represented integer") - self._wait_for_futures(futures) - return [f.value for f in futures] + if version == 0: + request = DescribeConfigsRequest[version](resources=[broker_resource]) + else: + request = DescribeConfigsRequest[version]( + resources=[broker_resource], + include_synonyms=include_synonyms) + requests.append((request, broker_id)) + + if len(topic_resources) > 0: + if version == 0: + request = DescribeConfigsRequest[version](resources=topic_resources) + else: + request = DescribeConfigsRequest[version](resources=topic_resources, include_synonyms=include_synonyms) + requests.append((request, None)) + + return self.send_requests(requests) @staticmethod def _convert_alter_config_resource_request(config_resource): @@ -1067,25 +1018,16 @@ def alter_configs(self, config_resources): Appropriate version of AlterConfigsResponse class. """ version = self._client.api_version(AlterConfigsRequest, max_version=1) - if version <= 1: - request = AlterConfigsRequest[version]( - resources=[self._convert_alter_config_resource_request(config_resource) for config_resource in config_resources] - ) - else: - raise NotImplementedError( - "Support for AlterConfigs v{} has not yet been added to KafkaAdminClient." - .format(version)) + request = AlterConfigsRequest[version]( + resources=[self._convert_alter_config_resource_request(config_resource) for config_resource in config_resources] + ) # TODO the Java client has the note: # // We must make a separate AlterConfigs request for every BROKER resource we want to alter # // and send the request to that specific broker. Other resources are grouped together into # // a single request that may be sent to any broker. # # So this is currently broken as it always sends to the least_loaded_node() - future = self._send_request_to_node(self._client.least_loaded_node(), request) - - self._wait_for_futures([future]) - response = future.value - return response + return self.send_request(request) # alter replica logs dir protocol not yet implemented # Note: have to lookup the broker with the replica assignment and send the request to that broker @@ -1129,16 +1071,11 @@ def create_partitions(self, topic_partitions, timeout_ms=None, validate_only=Fal """ version = self._client.api_version(CreatePartitionsRequest, max_version=1) timeout_ms = self._validate_timeout(timeout_ms) - if version <= 1: - request = CreatePartitionsRequest[version]( - topic_partitions=[self._convert_create_partitions_request(topic_name, new_partitions) for topic_name, new_partitions in topic_partitions.items()], - timeout=timeout_ms, - validate_only=validate_only - ) - else: - raise NotImplementedError( - "Support for CreatePartitions v{} has not yet been added to KafkaAdminClient." - .format(version)) + request = CreatePartitionsRequest[version]( + topic_partitions=[self._convert_create_partitions_request(topic_name, new_partitions) for topic_name, new_partitions in topic_partitions.items()], + timeout=timeout_ms, + validate_only=validate_only + ) return self._send_request_to_controller(request) def _get_leader_for_partitions(self, partitions, timeout_ms=None): @@ -1195,8 +1132,6 @@ def delete_records(self, records_to_delete, timeout_ms=None, partition_leader_id timeout_ms = self._validate_timeout(timeout_ms) responses = [] version = self._client.api_version(DeleteRecordsRequest, max_version=0) - if version is None: - raise IncompatibleBrokerVersion("Broker does not support DeleteGroupsRequest") # We want to make as few requests as possible # If a single node serves as a partition leader for multiple partitions (and/or @@ -1221,10 +1156,8 @@ def delete_records(self, records_to_delete, timeout_ms=None, partition_leader_id ], timeout_ms=timeout_ms ) - future = self._send_request_to_node(leader, request) - self._wait_for_futures([future]) - - responses.append(future.value.to_object()) + response = self.send_request(request, node_id=leader) + responses.append(response.to_object()) partition2result = {} partition2error = {} @@ -1266,15 +1199,14 @@ def delete_records(self, records_to_delete, timeout_ms=None, partition_leader_id # describe delegation_token protocol not yet implemented # Note: send the request to the least_loaded_node() - def _describe_consumer_groups_send_request(self, group_id, group_coordinator_id, include_authorized_operations=False): + def _describe_consumer_groups_request(self, group_id, include_authorized_operations=False): """Send a DescribeGroupsRequest to the group's coordinator. Arguments: group_id: The group name as a string - group_coordinator_id: The node_id of the groups' coordinator broker. Returns: - A message future. + DescribeGroupsRequest object """ version = self._client.api_version(DescribeGroupsRequest, max_version=3) if version <= 2: @@ -1295,11 +1227,7 @@ def _describe_consumer_groups_send_request(self, group_id, group_coordinator_id, groups=(group_id,), include_authorized_operations=include_authorized_operations ) - else: - raise NotImplementedError( - "Support for DescribeGroupsRequest_v{} has not yet been added to KafkaAdminClient." - .format(version)) - return self._send_request_to_node(group_coordinator_id, request) + return request def _describe_consumer_groups_process_response(self, response): """Process a DescribeGroupsResponse into a group description.""" @@ -1378,46 +1306,25 @@ def describe_consumer_groups(self, group_ids, group_coordinator_id=None, include plan to change this to return namedtuples as well as decoding the partition assignments. """ - group_descriptions = [] - if group_coordinator_id is not None: groups_coordinators = {group_id: group_coordinator_id for group_id in group_ids} else: groups_coordinators = self._find_coordinator_ids(group_ids) - futures = [ - self._describe_consumer_groups_send_request( - group_id, - coordinator_id, - include_authorized_operations) + requests = [ + (self._describe_consumer_groups_request(group_id, include_authorized_operations), coordinator_id) for group_id, coordinator_id in groups_coordinators.items() ] - self._wait_for_futures(futures) - - for future in futures: - response = future.value - group_description = self._describe_consumer_groups_process_response(response) - group_descriptions.append(group_description) + return self.send_requests(requests, response_fn=self._describe_consumer_groups_process_response) - return group_descriptions - - def _list_consumer_groups_send_request(self, broker_id): + def _list_consumer_groups_request(self): """Send a ListGroupsRequest to a broker. - Arguments: - broker_id (int): The broker's node_id. - Returns: - A message future + ListGroupsRequest object """ version = self._client.api_version(ListGroupsRequest, max_version=2) - if version <= 2: - request = ListGroupsRequest[version]() - else: - raise NotImplementedError( - "Support for ListGroupsRequest_v{} has not yet been added to KafkaAdminClient." - .format(version)) - return self._send_request_to_node(broker_id, request) + return ListGroupsRequest[version]() def _list_consumer_groups_process_response(self, response): """Process a ListGroupsResponse into a list of groups.""" @@ -1467,23 +1374,20 @@ def list_consumer_groups(self, broker_ids=None): # because if a group coordinator fails after being queried, and its # consumer groups move to new brokers that haven't yet been queried, # then the same group could be returned by multiple brokers. - consumer_groups = set() if broker_ids is None: broker_ids = [broker.nodeId for broker in self._client.cluster.brokers()] - futures = [self._list_consumer_groups_send_request(b) for b in broker_ids] - self._wait_for_futures(futures) - for f in futures: - response = f.value - consumer_groups.update(self._list_consumer_groups_process_response(response)) - return list(consumer_groups) + requests = [ + (self._list_consumer_groups_request(), broker_id) + for broker_id in broker_ids + ] + consumer_groups = self.send_requests(requests, response_fn=self._list_consumer_groups_process_response) + return list(set().union(*consumer_groups)) - def _list_consumer_group_offsets_send_request(self, group_id, - group_coordinator_id, partitions=None): + def _list_consumer_group_offsets_request(self, group_id, partitions=None): """Send an OffsetFetchRequest to a broker. Arguments: group_id (str): The consumer group id name for which to fetch offsets. - group_coordinator_id (int): The node_id of the group's coordinator broker. Keyword Arguments: partitions: A list of TopicPartitions for which to fetch @@ -1491,30 +1395,24 @@ def _list_consumer_group_offsets_send_request(self, group_id, known offsets for the consumer group. Default: None. Returns: - A message future + OffsetFetchRequest object """ version = self._client.api_version(OffsetFetchRequest, max_version=5) - if version <= 5: - if partitions is None: - if version <= 1: - raise ValueError( - """OffsetFetchRequest_v{} requires specifying the - partitions for which to fetch offsets. Omitting the - partitions is only supported on brokers >= 0.10.2. - For details, see KIP-88.""".format(version)) - topics_partitions = None - else: - # transform from [TopicPartition("t1", 1), TopicPartition("t1", 2)] to [("t1", [1, 2])] - topics_partitions_dict = defaultdict(set) - for topic, partition in partitions: - topics_partitions_dict[topic].add(partition) - topics_partitions = list(six.iteritems(topics_partitions_dict)) - request = OffsetFetchRequest[version](group_id, topics_partitions) + if partitions is None: + if version <= 1: + raise ValueError( + """OffsetFetchRequest_v{} requires specifying the + partitions for which to fetch offsets. Omitting the + partitions is only supported on brokers >= 0.10.2. + For details, see KIP-88.""".format(version)) + topics_partitions = None else: - raise NotImplementedError( - "Support for OffsetFetchRequest_v{} has not yet been added to KafkaAdminClient." - .format(version)) - return self._send_request_to_node(group_coordinator_id, request) + # transform from [TopicPartition("t1", 1), TopicPartition("t1", 2)] to [("t1", [1, 2])] + topics_partitions_dict = defaultdict(set) + for topic, partition in partitions: + topics_partitions_dict[topic].add(partition) + topics_partitions = list(six.iteritems(topics_partitions_dict)) + return OffsetFetchRequest[version](group_id, topics_partitions) def _list_consumer_group_offsets_process_response(self, response): """Process an OffsetFetchResponse. @@ -1592,10 +1490,8 @@ def list_consumer_group_offsets(self, group_id, group_coordinator_id=None, """ if group_coordinator_id is None: group_coordinator_id = self._find_coordinator_ids([group_id])[group_id] - future = self._list_consumer_group_offsets_send_request( - group_id, group_coordinator_id, partitions) - self._wait_for_futures([future]) - response = future.value + request = self._list_consumer_group_offsets_request(group_id, partitions) + response = self.send_request(request, node_id=group_coordinator_id) return self._list_consumer_group_offsets_process_response(response) def delete_consumer_groups(self, group_ids, group_coordinator_id=None): @@ -1621,23 +1517,20 @@ def delete_consumer_groups(self, group_ids, group_coordinator_id=None): Returns: A list of tuples (group_id, KafkaError) """ + coordinators_groups = defaultdict(list) if group_coordinator_id is not None: - futures = [self._delete_consumer_groups_send_request(group_ids, group_coordinator_id)] + coordinators_groups[group_coordinator_id] = group_ids else: - coordinators_groups = defaultdict(list) for group_id, coordinator_id in self._find_coordinator_ids(group_ids).items(): coordinators_groups[coordinator_id].append(group_id) - futures = [ - self._delete_consumer_groups_send_request(group_ids, coordinator_id) - for coordinator_id, group_ids in coordinators_groups.items() - ] - self._wait_for_futures(futures) + requests = [ + (self._delete_consumer_groups_request(group_ids), coordinator_id) + for coordinator_id, group_ids in coordinators_groups.items() + ] - results = [] - for f in futures: - results.extend(self._convert_delete_groups_response(f.value)) - return results + results = self.send_requests(requests, response_fn=self._convert_delete_groups_response) + return list(itertools.chain(*results)) def _convert_delete_groups_response(self, response): """Parse the DeleteGroupsResponse, mapping group IDs to their respective errors. @@ -1658,24 +1551,17 @@ def _convert_delete_groups_response(self, response): "Support for DeleteGroupsResponse_v{} has not yet been added to KafkaAdminClient." .format(response.API_VERSION)) - def _delete_consumer_groups_send_request(self, group_ids, group_coordinator_id): - """Send a DeleteGroupsRequest to the specified broker (the group coordinator). + def _delete_consumer_groups_request(self, group_ids): + """Build a DeleteGroupsRequest to send to a broker (the group coordinator). Arguments: group_ids ([str]): A list of consumer group IDs to be deleted. - group_coordinator_id (int): The node_id of the broker coordinating these groups. Returns: - A future representing the in-flight DeleteGroupsRequest. + A DeleteGroupsRequest object. """ version = self._client.api_version(DeleteGroupsRequest, max_version=1) - if version <= 1: - request = DeleteGroupsRequest[version](group_ids) - else: - raise NotImplementedError( - "Support for DeleteGroupsRequest_v{} has not yet been added to KafkaAdminClient." - .format(version)) - return self._send_request_to_node(group_coordinator_id, request) + return DeleteGroupsRequest[version](group_ids) @staticmethod def _convert_topic_partitions(topic_partitions): @@ -1722,35 +1608,11 @@ def perform_leader_election(self, election_type, topic_partitions=None, timeout_ # TODO convert structs to a more pythonic interface return self._send_request_to_controller(request) - def _wait_for_futures(self, futures): - """Block until all futures complete. If any fail, raise the encountered exception. - - Arguments: - futures: A list of Future objects awaiting results. - - Raises: - The first encountered exception if a future fails. - """ - while not all(future.succeeded() for future in futures): - for future in futures: - self._client.poll(future=future) - - if future.failed(): - raise future.exception # pylint: disable-msg=raising-bad-type - def describe_log_dirs(self): """Send a DescribeLogDirsRequest request to a broker. Returns: - A message future + DescribeLogDirsResponse object """ version = self._client.api_version(DescribeLogDirsRequest, max_version=0) - if version <= 0: - request = DescribeLogDirsRequest[version]() - future = self._send_request_to_node(self._client.least_loaded_node(), request) - self._wait_for_futures([future]) - else: - raise NotImplementedError( - "Support for DescribeLogDirsRequest_v{} has not yet been added to KafkaAdminClient." - .format(version)) - return future.value + return self.send_request(DescribeLogDirsRequest[version]()) diff --git a/test/integration/test_admin_integration.py b/test/integration/test_admin_integration.py index f95f367e8..1c38400e7 100644 --- a/test/integration/test_admin_integration.py +++ b/test/integration/test_admin_integration.py @@ -383,6 +383,3 @@ def test_delete_records_with_errors(kafka_admin_client, topic, send_messages): kafka_admin_client.delete_records({p0: 1000}) with pytest.raises(BrokerResponseError): kafka_admin_client.delete_records({p0: 1000, p1: 1000}) - - - From 6f6a6968aceefd19ccb9affc8ff8946ca21ff3ac Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 15:08:12 -0700 Subject: [PATCH 02/51] python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) --- kafka/admin/__main__.py | 7 + kafka/admin/config_resource.py | 6 + kafka/admin/new_topic.py | 9 +- kafka/cli/__init__.py | 0 kafka/cli/admin/__init__.py | 149 ++++++++++++++++++ kafka/cli/admin/cluster/__init__.py | 16 ++ kafka/cli/admin/cluster/describe.py | 9 ++ kafka/cli/admin/configs/__init__.py | 16 ++ kafka/cli/admin/configs/describe.py | 24 +++ kafka/cli/admin/consumer_groups/__init__.py | 19 +++ kafka/cli/admin/consumer_groups/delete.py | 10 ++ kafka/cli/admin/consumer_groups/describe.py | 10 ++ kafka/cli/admin/consumer_groups/list.py | 9 ++ .../cli/admin/consumer_groups/list_offsets.py | 10 ++ kafka/cli/admin/log_dirs/__init__.py | 16 ++ kafka/cli/admin/log_dirs/describe.py | 9 ++ kafka/cli/admin/topics/__init__.py | 19 +++ kafka/cli/admin/topics/create.py | 18 +++ kafka/cli/admin/topics/delete.py | 10 ++ kafka/cli/admin/topics/describe.py | 10 ++ kafka/cli/admin/topics/list.py | 9 ++ kafka/cli/consumer/__init__.py | 90 +++++++++++ kafka/cli/producer/__init__.py | 85 ++++++++++ kafka/consumer/__main__.py | 7 + kafka/producer/__main__.py | 7 + test/test_admin.py | 28 +++- 26 files changed, 589 insertions(+), 13 deletions(-) create mode 100644 kafka/admin/__main__.py create mode 100644 kafka/cli/__init__.py create mode 100644 kafka/cli/admin/__init__.py create mode 100644 kafka/cli/admin/cluster/__init__.py create mode 100644 kafka/cli/admin/cluster/describe.py create mode 100644 kafka/cli/admin/configs/__init__.py create mode 100644 kafka/cli/admin/configs/describe.py create mode 100644 kafka/cli/admin/consumer_groups/__init__.py create mode 100644 kafka/cli/admin/consumer_groups/delete.py create mode 100644 kafka/cli/admin/consumer_groups/describe.py create mode 100644 kafka/cli/admin/consumer_groups/list.py create mode 100644 kafka/cli/admin/consumer_groups/list_offsets.py create mode 100644 kafka/cli/admin/log_dirs/__init__.py create mode 100644 kafka/cli/admin/log_dirs/describe.py create mode 100644 kafka/cli/admin/topics/__init__.py create mode 100644 kafka/cli/admin/topics/create.py create mode 100644 kafka/cli/admin/topics/delete.py create mode 100644 kafka/cli/admin/topics/describe.py create mode 100644 kafka/cli/admin/topics/list.py create mode 100644 kafka/cli/consumer/__init__.py create mode 100644 kafka/cli/producer/__init__.py create mode 100644 kafka/consumer/__main__.py create mode 100644 kafka/producer/__main__.py diff --git a/kafka/admin/__main__.py b/kafka/admin/__main__.py new file mode 100644 index 000000000..776063869 --- /dev/null +++ b/kafka/admin/__main__.py @@ -0,0 +1,7 @@ +from __future__ import absolute_import + +import sys + +from kafka.cli.admin import run_cli + +sys.exit(run_cli()) diff --git a/kafka/admin/config_resource.py b/kafka/admin/config_resource.py index e3294c9c4..06754ba9f 100644 --- a/kafka/admin/config_resource.py +++ b/kafka/admin/config_resource.py @@ -34,3 +34,9 @@ def __init__( self.resource_type = resource_type self.name = name self.configs = configs + + def __str__(self): + return "ConfigResource %s=%s" % (self.resource_type, self.name) + + def __repr__(self): + return "ConfigResource(%s, %s, %s)" % (self.resource_type, self.name, self.configs) diff --git a/kafka/admin/new_topic.py b/kafka/admin/new_topic.py index 645ac383a..e43c52226 100644 --- a/kafka/admin/new_topic.py +++ b/kafka/admin/new_topic.py @@ -1,7 +1,5 @@ from __future__ import absolute_import -from kafka.errors import IllegalArgumentError - class NewTopic(object): """ A class for new topic creation @@ -16,17 +14,14 @@ class NewTopic(object): topic_configs (dict of str: str): A mapping of config key and value for the topic. """ - def __init__( self, name, - num_partitions, - replication_factor, + num_partitions=-1, + replication_factor=-1, replica_assignments=None, topic_configs=None, ): - if not (num_partitions == -1 or replication_factor == -1) ^ (replica_assignments is None): - raise IllegalArgumentError('either num_partitions/replication_factor or replica_assignment must be specified') self.name = name self.num_partitions = num_partitions self.replication_factor = replication_factor diff --git a/kafka/cli/__init__.py b/kafka/cli/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/kafka/cli/admin/__init__.py b/kafka/cli/admin/__init__.py new file mode 100644 index 000000000..02971e333 --- /dev/null +++ b/kafka/cli/admin/__init__.py @@ -0,0 +1,149 @@ +from __future__ import absolute_import + +import argparse +import json +import logging +from pprint import pprint + +from kafka.admin.client import KafkaAdminClient +from .cluster import ClusterSubCommand +from .configs import ConfigsSubCommand +from .consumer_groups import ConsumerGroupsSubCommand +from .log_dirs import LogDirsSubCommand +from .topics import TopicsSubCommand + +def main_parser(): + parser = argparse.ArgumentParser( + prog='python -m kafka.admin', + description='Kafka admin client', + ) + parser.add_argument( + '-b', '--bootstrap-servers', type=str, action='append', required=True, + help='host:port for cluster bootstrap servers') + parser.add_argument( + '-c', '--extra-config', type=str, action='append', + help='additional configuration properties for admin client') + parser.add_argument( + '-l', '--log-level', type=str, + help='logging level, passed to logging.basicConfig') + parser.add_argument( + '-f', '--format', type=str, default='raw', + help='output format: raw|json') + return parser + + +_LOGGING_LEVELS = {'NOTSET': 0, 'DEBUG': 10, 'INFO': 20, 'WARNING': 30, 'ERROR': 40, 'CRITICAL': 50} + + +def build_kwargs(props): + kwargs = {} + for prop in props or []: + k, v = prop.split('=') + try: + v = int(v) + except ValueError: + pass + if v == 'None': + v = None + elif v == 'False': + v = False + elif v == 'True': + v = True + kwargs[k] = v + return kwargs + + +def run_cli(args=None): + parser = main_parser() + subparsers = parser.add_subparsers(help='subcommands') + for cmd in [ClusterSubCommand, ConfigsSubCommand, LogDirsSubCommand, + TopicsSubCommand, ConsumerGroupsSubCommand]: + cmd.add_subparser(subparsers) + + config = parser.parse_args(args) + if config.log_level: + logging.basicConfig(level=_LOGGING_LEVELS[config.log_level.upper()]) + if config.format not in ('raw', 'json'): + raise ValueError('Unrecognized format: %s' % config.format) + logger = logging.getLogger(__name__) + + kwargs = build_kwargs(config.extra_config) + client = KafkaAdminClient(bootstrap_servers=config.bootstrap_servers, **kwargs) + try: + result = config.command(client, config) + if config.format == 'raw': + pprint(result) + elif config.format == 'json': + print(json.dumps(result)) + return 0 + except AttributeError: + parser.print_help() + return 2 + except Exception: + logger.exception('Error!') + return 1 + +if __name__ == '__main__': + import sys + sys.exit(run_cli()) + + +# Commands TODO: + # [acls] + # describe + # create + # delete + + # [configs] + # alter + # IncrementalAlterConfigs (not supported yet) + + # [partitions] + # create + # alter-reassignments (AlterPartitionReassignments - not supported yet) + # list-reassignments (ListPartitionReassignments - not supported yet) + + # [records] + # delete + + # [consumer-groups] + # remove-members (not supported yet) + # delete-offsets (not supported yet) + # alter-offsets (not supported yet) + + # [offsets] + # list (not supported yet) + # delete (OffsetDelete - not supported yet) + + # leader-election + # perform_leader_election + + # [log-dirs] + # describe (currently broken) + # alter (AlterReplicaLogDirs - not supported yet) + + # [client-quotas] + # describe (DescribeClientQuotas - not supported yet) + # alter (AlterClientQuotas - not supported yet) + + # DescribeQuorum (not supported yet) + + # [producers] + # describe (DescribeProducers - not supported yet) + + # [transactions] + # describe (DescribeTransactions - not supported yet) + # list (ListTransactions - not supported yet) + # abort (not supported yet) + + # [topics] + # describe-partitions (DescribeTopicPartitions - not supported yet) + + # [cluster] + # describe-features (DescribeFeatures - not supported yet) + # update-features (UpdateFeatures - not supported yet) + # version + # api-versions + + + diff --git a/kafka/cli/admin/cluster/__init__.py b/kafka/cli/admin/cluster/__init__.py new file mode 100644 index 000000000..735228565 --- /dev/null +++ b/kafka/cli/admin/cluster/__init__.py @@ -0,0 +1,16 @@ +from __future__ import absolute_import + +import sys + +from .describe import DescribeCluster + + +class ClusterSubCommand: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('cluster', help='Manage Kafka Cluster') + commands = parser.add_subparsers() + for cmd in [DescribeCluster]: + cmd.add_subparser(commands) + parser.set_defaults(command=lambda *_args: parser.print_help() or sys.exit(2)) diff --git a/kafka/cli/admin/cluster/describe.py b/kafka/cli/admin/cluster/describe.py new file mode 100644 index 000000000..6a2ff06e5 --- /dev/null +++ b/kafka/cli/admin/cluster/describe.py @@ -0,0 +1,9 @@ +from __future__ import absolute_import + + +class DescribeCluster: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('describe', help='Describe Kafka Cluster') + parser.set_defaults(command=lambda cli, _args: cli.describe_cluster()) diff --git a/kafka/cli/admin/configs/__init__.py b/kafka/cli/admin/configs/__init__.py new file mode 100644 index 000000000..7ec6d1042 --- /dev/null +++ b/kafka/cli/admin/configs/__init__.py @@ -0,0 +1,16 @@ +from __future__ import absolute_import + +import sys + +from .describe import DescribeConfigs + + +class ConfigsSubCommand: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('configs', help='Manage Kafka Configuration') + commands = parser.add_subparsers() + for cmd in [DescribeConfigs]: + cmd.add_subparser(commands) + parser.set_defaults(command=lambda *_args: parser.print_help() or sys.exit(2)) diff --git a/kafka/cli/admin/configs/describe.py b/kafka/cli/admin/configs/describe.py new file mode 100644 index 000000000..3ff366667 --- /dev/null +++ b/kafka/cli/admin/configs/describe.py @@ -0,0 +1,24 @@ +from __future__ import absolute_import + +from kafka.admin.config_resource import ConfigResource + + +class DescribeConfigs: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('describe', help='Describe Kafka Configs') + parser.add_argument('-t', '--topic', type=str, action='append', dest='topics', default=[]) + parser.add_argument('-b', '--broker', type=str, action='append', dest='brokers', default=[]) + parser.set_defaults(command=cls.command) + + @classmethod + def command(cls, client, args): + resources = [] + for topic in args.topics: + resources.append(ConfigResource('TOPIC', topic)) + for broker in args.brokers: + resources.append(ConfigResource('BROKER', broker)) + + response = client.describe_configs(resources) + return list(zip([(r.resource_type.name, r.name) for r in resources], [{str(vals[0]): vals[1] for vals in r.resources[0][4]} for r in response])) diff --git a/kafka/cli/admin/consumer_groups/__init__.py b/kafka/cli/admin/consumer_groups/__init__.py new file mode 100644 index 000000000..cfb1bdb4f --- /dev/null +++ b/kafka/cli/admin/consumer_groups/__init__.py @@ -0,0 +1,19 @@ +from __future__ import absolute_import + +import sys + +from .delete import DeleteConsumerGroups +from .describe import DescribeConsumerGroups +from .list import ListConsumerGroups +from .list_offsets import ListConsumerGroupOffsets + + +class ConsumerGroupsSubCommand: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('consumer-groups', help='Manage Kafka Consumer Groups') + commands = parser.add_subparsers() + for cmd in [ListConsumerGroups, DescribeConsumerGroups, ListConsumerGroupOffsets, DeleteConsumerGroups]: + cmd.add_subparser(commands) + parser.set_defaults(command=lambda *_args: parser.print_help() or sys.exit(2)) diff --git a/kafka/cli/admin/consumer_groups/delete.py b/kafka/cli/admin/consumer_groups/delete.py new file mode 100644 index 000000000..5724ae551 --- /dev/null +++ b/kafka/cli/admin/consumer_groups/delete.py @@ -0,0 +1,10 @@ +from __future__ import absolute_import + + +class DeleteConsumerGroups: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('delete', help='Delete Consumer Groups') + parser.add_argument('-g', '--group-id', type=str, action='append', dest='groups', required=True) + parser.set_defaults(command=lambda cli, args: cli.delete_consumer_groups(args.groups)) diff --git a/kafka/cli/admin/consumer_groups/describe.py b/kafka/cli/admin/consumer_groups/describe.py new file mode 100644 index 000000000..02298e9a4 --- /dev/null +++ b/kafka/cli/admin/consumer_groups/describe.py @@ -0,0 +1,10 @@ +from __future__ import absolute_import + + +class DescribeConsumerGroups: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('describe', help='Describe Consumer Groups') + parser.add_argument('-g', '--group-id', type=str, action='append', dest='groups', required=True) + parser.set_defaults(command=lambda cli, args: cli.describe_consumer_groups(args.groups)) diff --git a/kafka/cli/admin/consumer_groups/list.py b/kafka/cli/admin/consumer_groups/list.py new file mode 100644 index 000000000..6c02f3bee --- /dev/null +++ b/kafka/cli/admin/consumer_groups/list.py @@ -0,0 +1,9 @@ +from __future__ import absolute_import + + +class ListConsumerGroups: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('list', help='List Consumer Groups') + parser.set_defaults(command=lambda cli, _args: cli.list_consumer_groups()) diff --git a/kafka/cli/admin/consumer_groups/list_offsets.py b/kafka/cli/admin/consumer_groups/list_offsets.py new file mode 100644 index 000000000..7c05c5aae --- /dev/null +++ b/kafka/cli/admin/consumer_groups/list_offsets.py @@ -0,0 +1,10 @@ +from __future__ import absolute_import + + +class ListConsumerGroupOffsets: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('list-offsets', help='List Offsets for Consumer Group') + parser.add_argument('-g', '--group-id', type=str, required=True) + parser.set_defaults(command=lambda cli, args: cli.list_consumer_group_offsets(args.group_id)) diff --git a/kafka/cli/admin/log_dirs/__init__.py b/kafka/cli/admin/log_dirs/__init__.py new file mode 100644 index 000000000..3a4a94351 --- /dev/null +++ b/kafka/cli/admin/log_dirs/__init__.py @@ -0,0 +1,16 @@ +from __future__ import absolute_import + +import sys + +from .describe import DescribeLogDirs + + +class LogDirsSubCommand: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('log-dirs', help='Manage Kafka Topic/Partition Log Directories') + commands = parser.add_subparsers() + for cmd in [DescribeLogDirs]: + cmd.add_subparser(commands) + parser.set_defaults(command=lambda *_args: parser.print_help() or sys.exit(2)) diff --git a/kafka/cli/admin/log_dirs/describe.py b/kafka/cli/admin/log_dirs/describe.py new file mode 100644 index 000000000..a0c301f82 --- /dev/null +++ b/kafka/cli/admin/log_dirs/describe.py @@ -0,0 +1,9 @@ +from __future__ import absolute_import + + +class DescribeLogDirs: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('describe', help='Get topic log directories for brokers') + parser.set_defaults(command=lambda cli, _args: cli.describe_log_dirs()) diff --git a/kafka/cli/admin/topics/__init__.py b/kafka/cli/admin/topics/__init__.py new file mode 100644 index 000000000..dbf569445 --- /dev/null +++ b/kafka/cli/admin/topics/__init__.py @@ -0,0 +1,19 @@ +from __future__ import absolute_import + +import sys + +from .create import CreateTopic +from .delete import DeleteTopic +from .describe import DescribeTopics +from .list import ListTopics + + +class TopicsSubCommand: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('topics', help='List/Describe/Create/Delete Kafka Topics') + commands = parser.add_subparsers() + for cmd in [ListTopics, DescribeTopics, CreateTopic, DeleteTopic]: + cmd.add_subparser(commands) + parser.set_defaults(command=lambda *_args: parser.print_help() or sys.exit(2)) diff --git a/kafka/cli/admin/topics/create.py b/kafka/cli/admin/topics/create.py new file mode 100644 index 000000000..d033f6b30 --- /dev/null +++ b/kafka/cli/admin/topics/create.py @@ -0,0 +1,18 @@ +from __future__ import absolute_import + +from kafka.admin.new_topic import NewTopic + + +class CreateTopic: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('create', help='Create a Kafka Topic') + parser.add_argument('-t', '--topic', type=str, required=True) + parser.add_argument('--num-partitions', type=int, default=-1) + parser.add_argument('--replication-factor', type=int, default=-1) + parser.set_defaults(command=cls.command) + + @classmethod + def command(cls, client, args): + return client.create_topics([NewTopic(args.topic, args.num_partitions, args.replication_factor)]) diff --git a/kafka/cli/admin/topics/delete.py b/kafka/cli/admin/topics/delete.py new file mode 100644 index 000000000..a88400ef2 --- /dev/null +++ b/kafka/cli/admin/topics/delete.py @@ -0,0 +1,10 @@ +from __future__ import absolute_import + + +class DeleteTopic: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('delete', help='Delete Kafka Topic') + parser.add_argument('-t', '--topic', type=str, required=True) + parser.set_defaults(command=lambda cli, args: cli.delete_topics([args.topic])) diff --git a/kafka/cli/admin/topics/describe.py b/kafka/cli/admin/topics/describe.py new file mode 100644 index 000000000..2e96871d6 --- /dev/null +++ b/kafka/cli/admin/topics/describe.py @@ -0,0 +1,10 @@ +from __future__ import absolute_import + + +class DescribeTopics: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('describe', help='Describe Kafka Topics') + parser.add_argument('-t', '--topic', type=str, action='append', dest='topics') + parser.set_defaults(command=lambda cli, args: cli.describe_topics(args.topics or None)) diff --git a/kafka/cli/admin/topics/list.py b/kafka/cli/admin/topics/list.py new file mode 100644 index 000000000..2dbf3828e --- /dev/null +++ b/kafka/cli/admin/topics/list.py @@ -0,0 +1,9 @@ +from __future__ import absolute_import + + +class ListTopics: + + @classmethod + def add_subparser(cls, subparsers): + parser = subparsers.add_parser('list', help='List Kafka Topics') + parser.set_defaults(command=lambda cli, _args: cli.list_topics()) diff --git a/kafka/cli/consumer/__init__.py b/kafka/cli/consumer/__init__.py new file mode 100644 index 000000000..7b1991075 --- /dev/null +++ b/kafka/cli/consumer/__init__.py @@ -0,0 +1,90 @@ +from __future__ import absolute_import, print_function + +import argparse +import logging + +from kafka import KafkaConsumer + + +def main_parser(): + parser = argparse.ArgumentParser( + prog='python -m kafka.consumer', + description='Kafka console consumer', + ) + parser.add_argument( + '-b', '--bootstrap-servers', type=str, action='append', required=True, + help='host:port for cluster bootstrap servers') + parser.add_argument( + '-t', '--topic', type=str, action='append', dest='topics', required=True, + help='subscribe to topic') + parser.add_argument( + '-g', '--group', type=str, required=True, + help='consumer group') + parser.add_argument( + '-c', '--extra-config', type=str, action='append', + help='additional configuration properties for kafka consumer') + parser.add_argument( + '-l', '--log-level', type=str, + help='logging level, passed to logging.basicConfig') + parser.add_argument( + '-f', '--format', type=str, default='str', + help='output format: str|raw|full') + parser.add_argument( + '--encoding', type=str, default='utf-8', help='encoding to use for str output decode()') + return parser + + +_LOGGING_LEVELS = {'NOTSET': 0, 'DEBUG': 10, 'INFO': 20, 'WARNING': 30, 'ERROR': 40, 'CRITICAL': 50} + + +def build_kwargs(props): + kwargs = {} + for prop in props or []: + k, v = prop.split('=') + try: + v = int(v) + except ValueError: + pass + if v == 'None': + v = None + elif v == 'False': + v = False + elif v == 'True': + v = True + kwargs[k] = v + return kwargs + + +def run_cli(args=None): + parser = main_parser() + config = parser.parse_args(args) + if config.log_level: + logging.basicConfig(level=_LOGGING_LEVELS[config.log_level.upper()]) + if config.format not in ('str', 'raw', 'full'): + raise ValueError('Unrecognized format: %s' % config.format) + logger = logging.getLogger(__name__) + + kwargs = build_kwargs(config.extra_config) + consumer = KafkaConsumer(bootstrap_servers=config.bootstrap_servers, group_id=config.group, **kwargs) + consumer.subscribe(config.topics) + try: + for m in consumer: + if config.format == 'str': + print(m.value.decode(config.encoding)) + elif config.format == 'full': + print(m) + else: + print(m.value) + except KeyboardInterrupt: + logger.info('Bye!') + return 0 + except Exception: + logger.exception('Error!') + return 1 + finally: + consumer.close() + + +if __name__ == '__main__': + import sys + sys.exit(run_cli()) diff --git a/kafka/cli/producer/__init__.py b/kafka/cli/producer/__init__.py new file mode 100644 index 000000000..186eae1c3 --- /dev/null +++ b/kafka/cli/producer/__init__.py @@ -0,0 +1,85 @@ +from __future__ import absolute_import, print_function + +import argparse +import logging + +from kafka import KafkaProducer + + +def main_parser(): + parser = argparse.ArgumentParser( + prog='python -m kafka.producer', + description='Kafka console producer', + ) + parser.add_argument( + '-b', '--bootstrap-servers', type=str, action='append', required=True, + help='host:port for cluster bootstrap servers') + parser.add_argument( + '-t', '--topic', type=str, required=True, + help='publish to topic') + parser.add_argument( + '-c', '--extra-config', type=str, action='append', + help='additional configuration properties for kafka producer') + parser.add_argument( + '-l', '--log-level', type=str, + help='logging level, passed to logging.basicConfig') + parser.add_argument( + '--encoding', type=str, default='utf-8', + help='byte encoding for produced messages') + return parser + + +_LOGGING_LEVELS = {'NOTSET': 0, 'DEBUG': 10, 'INFO': 20, 'WARNING': 30, 'ERROR': 40, 'CRITICAL': 50} + + +def build_kwargs(props): + kwargs = {} + for prop in props or []: + k, v = prop.split('=') + try: + v = int(v) + except ValueError: + pass + if v == 'None': + v = None + elif v == 'False': + v = False + elif v == 'True': + v = True + kwargs[k] = v + return kwargs + + +def run_cli(args=None): + parser = main_parser() + config = parser.parse_args(args) + if config.log_level: + logging.basicConfig(level=_LOGGING_LEVELS[config.log_level.upper()]) + logger = logging.getLogger(__name__) + + kwargs = build_kwargs(config.extra_config) + producer = KafkaProducer(bootstrap_servers=config.bootstrap_servers, **kwargs) + + def log_result(res_or_err): + if isinstance(res_or_err, Exception): + logger.error("Error producing message", exc_info=res_or_err) + else: + logger.info("Message produced: %s", res_or_err) + + try: + while True: + value = input() + producer.send(config.topic, value=value.encode(config.encoding)).add_both(log_result) + except KeyboardInterrupt: + logger.info('Bye!') + return 0 + except Exception: + logger.exception('Error!') + return 1 + finally: + producer.close() + + +if __name__ == '__main__': + import sys + sys.exit(run_cli()) diff --git a/kafka/consumer/__main__.py b/kafka/consumer/__main__.py new file mode 100644 index 000000000..0356a1aae --- /dev/null +++ b/kafka/consumer/__main__.py @@ -0,0 +1,7 @@ +from __future__ import absolute_import + +import sys + +from kafka.cli.consumer import run_cli + +sys.exit(run_cli()) diff --git a/kafka/producer/__main__.py b/kafka/producer/__main__.py new file mode 100644 index 000000000..e5fd1b1d1 --- /dev/null +++ b/kafka/producer/__main__.py @@ -0,0 +1,7 @@ +from __future__ import absolute_import + +import sys + +from kafka.cli.producer import run_cli + +sys.exit(run_cli()) diff --git a/test/test_admin.py b/test/test_admin.py index cdb74242e..e6f7937a7 100644 --- a/test/test_admin.py +++ b/test/test_admin.py @@ -58,18 +58,34 @@ def test_acl_resource(): ) def test_new_topic(): - with pytest.raises(IllegalArgumentError): - _bad_topic = kafka.admin.NewTopic('foo', -1, -1) - with pytest.raises(IllegalArgumentError): - _bad_topic = kafka.admin.NewTopic('foo', 1, -1) - with pytest.raises(IllegalArgumentError): - _bad_topic = kafka.admin.NewTopic('foo', 1, 1, {1: [1, 1, 1]}) + good_topic = kafka.admin.NewTopic('foo') + assert good_topic.name == 'foo' + assert good_topic.num_partitions == -1 + assert good_topic.replication_factor == -1 + assert good_topic.replica_assignments == {} + assert good_topic.topic_configs == {} + + good_topic = kafka.admin.NewTopic('foo', 1) + assert good_topic.name == 'foo' + assert good_topic.num_partitions == 1 + assert good_topic.replication_factor == -1 + assert good_topic.replica_assignments == {} + assert good_topic.topic_configs == {} + + good_topic = kafka.admin.NewTopic('foo', 1, 1, {1: [1, 1, 1]}) + assert good_topic.name == 'foo' + assert good_topic.num_partitions == 1 + assert good_topic.replication_factor == 1 + assert good_topic.replica_assignments == {1: [1, 1, 1]} + assert good_topic.topic_configs == {} + good_topic = kafka.admin.NewTopic('foo', 1, 2) assert good_topic.name == 'foo' assert good_topic.num_partitions == 1 assert good_topic.replication_factor == 2 assert good_topic.replica_assignments == {} assert good_topic.topic_configs == {} + good_topic = kafka.admin.NewTopic('bar', -1, -1, {1: [1, 2, 3]}, {'key': 'value'}) assert good_topic.name == 'bar' assert good_topic.num_partitions == -1 From 7b565ae5c30a89bc78ef80658b78a86e6a34bcb2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 15:55:49 -0700 Subject: [PATCH 03/51] Drop unused imports (#2651) --- kafka/consumer/group.py | 2 +- kafka/protocol/parser.py | 2 +- test/integration/test_admin_integration.py | 2 +- test/test_assignors.py | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 4eb9e2ab4..5ac14ebfc 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -5,7 +5,7 @@ import socket import time -from kafka.errors import KafkaConfigurationError, KafkaTimeoutError, UnsupportedVersionError +from kafka.errors import KafkaConfigurationError, UnsupportedVersionError from kafka.vendor import six diff --git a/kafka/protocol/parser.py b/kafka/protocol/parser.py index 4bc427330..8683a952a 100644 --- a/kafka/protocol/parser.py +++ b/kafka/protocol/parser.py @@ -6,7 +6,7 @@ import kafka.errors as Errors from kafka.protocol.find_coordinator import FindCoordinatorResponse from kafka.protocol.frame import KafkaBytes -from kafka.protocol.types import Int32, TaggedFields +from kafka.protocol.types import Int32 from kafka.version import __version__ log = logging.getLogger(__name__) diff --git a/test/integration/test_admin_integration.py b/test/integration/test_admin_integration.py index 1c38400e7..5292080bb 100644 --- a/test/integration/test_admin_integration.py +++ b/test/integration/test_admin_integration.py @@ -9,7 +9,7 @@ from kafka.admin import ( ACLFilter, ACLOperation, ACLPermissionType, ResourcePattern, ResourceType, ACL, ConfigResource, ConfigResourceType) from kafka.errors import ( - BrokerResponseError, KafkaError, NoError, CoordinatorNotAvailableError, NonEmptyGroupError, + BrokerResponseError, NoError, CoordinatorNotAvailableError, NonEmptyGroupError, GroupIdNotFoundError, OffsetOutOfRangeError, UnknownTopicOrPartitionError) diff --git a/test/test_assignors.py b/test/test_assignors.py index 858ef426d..91d7157d6 100644 --- a/test/test_assignors.py +++ b/test/test_assignors.py @@ -9,8 +9,8 @@ from kafka.structs import TopicPartition from kafka.coordinator.assignors.range import RangePartitionAssignor from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor -from kafka.coordinator.assignors.sticky.sticky_assignor import StickyPartitionAssignor, StickyAssignorUserDataV1 -from kafka.coordinator.protocol import ConsumerProtocolMemberAssignment, ConsumerProtocolMemberMetadata +from kafka.coordinator.assignors.sticky.sticky_assignor import StickyPartitionAssignor +from kafka.coordinator.protocol import ConsumerProtocolMemberAssignment from kafka.vendor import six From c95dad942e6d68aa4ff4e197a16f848ca82ab0a8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 16:12:45 -0700 Subject: [PATCH 04/51] Fix coordinator lock contention during close() (#2652) --- kafka/coordinator/base.py | 35 ++++++++++++++++++----------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 0eb7f0eec..5e1f72621 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -857,14 +857,12 @@ def _disable_heartbeat_thread(self): self._heartbeat_thread.disable() def _close_heartbeat_thread(self, timeout_ms=None): - with self._lock: - if self._heartbeat_thread is not None: - heartbeat_log.info('Stopping heartbeat thread') - try: - self._heartbeat_thread.close(timeout_ms=timeout_ms) - except ReferenceError: - pass - self._heartbeat_thread = None + if self._heartbeat_thread is not None: + try: + self._heartbeat_thread.close(timeout_ms=timeout_ms) + except ReferenceError: + pass + self._heartbeat_thread = None def __del__(self): try: @@ -1047,17 +1045,20 @@ def disable(self): self.enabled = False def close(self, timeout_ms=None): - if self.closed: - return - self.closed = True + with self.coordinator._lock: + if self.closed: + return - # Generally this should not happen - close() is triggered - # by the coordinator. But in some cases GC may close the coordinator - # from within the heartbeat thread. - if threading.current_thread() == self: - return + heartbeat_log.info('Stopping heartbeat thread') + self.closed = True - with self.coordinator._lock: + # Generally this should not happen - close() is triggered + # by the coordinator. But in some cases GC may close the coordinator + # from within the heartbeat thread. + if threading.current_thread() == self: + return + + # Notify coordinator lock to wake thread from sleep/lock.wait self.coordinator._lock.notify() if self.is_alive(): From 8d75d9a04561e272b7b470d3534c43456fde96a0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 16:13:08 -0700 Subject: [PATCH 05/51] KIP-345 Consumer group static membership (#2625) --- kafka/admin/client.py | 8 +- kafka/consumer/group.py | 24 +++ kafka/coordinator/assignors/abstract.py | 5 +- kafka/coordinator/assignors/range.py | 27 +-- kafka/coordinator/assignors/roundrobin.py | 25 ++- .../assignors/sticky/sticky_assignor.py | 17 +- kafka/coordinator/base.py | 171 +++++++++++++----- kafka/coordinator/consumer.py | 60 +++++- kafka/coordinator/protocol.py | 16 +- kafka/coordinator/subscription.py | 37 ++++ kafka/protocol/commit.py | 39 +++- kafka/protocol/group.py | 138 +++++++++++++- test/integration/test_admin_integration.py | 2 +- test/test_assignors.py | 91 +++++----- test/test_coordinator.py | 34 ++-- 15 files changed, 520 insertions(+), 174 deletions(-) create mode 100644 kafka/coordinator/subscription.py diff --git a/kafka/admin/client.py b/kafka/admin/client.py index c5d8f8636..82edbc868 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -13,7 +13,7 @@ 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 +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0, ConsumerProtocol_v0 import kafka.errors as Errors from kafka.errors import ( IncompatibleBrokerVersion, KafkaConfigurationError, UnknownTopicOrPartitionError, @@ -1242,7 +1242,7 @@ def _describe_consumer_groups_process_response(self, response): 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) + protocol_type_is_consumer = (protocol_type == ConsumerProtocol_v0.PROTOCOL_TYPE or not protocol_type) if isinstance(group_information_field, Array): member_information_list = [] member_schema = group_information_field.array_of @@ -1251,9 +1251,9 @@ def _describe_consumer_groups_process_response(self, response): 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)) + member_information.append(ConsumerProtocolMemberMetadata_v0.decode(member)) elif member_name == 'member_assignment' and member: - member_information.append(ConsumerProtocolMemberAssignment.decode(member)) + member_information.append(ConsumerProtocolMemberAssignment_v0.decode(member)) else: member_information.append(member) member_info_tuple = MemberInformation._make(member_information) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 5ac14ebfc..4b688bc76 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -2,6 +2,7 @@ import copy import logging +import re import socket import time @@ -57,6 +58,14 @@ class KafkaConsumer(six.Iterator): committing offsets. If None, auto-partition assignment (via group coordinator) and offset commits are disabled. Default: None + group_instance_id (str): A unique identifier of the consumer instance + provided by end user. Only non-empty strings are permitted. If set, + the consumer is treated as a static member, which means that only + one instance with this ID is allowed in the consumer group at any + time. This can be used in combination with a larger session timeout + to avoid group rebalances caused by transient unavailability (e.g. + process restarts). If not set, the consumer will join the group as + a dynamic member, which is the traditional behavior. Default: None key_deserializer (callable): Any callable that takes a raw message key and returns a deserialized key. value_deserializer (callable): Any callable that takes a @@ -276,6 +285,7 @@ class KafkaConsumer(six.Iterator): 'bootstrap_servers': 'localhost', 'client_id': 'kafka-python-' + __version__, 'group_id': None, + 'group_instance_id': None, 'key_deserializer': None, 'value_deserializer': None, 'enable_incremental_fetch_sessions': True, @@ -408,6 +418,10 @@ def __init__(self, *topics, **configs): "Request timeout (%s) must be larger than session timeout (%s)" % (self.config['request_timeout_ms'], self.config['session_timeout_ms'])) + if self.config['group_instance_id'] is not None: + if self.config['group_id'] is None: + raise KafkaConfigurationError("group_instance_id requires group_id") + self._subscription = SubscriptionState(self.config['auto_offset_reset']) self._fetcher = Fetcher( self._client, self._subscription, metrics=self._metrics, **self.config) @@ -423,6 +437,16 @@ def __init__(self, *topics, **configs): self._subscription.subscribe(topics=topics) self._client.set_topics(topics) + def _validate_group_instance_id(self, group_instance_id): + if not group_instance_id or not isinstance(group_instance_id, str): + raise KafkaConfigurationError("group_instance_id must be non-empty string") + if group_instance_id in (".", ".."): + raise KafkaConfigurationError("group_instance_id cannot be \".\" or \"..\"") + if len(group_instance_id) > 249: + raise KafkaConfigurationError("group_instance_id can't be longer than 249 characters") + if not re.match(r'^[A-Za-z0-9\.\_\-]+$', group_instance_id): + raise KafkaConfigurationError("group_instance_id is illegal: it contains a character other than ASCII alphanumerics, '.', '_' and '-'") + def bootstrap_connected(self): """Return True if the bootstrap is connected.""" return self._client.bootstrap_connected() diff --git a/kafka/coordinator/assignors/abstract.py b/kafka/coordinator/assignors/abstract.py index a1fef3840..a6fe970d2 100644 --- a/kafka/coordinator/assignors/abstract.py +++ b/kafka/coordinator/assignors/abstract.py @@ -23,8 +23,9 @@ def assign(self, cluster, members): Arguments: cluster (ClusterMetadata): metadata for use in assignment - members (dict of {member_id: MemberMetadata}): decoded metadata for - each member in the group. + members (dict of {member_id: Subscription}): decoded metadata + for each member in the group, including group_instance_id + when available. Returns: dict: {member_id: MemberAssignment} diff --git a/kafka/coordinator/assignors/range.py b/kafka/coordinator/assignors/range.py index 299e39c48..d639d5b75 100644 --- a/kafka/coordinator/assignors/range.py +++ b/kafka/coordinator/assignors/range.py @@ -1,12 +1,13 @@ from __future__ import absolute_import import collections +import itertools import logging from kafka.vendor import six from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor -from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0 log = logging.getLogger(__name__) @@ -32,37 +33,41 @@ class RangePartitionAssignor(AbstractPartitionAssignor): version = 0 @classmethod - def assign(cls, cluster, member_metadata): + def assign(cls, cluster, group_subscriptions): consumers_per_topic = collections.defaultdict(list) - for member, metadata in six.iteritems(member_metadata): - for topic in metadata.subscription: - consumers_per_topic[topic].append(member) + for member_id, subscription in six.iteritems(group_subscriptions): + for topic in subscription.topics: + consumers_per_topic[topic].append((subscription.group_instance_id, member_id)) # construct {member_id: {topic: [partition, ...]}} assignment = collections.defaultdict(dict) + for topic in consumers_per_topic: + # group by static members (True) v dynamic members (False) + grouped = {k: list(g) for k, g in itertools.groupby(consumers_per_topic[topic], key=lambda ids: ids[0] is not None)} + consumers_per_topic[topic] = sorted(grouped.get(True, [])) + sorted(grouped.get(False, [])) # sorted static members first, then sorted dynamic + for topic, consumers_for_topic in six.iteritems(consumers_per_topic): partitions = cluster.partitions_for_topic(topic) if partitions is None: log.warning('No partition metadata for topic %s', topic) continue partitions = sorted(partitions) - consumers_for_topic.sort() partitions_per_consumer = len(partitions) // len(consumers_for_topic) consumers_with_extra = len(partitions) % len(consumers_for_topic) - for i, member in enumerate(consumers_for_topic): + for i, (_group_instance_id, member_id) in enumerate(consumers_for_topic): start = partitions_per_consumer * i start += min(i, consumers_with_extra) length = partitions_per_consumer if not i + 1 > consumers_with_extra: length += 1 - assignment[member][topic] = partitions[start:start+length] + assignment[member_id][topic] = partitions[start:start+length] protocol_assignment = {} - for member_id in member_metadata: - protocol_assignment[member_id] = ConsumerProtocolMemberAssignment( + for member_id in group_subscriptions: + protocol_assignment[member_id] = ConsumerProtocolMemberAssignment_v0( cls.version, sorted(assignment[member_id].items()), b'') @@ -70,7 +75,7 @@ def assign(cls, cluster, member_metadata): @classmethod def metadata(cls, topics): - return ConsumerProtocolMemberMetadata(cls.version, list(topics), b'') + return ConsumerProtocolMemberMetadata_v0(cls.version, list(topics), b'') @classmethod def on_assignment(cls, assignment): diff --git a/kafka/coordinator/assignors/roundrobin.py b/kafka/coordinator/assignors/roundrobin.py index 2d24a5c8b..8d83972cc 100644 --- a/kafka/coordinator/assignors/roundrobin.py +++ b/kafka/coordinator/assignors/roundrobin.py @@ -7,7 +7,7 @@ from kafka.vendor import six from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor -from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0 from kafka.structs import TopicPartition log = logging.getLogger(__name__) @@ -49,10 +49,10 @@ class RoundRobinPartitionAssignor(AbstractPartitionAssignor): version = 0 @classmethod - def assign(cls, cluster, member_metadata): + def assign(cls, cluster, group_subscriptions): all_topics = set() - for metadata in six.itervalues(member_metadata): - all_topics.update(metadata.subscription) + for subscription in six.itervalues(group_subscriptions): + all_topics.update(subscription.topics) all_topic_partitions = [] for topic in all_topics: @@ -67,21 +67,26 @@ def assign(cls, cluster, member_metadata): # construct {member_id: {topic: [partition, ...]}} assignment = collections.defaultdict(lambda: collections.defaultdict(list)) - member_iter = itertools.cycle(sorted(member_metadata.keys())) + # Sort static and dynamic members separately to maintain stable static assignments + ungrouped = [(subscription.group_instance_id, member_id) for member_id, subscription in six.iteritems(group_subscriptions)] + grouped = {k: list(g) for k, g in itertools.groupby(ungrouped, key=lambda ids: ids[0] is not None)} + member_list = sorted(grouped.get(True, [])) + sorted(grouped.get(False, [])) # sorted static members first, then sorted dynamic + member_iter = itertools.cycle(member_list) + for partition in all_topic_partitions: - member_id = next(member_iter) + _group_instance_id, member_id = next(member_iter) # Because we constructed all_topic_partitions from the set of # member subscribed topics, we should be safe assuming that # each topic in all_topic_partitions is in at least one member # subscription; otherwise this could yield an infinite loop - while partition.topic not in member_metadata[member_id].subscription: + while partition.topic not in group_subscriptions[member_id].topics: member_id = next(member_iter) assignment[member_id][partition.topic].append(partition.partition) protocol_assignment = {} - for member_id in member_metadata: - protocol_assignment[member_id] = ConsumerProtocolMemberAssignment( + for member_id in group_subscriptions: + protocol_assignment[member_id] = ConsumerProtocolMemberAssignment_v0( cls.version, sorted(assignment[member_id].items()), b'') @@ -89,7 +94,7 @@ def assign(cls, cluster, member_metadata): @classmethod def metadata(cls, topics): - return ConsumerProtocolMemberMetadata(cls.version, list(topics), b'') + return ConsumerProtocolMemberMetadata_v0(cls.version, list(topics), b'') @classmethod def on_assignment(cls, assignment): diff --git a/kafka/coordinator/assignors/sticky/sticky_assignor.py b/kafka/coordinator/assignors/sticky/sticky_assignor.py index 69f68f564..3166356fe 100644 --- a/kafka/coordinator/assignors/sticky/sticky_assignor.py +++ b/kafka/coordinator/assignors/sticky/sticky_assignor.py @@ -5,7 +5,7 @@ from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor from kafka.coordinator.assignors.sticky.partition_movements import PartitionMovements from kafka.coordinator.assignors.sticky.sorted_set import SortedSet -from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment +from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0 from kafka.coordinator.protocol import Schema from kafka.protocol.struct import Struct from kafka.protocol.types import String, Array, Int32 @@ -66,6 +66,7 @@ class StickyAssignorUserDataV1(Struct): class StickyAssignmentExecutor: def __init__(self, cluster, members): + # a mapping of member_id => StickyAssignorMemberMetadataV1 self.members = members # a mapping between consumers and their assigned partitions that is updated during assignment procedure self.current_assignment = defaultdict(list) @@ -603,7 +604,7 @@ def assign(cls, cluster, members): assignment = {} for member_id in members: - assignment[member_id] = ConsumerProtocolMemberAssignment( + assignment[member_id] = ConsumerProtocolMemberAssignment_v0( cls.version, sorted(executor.get_final_assignment(member_id)), b'' ) return assignment @@ -625,16 +626,16 @@ def parse_member_metadata(cls, metadata): user_data = metadata.user_data if not user_data: return StickyAssignorMemberMetadataV1( - partitions=[], generation=cls.DEFAULT_GENERATION_ID, subscription=metadata.subscription + partitions=[], generation=cls.DEFAULT_GENERATION_ID, subscription=metadata.topics ) try: decoded_user_data = StickyAssignorUserDataV1.decode(user_data) - except Exception as e: + except Exception: # ignore the consumer's previous assignment if it cannot be parsed - log.error("Could not parse member data", e) # pylint: disable=logging-too-many-args + log.exception("Could not parse member data") return StickyAssignorMemberMetadataV1( - partitions=[], generation=cls.DEFAULT_GENERATION_ID, subscription=metadata.subscription + partitions=[], generation=cls.DEFAULT_GENERATION_ID, subscription=metadata.topics ) member_partitions = [] @@ -642,7 +643,7 @@ def parse_member_metadata(cls, metadata): member_partitions.extend([TopicPartition(topic, partition) for partition in partitions]) return StickyAssignorMemberMetadataV1( # pylint: disable=no-member - partitions=member_partitions, generation=decoded_user_data.generation, subscription=metadata.subscription + partitions=member_partitions, generation=decoded_user_data.generation, subscription=metadata.topics ) @classmethod @@ -661,7 +662,7 @@ def _metadata(cls, topics, member_assignment_partitions, generation=-1): partitions_by_topic[topic_partition.topic].append(topic_partition.partition) data = StickyAssignorUserDataV1(list(partitions_by_topic.items()), generation) user_data = data.encode() - return ConsumerProtocolMemberMetadata(cls.version, list(topics), user_data) + return ConsumerProtocolMemberMetadata_v0(cls.version, list(topics), user_data) @classmethod def on_assignment(cls, assignment): diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 5e1f72621..d13ce4abb 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -16,7 +16,10 @@ from kafka.metrics import AnonMeasurable from kafka.metrics.stats import Avg, Count, Max, Rate from kafka.protocol.find_coordinator import FindCoordinatorRequest -from kafka.protocol.group import HeartbeatRequest, JoinGroupRequest, LeaveGroupRequest, SyncGroupRequest, DEFAULT_GENERATION_ID, UNKNOWN_MEMBER_ID +from kafka.protocol.group import ( + HeartbeatRequest, JoinGroupRequest, LeaveGroupRequest, SyncGroupRequest, + DEFAULT_GENERATION_ID, UNKNOWN_MEMBER_ID, GroupMember, +) from kafka.util import Timer log = logging.getLogger('kafka.coordinator') @@ -35,9 +38,12 @@ def __init__(self, generation_id, member_id, protocol): self.member_id = member_id self.protocol = protocol - @property - def is_valid(self): - return self.generation_id != DEFAULT_GENERATION_ID + def has_member_id(self): + """ + True if this generation has a valid member id, False otherwise. + A member might have an id before it becomes part of a group generation. + """ + return self.member_id != UNKNOWN_MEMBER_ID def __eq__(self, other): return (self.generation_id == other.generation_id and @@ -94,6 +100,7 @@ class BaseCoordinator(object): DEFAULT_CONFIG = { 'group_id': 'kafka-python-default-group', + 'group_instance_id': None, 'session_timeout_ms': 10000, 'heartbeat_interval_ms': 3000, 'max_poll_interval_ms': 300000, @@ -135,7 +142,6 @@ def __init__(self, client, **configs): "and session_timeout_ms") self._client = client - self.group_id = self.config['group_id'] self.heartbeat = Heartbeat(**self.config) self._heartbeat_thread = None self._lock = threading.Condition() @@ -152,6 +158,14 @@ def __init__(self, client, **configs): else: self._sensors = None + @property + def group_id(self): + return self.config['group_id'] + + @property + def group_instance_id(self): + return self.config['group_instance_id'] + @abc.abstractmethod def protocol_type(self): """ @@ -205,10 +219,10 @@ def _perform_assignment(self, leader_id, protocol, members): Arguments: leader_id (str): The id of the leader (which is this member) protocol (str): the chosen group protocol (assignment strategy) - members (list): [(member_id, metadata_bytes)] from - JoinGroupResponse. metadata_bytes are associated with the chosen - group protocol, and the Coordinator subclass is responsible for - decoding metadata_bytes based on that protocol. + members (list): [GroupMember] from JoinGroupResponse. + metadata_bytes are associated with the chosen group protocol, + and the Coordinator subclass is responsible for decoding + metadata_bytes based on that protocol. Returns: dict: {member_id: assignment}; assignment must either be bytes @@ -534,7 +548,7 @@ def _send_join_group_request(self): (protocol, metadata if isinstance(metadata, bytes) else metadata.encode()) for protocol, metadata in self.group_protocols() ] - version = self._client.api_version(JoinGroupRequest, max_version=4) + version = self._client.api_version(JoinGroupRequest, max_version=5) if version == 0: request = JoinGroupRequest[version]( self.group_id, @@ -542,12 +556,21 @@ def _send_join_group_request(self): self._generation.member_id, self.protocol_type(), member_metadata) + elif version <= 4: + request = JoinGroupRequest[version]( + self.group_id, + self.config['session_timeout_ms'], + self.config['max_poll_interval_ms'], + self._generation.member_id, + self.protocol_type(), + member_metadata) else: request = JoinGroupRequest[version]( self.group_id, self.config['session_timeout_ms'], self.config['max_poll_interval_ms'], self._generation.member_id, + self.group_instance_id, self.protocol_type(), member_metadata) @@ -621,16 +644,17 @@ def _handle_join_group_response(self, future, send_time, response): future.failure(error_type()) elif error_type in (Errors.InconsistentGroupProtocolError, Errors.InvalidSessionTimeoutError, - Errors.InvalidGroupIdError): + Errors.InvalidGroupIdError, + Errors.GroupAuthorizationFailedError, + Errors.GroupMaxSizeReachedError, + Errors.FencedInstanceIdError): # log the error and re-throw the exception - error = error_type(response) log.error("Attempt to join group %s failed due to fatal error: %s", - self.group_id, error) - future.failure(error) - elif error_type is Errors.GroupAuthorizationFailedError: - log.error("Attempt to join group %s failed due to group authorization error", - self.group_id) - future.failure(error_type(self.group_id)) + self.group_id, error_type.__name__) + if error_type in (Errors.GroupAuthorizationFailedError, Errors.GroupMaxSizeReachedError): + future.failure(error_type(self.group_id)) + else: + future.failure(error_type()) elif error_type is Errors.MemberIdRequiredError: # Broker requires a concrete member id to be allowed to join the group. Update member id # and send another join group request in next cycle. @@ -651,12 +675,20 @@ def _handle_join_group_response(self, future, send_time, response): def _on_join_follower(self): # send follower's sync group with an empty assignment - version = self._client.api_version(SyncGroupRequest, max_version=2) - request = SyncGroupRequest[version]( - self.group_id, - self._generation.generation_id, - self._generation.member_id, - {}) + version = self._client.api_version(SyncGroupRequest, max_version=3) + if version <= 2: + request = SyncGroupRequest[version]( + self.group_id, + self._generation.generation_id, + self._generation.member_id, + []) + else: + request = SyncGroupRequest[version]( + self.group_id, + self._generation.generation_id, + self._generation.member_id, + self.group_instance_id, + []) log.debug("Sending follower SyncGroup for group %s to coordinator %s: %s", self.group_id, self.coordinator_id, request) return self._send_sync_group_request(request) @@ -673,21 +705,32 @@ def _on_join_leader(self, response): Future: resolves to member assignment encoded-bytes """ try: + members = [GroupMember(*member) if response.API_VERSION >= 5 else GroupMember(member[0], None, member[1]) + for member in response.members] group_assignment = self._perform_assignment(response.leader_id, response.group_protocol, - response.members) + members) + for member_id, assignment in six.iteritems(group_assignment): + if not isinstance(assignment, bytes): + group_assignment[member_id] = assignment.encode() + except Exception as e: return Future().failure(e) - version = self._client.api_version(SyncGroupRequest, max_version=2) - request = SyncGroupRequest[version]( - self.group_id, - self._generation.generation_id, - self._generation.member_id, - [(member_id, - assignment if isinstance(assignment, bytes) else assignment.encode()) - for member_id, assignment in six.iteritems(group_assignment)]) - + version = self._client.api_version(SyncGroupRequest, max_version=3) + if version <= 2: + request = SyncGroupRequest[version]( + self.group_id, + self._generation.generation_id, + self._generation.member_id, + group_assignment.items()) + else: + request = SyncGroupRequest[version]( + self.group_id, + self._generation.generation_id, + self._generation.member_id, + self.group_instance_id, + group_assignment.items()) log.debug("Sending leader SyncGroup for group %s to coordinator %s: %s", self.group_id, self.coordinator_id, request) return self._send_sync_group_request(request) @@ -727,6 +770,10 @@ def _handle_sync_group_response(self, future, send_time, response): log.info("SyncGroup for group %s failed due to coordinator" " rebalance", self.group_id) future.failure(error_type(self.group_id)) + elif error_type is Errors.FencedInstanceIdError: + log.error("SyncGroup for group %s failed due to fenced id error: %s", + self.group_id, self.group_instance_id) + future.failure(error_type((self.group_id, self.group_instance_id))) elif error_type in (Errors.UnknownMemberIdError, Errors.IllegalGenerationError): error = error_type() @@ -877,20 +924,28 @@ def close(self, timeout_ms=None): if self.config['api_version'] >= (0, 9): self.maybe_leave_group(timeout_ms=timeout_ms) + def is_dynamic_member(self): + return self.group_instance_id is None or self.config['api_version'] < (2, 3) + def maybe_leave_group(self, timeout_ms=None): """Leave the current group and reset local generation/memberId.""" if self.config['api_version'] < (0, 9): raise Errors.UnsupportedVersionError('Group Coordinator APIs require 0.9+ broker') with self._client._lock, self._lock: - if (not self.coordinator_unknown() - and self.state is not MemberState.UNJOINED - and self._generation.is_valid): + # Starting from 2.3, only dynamic members will send LeaveGroupRequest to the broker, + # consumer with valid group.instance.id is viewed as static member that never sends LeaveGroup, + # and the membership expiration is only controlled by session timeout. + if (self.is_dynamic_member() and not self.coordinator_unknown() + and self.state is not MemberState.UNJOINED and self._generation.has_member_id()): # this is a minimal effort attempt to leave the group. we do not # attempt any resending if the request fails or times out. - log.info('Leaving consumer group (%s).', self.group_id) - version = self._client.api_version(LeaveGroupRequest, max_version=2) - request = LeaveGroupRequest[version](self.group_id, self._generation.member_id) + log.info('Leaving consumer group %s (member %s).', self.group_id, self._generation.member_id) + version = self._client.api_version(LeaveGroupRequest, max_version=3) + if version <= 2: + request = LeaveGroupRequest[version](self.group_id, self._generation.member_id) + else: + request = LeaveGroupRequest[version](self.group_id, [(self._generation.member_id, self.group_instance_id)]) log.debug('Sending LeaveGroupRequest to %s: %s', self.coordinator_id, request) future = self._client.send(self.coordinator_id, request) future.add_callback(self._handle_leave_group_response) @@ -908,6 +963,15 @@ def _handle_leave_group_response(self, response): else: log.error("LeaveGroup request for group %s failed with error: %s", self.group_id, error_type()) + if response.API_VERSION >= 3: + for member_id, group_instance_id, error_code in response.members: + error_type = Errors.for_code(error_code) + if error_type is Errors.NoError: + log.debug("LeaveGroup request for member %s / group instance %s returned successfully", + member_id, group_instance_id) + else: + log.error("LeaveGroup request for member %s / group instance %s failed with error: %s", + member_id, group_instance_id, error_type()) def _send_heartbeat_request(self): """Send a heartbeat request""" @@ -920,10 +984,20 @@ def _send_heartbeat_request(self): e = Errors.NodeNotReadyError(self.coordinator_id) return Future().failure(e) - version = self._client.api_version(HeartbeatRequest, max_version=2) - request = HeartbeatRequest[version](self.group_id, - self._generation.generation_id, - self._generation.member_id) + version = self._client.api_version(HeartbeatRequest, max_version=3) + if version <=2: + request = HeartbeatRequest[version]( + self.group_id, + self._generation.generation_id, + self._generation.member_id, + ) + else: + request = HeartbeatRequest[version]( + self.group_id, + self._generation.generation_id, + self._generation.member_id, + self.group_instance_id, + ) heartbeat_log.debug("Sending HeartbeatRequest to %s: %s", self.coordinator_id, request) future = Future() _f = self._client.send(self.coordinator_id, request) @@ -957,6 +1031,10 @@ def _handle_heartbeat_response(self, future, send_time, response): " current.", self.group_id) self.reset_generation() future.failure(error_type()) + elif error_type is Errors.FencedInstanceIdError: + heartbeat_log.error("Heartbeat failed for group %s due to fenced id error: %s", + self.group_id, self.group_instance_id) + future.failure(error_type((self.group_id, self.group_instance_id))) elif error_type is Errors.UnknownMemberIdError: heartbeat_log.warning("Heartbeat: local member_id was not recognized;" " this consumer needs to re-join") @@ -1179,6 +1257,11 @@ def _handle_heartbeat_failure(self, exception): # then the session timeout may expire before we can rejoin. heartbeat_log.debug('Treating RebalanceInProgressError as successful heartbeat') self.coordinator.heartbeat.received_heartbeat() + elif isinstance(exception, Errors.FencedInstanceIdError): + heartbeat_log.error("Heartbeat thread caught fenced group_instance_id %s error", + self.coordinator.group_instance_id) + self.failed = exception + self.disable() else: heartbeat_log.debug('Heartbeat failure: %s', exception) self.coordinator.heartbeat.fail_heartbeat() diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index dca10ae1a..a7aac4352 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -13,6 +13,7 @@ from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor from kafka.coordinator.assignors.sticky.sticky_assignor import StickyPartitionAssignor from kafka.coordinator.protocol import ConsumerProtocol +from kafka.coordinator.subscription import Subscription import kafka.errors as Errors from kafka.future import Future from kafka.metrics import AnonMeasurable @@ -29,6 +30,7 @@ class ConsumerCoordinator(BaseCoordinator): """This class manages the coordination process with the consumer coordinator.""" DEFAULT_CONFIG = { 'group_id': 'kafka-python-default-group', + 'group_instance_id': None, 'enable_auto_commit': True, 'auto_commit_interval_ms': 5000, 'default_offset_commit_callback': None, @@ -50,6 +52,14 @@ def __init__(self, client, subscription, **configs): group_id (str): name of the consumer group to join for dynamic partition assignment (if enabled), and to use for fetching and committing offsets. Default: 'kafka-python-default-group' + group_instance_id (str): A unique identifier of the consumer instance + provided by end user. Only non-empty strings are permitted. If set, + the consumer is treated as a static member, which means that only + one instance with this ID is allowed in the consumer group at any + time. This can be used in combination with a larger session timeout + to avoid group rebalances caused by transient unavailability (e.g. + process restarts). If not set, the consumer will join the group as + a dynamic member, which is the traditional behavior. Default: None enable_auto_commit (bool): If true the consumer's offset will be periodically committed in the background. Default: True. auto_commit_interval_ms (int): milliseconds between automatic @@ -96,6 +106,7 @@ def __init__(self, client, subscription, **configs): self.next_auto_commit_deadline = None self.completed_offset_commits = collections.deque() self._offset_fetch_futures = dict() + self._async_commit_fenced = False if self.config['default_offset_commit_callback'] is None: self.config['default_offset_commit_callback'] = self._default_offset_commit_callback @@ -140,7 +151,7 @@ def __del__(self): super(ConsumerCoordinator, self).__del__() def protocol_type(self): - return ConsumerProtocol.PROTOCOL_TYPE + return ConsumerProtocol[0].PROTOCOL_TYPE def group_protocols(self): """Returns list of preferred (protocols, metadata)""" @@ -228,7 +239,7 @@ def _on_join_complete(self, generation, member_id, protocol, assignor = self._lookup_assignor(protocol) assert assignor, 'Coordinator selected invalid assignment protocol: %s' % (protocol,) - assignment = ConsumerProtocol.ASSIGNMENT.decode(member_assignment_bytes) + assignment = ConsumerProtocol[0].ASSIGNMENT.decode(member_assignment_bytes) try: self._subscription.assign_from_subscribed(assignment.partitions()) @@ -323,12 +334,15 @@ def time_to_next_poll(self): def _perform_assignment(self, leader_id, assignment_strategy, members): assignor = self._lookup_assignor(assignment_strategy) assert assignor, 'Invalid assignment protocol: %s' % (assignment_strategy,) - member_metadata = {} + member_subscriptions = {} all_subscribed_topics = set() - for member_id, metadata_bytes in members: - metadata = ConsumerProtocol.METADATA.decode(metadata_bytes) - member_metadata[member_id] = metadata - all_subscribed_topics.update(metadata.subscription) # pylint: disable-msg=no-member + for member in members: + subscription = Subscription( + ConsumerProtocol[0].METADATA.decode(member.metadata_bytes), + member.group_instance_id + ) + member_subscriptions[member.member_id] = subscription + all_subscribed_topics.update(subscription.topics) # the leader will begin watching for changes to any of the topics # the group is interested in, which ensures that all metadata changes @@ -346,9 +360,9 @@ def _perform_assignment(self, leader_id, assignment_strategy, members): log.debug("Performing assignment for group %s using strategy %s" " with subscriptions %s", self.group_id, assignor.name, - member_metadata) + member_subscriptions) - assignments = assignor.assign(self._cluster, member_metadata) + assignments = assignor.assign(self._cluster, member_subscriptions) log.debug("Finished assignment for group %s: %s", self.group_id, assignments) @@ -474,6 +488,8 @@ def close(self, autocommit=True, timeout_ms=None): super(ConsumerCoordinator, self).close(timeout_ms=timeout_ms) def _invoke_completed_offset_commit_callbacks(self): + if self._async_commit_fenced: + raise Errors.FencedInstanceIdError("Got fenced exception for group_instance_id %s" % (self.group_instance_id,)) while self.completed_offset_commits: callback, offsets, res_or_exc = self.completed_offset_commits.popleft() callback(offsets, res_or_exc) @@ -525,6 +541,10 @@ def _do_commit_offsets_async(self, offsets, callback=None): callback = self.config['default_offset_commit_callback'] future = self._send_offset_commit_request(offsets) future.add_both(lambda res: self.completed_offset_commits.appendleft((callback, offsets, res))) + def _maybe_set_async_commit_fenced(exc): + if isinstance(exc, Errors.FencedInstanceIdError): + self._async_commit_fenced = True + future.add_errback(_maybe_set_async_commit_fenced) return future def commit_offsets_sync(self, offsets, timeout_ms=None): @@ -623,7 +643,7 @@ def _send_offset_commit_request(self, offsets): for tp, offset in six.iteritems(offsets): offset_data[tp.topic][tp.partition] = offset - version = self._client.api_version(OffsetCommitRequest, max_version=6) + version = self._client.api_version(OffsetCommitRequest, max_version=7) if version > 1 and self._subscription.partitions_auto_assigned(): generation = self.generation_if_stable() else: @@ -701,11 +721,26 @@ def _send_offset_commit_request(self, offsets): ) for partition, offset in six.iteritems(partitions)] ) for topic, partitions in six.iteritems(offset_data)] ) + elif version <= 6: + request = OffsetCommitRequest[version]( + self.group_id, + generation.generation_id, + generation.member_id, + [( + topic, [( + partition, + offset.offset, + offset.leader_epoch, + offset.metadata + ) for partition, offset in six.iteritems(partitions)] + ) for topic, partitions in six.iteritems(offset_data)] + ) else: request = OffsetCommitRequest[version]( self.group_id, generation.generation_id, generation.member_id, + self.group_instance_id, [( topic, [( partition, @@ -779,6 +814,11 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): self.request_rejoin() future.failure(Errors.CommitFailedError(error_type())) return + elif error_type is Errors.FencedInstanceIdError: + log.error("OffsetCommit for group %s failed due to fenced id error: %s", + self.group_id, self.group_instance_id) + future.failure(error_type()) + return elif error_type in (Errors.UnknownMemberIdError, Errors.IllegalGenerationError): # need reset generation and re-join group diff --git a/kafka/coordinator/protocol.py b/kafka/coordinator/protocol.py index 56a390159..bfa1c4695 100644 --- a/kafka/coordinator/protocol.py +++ b/kafka/coordinator/protocol.py @@ -5,14 +5,14 @@ from kafka.structs import TopicPartition -class ConsumerProtocolMemberMetadata(Struct): +class ConsumerProtocolMemberMetadata_v0(Struct): SCHEMA = Schema( ('version', Int16), - ('subscription', Array(String('utf-8'))), + ('topics', Array(String('utf-8'))), ('user_data', Bytes)) -class ConsumerProtocolMemberAssignment(Struct): +class ConsumerProtocolMemberAssignment_v0(Struct): SCHEMA = Schema( ('version', Int16), ('assignment', Array( @@ -26,8 +26,10 @@ def partitions(self): for partition in partitions] -class ConsumerProtocol(object): +class ConsumerProtocol_v0(object): PROTOCOL_TYPE = 'consumer' - ASSIGNMENT_STRATEGIES = ('range', 'roundrobin') - METADATA = ConsumerProtocolMemberMetadata - ASSIGNMENT = ConsumerProtocolMemberAssignment + METADATA = ConsumerProtocolMemberMetadata_v0 + ASSIGNMENT = ConsumerProtocolMemberAssignment_v0 + + +ConsumerProtocol = [ConsumerProtocol_v0] diff --git a/kafka/coordinator/subscription.py b/kafka/coordinator/subscription.py new file mode 100644 index 000000000..ca49c1bc0 --- /dev/null +++ b/kafka/coordinator/subscription.py @@ -0,0 +1,37 @@ +from __future__ import absolute_import + + +class Subscription(object): + __slots__ = ('_metadata', '_group_instance_id') + def __init__(self, metadata, group_instance_id): + self._metadata = metadata + self._group_instance_id = group_instance_id + + @property + def version(self): + return self._metadata.version + + @property + def user_data(self): + return self._metadata.user_data + + @property + def topics(self): + return self._metadata.topics + + # Alias for old interface / name + subscription = topics + + @property + def group_instance_id(self): + return self._group_instance_id + + def encode(self): + return self._metadata.encode() + + def __eq__(self, other): + return ( + isinstance(other, Subscription) and + self._metadata == other._metadata and + self._group_instance_id == other._group_instance_id + ) diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py index a0439e7ef..4cbc43afd 100644 --- a/kafka/protocol/commit.py +++ b/kafka/protocol/commit.py @@ -59,6 +59,12 @@ class OffsetCommitResponse_v6(Response): SCHEMA = OffsetCommitResponse_v5.SCHEMA +class OffsetCommitResponse_v7(Response): + API_KEY = 8 + API_VERSION = 7 + SCHEMA = OffsetCommitResponse_v6.SCHEMA + + class OffsetCommitRequest_v0(Request): API_KEY = 8 API_VERSION = 0 # Zookeeper-backed storage @@ -162,17 +168,34 @@ class OffsetCommitRequest_v6(Request): ) +class OffsetCommitRequest_v7(Request): + API_KEY = 8 + API_VERSION = 7 + RESPONSE_TYPE = OffsetCommitResponse_v7 + SCHEMA = Schema( + ('group_id', String('utf-8')), + ('generation_id', Int32), + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')), # added for static membership / kip-345 + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('leader_epoch', Int32), + ('metadata', String('utf-8')))))) + ) + + OffsetCommitRequest = [ - OffsetCommitRequest_v0, OffsetCommitRequest_v1, - OffsetCommitRequest_v2, OffsetCommitRequest_v3, - OffsetCommitRequest_v4, OffsetCommitRequest_v5, - OffsetCommitRequest_v6, + OffsetCommitRequest_v0, OffsetCommitRequest_v1, OffsetCommitRequest_v2, + OffsetCommitRequest_v3, OffsetCommitRequest_v4, OffsetCommitRequest_v5, + OffsetCommitRequest_v6, OffsetCommitRequest_v7, ] OffsetCommitResponse = [ - OffsetCommitResponse_v0, OffsetCommitResponse_v1, - OffsetCommitResponse_v2, OffsetCommitResponse_v3, - OffsetCommitResponse_v4, OffsetCommitResponse_v5, - OffsetCommitResponse_v6, + OffsetCommitResponse_v0, OffsetCommitResponse_v1, OffsetCommitResponse_v2, + OffsetCommitResponse_v3, OffsetCommitResponse_v4, OffsetCommitResponse_v5, + OffsetCommitResponse_v6, OffsetCommitResponse_v7, ] diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index 74e19c94b..383f3cd2a 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -1,5 +1,7 @@ from __future__ import absolute_import +import collections + from kafka.protocol.api import Request, Response from kafka.protocol.struct import Struct from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String @@ -8,6 +10,9 @@ DEFAULT_GENERATION_ID = -1 UNKNOWN_MEMBER_ID = '' +GroupMember = collections.namedtuple("GroupMember", ["member_id", "group_instance_id", "metadata_bytes"]) +GroupMember.__new__.__defaults__ = (None,) * len(GroupMember._fields) + class JoinGroupResponse_v0(Response): API_KEY = 11 @@ -58,6 +63,23 @@ class JoinGroupResponse_v4(Response): SCHEMA = JoinGroupResponse_v3.SCHEMA +class JoinGroupResponse_v5(Response): + API_KEY = 11 + API_VERSION = 5 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ('generation_id', Int32), + ('group_protocol', String('utf-8')), + ('leader_id', String('utf-8')), + ('member_id', String('utf-8')), + ('members', Array( + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')), + ('member_metadata', Bytes))) + ) + + class JoinGroupRequest_v0(Request): API_KEY = 11 API_VERSION = 0 @@ -110,13 +132,31 @@ class JoinGroupRequest_v4(Request): SCHEMA = JoinGroupRequest_v3.SCHEMA +class JoinGroupRequest_v5(Request): + API_KEY = 11 + API_VERSION = 5 + RESPONSE_TYPE = JoinGroupResponse_v5 + SCHEMA = Schema( + ('group', String('utf-8')), + ('session_timeout', Int32), + ('rebalance_timeout', Int32), + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')), + ('protocol_type', String('utf-8')), + ('group_protocols', Array( + ('protocol_name', String('utf-8')), + ('protocol_metadata', Bytes))) + ) + + JoinGroupRequest = [ JoinGroupRequest_v0, JoinGroupRequest_v1, JoinGroupRequest_v2, - JoinGroupRequest_v3, JoinGroupRequest_v4, + JoinGroupRequest_v3, JoinGroupRequest_v4, JoinGroupRequest_v5, + ] JoinGroupResponse = [ JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v2, - JoinGroupResponse_v3, JoinGroupResponse_v4, + JoinGroupResponse_v3, JoinGroupResponse_v4, JoinGroupResponse_v5, ] @@ -153,6 +193,12 @@ class SyncGroupResponse_v2(Response): SCHEMA = SyncGroupResponse_v1.SCHEMA +class SyncGroupResponse_v3(Response): + API_KEY = 14 + API_VERSION = 3 + SCHEMA = SyncGroupResponse_v2.SCHEMA + + class SyncGroupRequest_v0(Request): API_KEY = 14 API_VERSION = 0 @@ -181,8 +227,29 @@ class SyncGroupRequest_v2(Request): SCHEMA = SyncGroupRequest_v1.SCHEMA -SyncGroupRequest = [SyncGroupRequest_v0, SyncGroupRequest_v1, SyncGroupRequest_v2] -SyncGroupResponse = [SyncGroupResponse_v0, SyncGroupResponse_v1, SyncGroupResponse_v2] +class SyncGroupRequest_v3(Request): + API_KEY = 14 + API_VERSION = 3 + RESPONSE_TYPE = SyncGroupResponse_v3 + SCHEMA = Schema( + ('group', String('utf-8')), + ('generation_id', Int32), + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')), + ('group_assignment', Array( + ('member_id', String('utf-8')), + ('member_metadata', Bytes))) + ) + + +SyncGroupRequest = [ + SyncGroupRequest_v0, SyncGroupRequest_v1, SyncGroupRequest_v2, + SyncGroupRequest_v3, +] +SyncGroupResponse = [ + SyncGroupResponse_v0, SyncGroupResponse_v1, SyncGroupResponse_v2, + SyncGroupResponse_v3, +] class MemberAssignment(Struct): @@ -218,6 +285,12 @@ class HeartbeatResponse_v2(Response): SCHEMA = HeartbeatResponse_v1.SCHEMA +class HeartbeatResponse_v3(Response): + API_KEY = 12 + API_VERSION = 3 + SCHEMA = HeartbeatResponse_v2.SCHEMA + + class HeartbeatRequest_v0(Request): API_KEY = 12 API_VERSION = 0 @@ -243,8 +316,26 @@ class HeartbeatRequest_v2(Request): SCHEMA = HeartbeatRequest_v1.SCHEMA -HeartbeatRequest = [HeartbeatRequest_v0, HeartbeatRequest_v1, HeartbeatRequest_v2] -HeartbeatResponse = [HeartbeatResponse_v0, HeartbeatResponse_v1, HeartbeatResponse_v2] +class HeartbeatRequest_v3(Request): + API_KEY = 12 + API_VERSION = 3 + RESPONSE_TYPE = HeartbeatResponse_v3 + SCHEMA = Schema( + ('group', String('utf-8')), + ('generation_id', Int32), + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')) + ) + + +HeartbeatRequest = [ + HeartbeatRequest_v0, HeartbeatRequest_v1, HeartbeatRequest_v2, + HeartbeatRequest_v3, +] +HeartbeatResponse = [ + HeartbeatResponse_v0, HeartbeatResponse_v1, HeartbeatResponse_v2, + HeartbeatResponse_v3, +] class LeaveGroupResponse_v0(Response): @@ -270,6 +361,19 @@ class LeaveGroupResponse_v2(Response): SCHEMA = LeaveGroupResponse_v1.SCHEMA +class LeaveGroupResponse_v3(Response): + API_KEY = 13 + API_VERSION = 3 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ('members', Array( + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')), + ('error_code', Int16))) + ) + + class LeaveGroupRequest_v0(Request): API_KEY = 13 API_VERSION = 0 @@ -294,5 +398,23 @@ class LeaveGroupRequest_v2(Request): SCHEMA = LeaveGroupRequest_v1.SCHEMA -LeaveGroupRequest = [LeaveGroupRequest_v0, LeaveGroupRequest_v1, LeaveGroupRequest_v2] -LeaveGroupResponse = [LeaveGroupResponse_v0, LeaveGroupResponse_v1, LeaveGroupResponse_v2] +class LeaveGroupRequest_v3(Request): + API_KEY = 13 + API_VERSION = 3 + RESPONSE_TYPE = LeaveGroupResponse_v3 + SCHEMA = Schema( + ('group', String('utf-8')), + ('members', Array( + ('member_id', String('utf-8')), + ('group_instance_id', String('utf-8')))) + ) + + +LeaveGroupRequest = [ + LeaveGroupRequest_v0, LeaveGroupRequest_v1, LeaveGroupRequest_v2, + LeaveGroupRequest_v3, +] +LeaveGroupResponse = [ + LeaveGroupResponse_v0, LeaveGroupResponse_v1, LeaveGroupResponse_v2, + LeaveGroupResponse_v3, +] diff --git a/test/integration/test_admin_integration.py b/test/integration/test_admin_integration.py index 5292080bb..93382c65c 100644 --- a/test/integration/test_admin_integration.py +++ b/test/integration/test_admin_integration.py @@ -231,7 +231,7 @@ def consumer_thread(i, group_id): else: assert(len(consumer_group.members) == 1) for member in consumer_group.members: - assert(member.member_metadata.subscription[0] == topic) + assert(member.member_metadata.topics[0] == topic) assert(member.member_assignment.assignment[0][0] == topic) consumer_groups.add(consumer_group.group) assert(sorted(list(consumer_groups)) == group_id_list) diff --git a/test/test_assignors.py b/test/test_assignors.py index 91d7157d6..f2ecc0325 100644 --- a/test/test_assignors.py +++ b/test/test_assignors.py @@ -10,7 +10,8 @@ from kafka.coordinator.assignors.range import RangePartitionAssignor from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor from kafka.coordinator.assignors.sticky.sticky_assignor import StickyPartitionAssignor -from kafka.coordinator.protocol import ConsumerProtocolMemberAssignment +from kafka.coordinator.protocol import ConsumerProtocolMemberAssignment_v0 +from kafka.coordinator.subscription import Subscription from kafka.vendor import six @@ -34,17 +35,17 @@ def create_cluster(mocker, topics, topics_partitions=None, topic_partitions_lamb def test_assignor_roundrobin(mocker): assignor = RoundRobinPartitionAssignor - member_metadata = { - 'C0': assignor.metadata({'t0', 't1'}), - 'C1': assignor.metadata({'t0', 't1'}), + group_subscriptions = { + 'C0': Subscription(assignor.metadata({'t0', 't1'}), None), + 'C1': Subscription(assignor.metadata({'t0', 't1'}), None), } cluster = create_cluster(mocker, {'t0', 't1'}, topics_partitions={0, 1, 2}) - ret = assignor.assign(cluster, member_metadata) + ret = assignor.assign(cluster, group_subscriptions) expected = { - 'C0': ConsumerProtocolMemberAssignment( + 'C0': ConsumerProtocolMemberAssignment_v0( assignor.version, [('t0', [0, 2]), ('t1', [1])], b''), - 'C1': ConsumerProtocolMemberAssignment( + 'C1': ConsumerProtocolMemberAssignment_v0( assignor.version, [('t0', [1]), ('t1', [0, 2])], b'') } assert ret == expected @@ -56,17 +57,17 @@ def test_assignor_roundrobin(mocker): def test_assignor_range(mocker): assignor = RangePartitionAssignor - member_metadata = { - 'C0': assignor.metadata({'t0', 't1'}), - 'C1': assignor.metadata({'t0', 't1'}), + group_subscriptions = { + 'C0': Subscription(assignor.metadata({'t0', 't1'}), None), + 'C1': Subscription(assignor.metadata({'t0', 't1'}), None), } cluster = create_cluster(mocker, {'t0', 't1'}, topics_partitions={0, 1, 2}) - ret = assignor.assign(cluster, member_metadata) + ret = assignor.assign(cluster, group_subscriptions) expected = { - 'C0': ConsumerProtocolMemberAssignment( + 'C0': ConsumerProtocolMemberAssignment_v0( assignor.version, [('t0', [0, 1]), ('t1', [0, 1])], b''), - 'C1': ConsumerProtocolMemberAssignment( + 'C1': ConsumerProtocolMemberAssignment_v0( assignor.version, [('t0', [2]), ('t1', [2])], b'') } assert ret == expected @@ -102,9 +103,9 @@ def test_sticky_assignor1(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C0': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t0', [0]), ('t1', [1]), ('t3', [0])], b''), - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t0', [1]), ('t2', [0]), ('t3', [1])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0]), ('t2', [1])], b''), + 'C0': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t0', [0]), ('t1', [1]), ('t3', [0])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t0', [1]), ('t2', [0]), ('t3', [1])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0]), ('t2', [1])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -115,10 +116,10 @@ def test_sticky_assignor1(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C0': ConsumerProtocolMemberAssignment( + 'C0': ConsumerProtocolMemberAssignment_v0( StickyPartitionAssignor.version, [('t0', [0]), ('t1', [1]), ('t2', [0]), ('t3', [0])], b'' ), - 'C2': ConsumerProtocolMemberAssignment( + 'C2': ConsumerProtocolMemberAssignment_v0( StickyPartitionAssignor.version, [('t0', [1]), ('t1', [0]), ('t2', [1]), ('t3', [1])], b'' ), } @@ -158,9 +159,9 @@ def test_sticky_assignor2(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C0': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t0', [0])], b''), - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0, 1])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t2', [0, 1, 2])], b''), + 'C0': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t0', [0])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0, 1])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t2', [0, 1, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -171,8 +172,8 @@ def test_sticky_assignor2(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t0', [0]), ('t1', [0, 1])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t2', [0, 1, 2])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t0', [0]), ('t1', [0, 1])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t2', [0, 1, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -187,7 +188,7 @@ def test_sticky_one_consumer_no_topic(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -202,7 +203,7 @@ def test_sticky_one_consumer_nonexisting_topic(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -217,7 +218,7 @@ def test_sticky_one_consumer_one_topic(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -232,7 +233,7 @@ def test_sticky_should_only_assign_partitions_from_subscribed_topics(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -247,7 +248,7 @@ def test_sticky_one_consumer_multiple_topics(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0, 1, 2]), ('t2', [0, 1, 2])], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0, 1, 2]), ('t2', [0, 1, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -263,8 +264,8 @@ def test_sticky_two_consumers_one_topic_one_partition(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [0])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [0])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -280,8 +281,8 @@ def test_sticky_two_consumers_one_topic_two_partitions(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [0])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [1])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [0])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [1])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -299,9 +300,9 @@ def test_sticky_multiple_consumers_mixed_topic_subscriptions(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0, 2])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t2', [0, 1])], b''), - 'C3': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [1])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0, 2])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t2', [0, 1])], b''), + 'C3': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [1])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -316,7 +317,7 @@ def test_sticky_add_remove_consumer_one_topic(mocker): assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), } assert_assignment(assignment, expected_assignment) @@ -356,8 +357,8 @@ def test_sticky_add_remove_topic_two_consumers(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0, 2])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [1])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0, 2])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [1])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -371,8 +372,8 @@ def test_sticky_add_remove_topic_two_consumers(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0, 2]), ('t2', [1])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [1]), ('t2', [0, 2])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0, 2]), ('t2', [1])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [1]), ('t2', [0, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -386,8 +387,8 @@ def test_sticky_add_remove_topic_two_consumers(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C1': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t2', [1])], b''), - 'C2': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t2', [0, 2])], b''), + 'C1': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t2', [1])], b''), + 'C2': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t2', [0, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -601,7 +602,7 @@ def topic_partitions(topic): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t1', [0]), ('t3', list(range(100)))], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t1', [0]), ('t3', list(range(100)))], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -616,7 +617,7 @@ def test_no_exceptions_when_only_subscribed_topic_is_deleted(mocker): sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [('t', [0, 1, 2])], b''), } assert_assignment(sticky_assignment, expected_assignment) @@ -630,7 +631,7 @@ def test_no_exceptions_when_only_subscribed_topic_is_deleted(mocker): cluster = create_cluster(mocker, topics={}, topics_partitions={}) sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) expected_assignment = { - 'C': ConsumerProtocolMemberAssignment(StickyPartitionAssignor.version, [], b''), + 'C': ConsumerProtocolMemberAssignment_v0(StickyPartitionAssignor.version, [], b''), } assert_assignment(sticky_assignment, expected_assignment) diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 4ffe1d28c..b7db5ad19 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -13,13 +13,15 @@ from kafka.coordinator.base import Generation, MemberState, HeartbeatThread from kafka.coordinator.consumer import ConsumerCoordinator from kafka.coordinator.protocol import ( - ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment) + ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0) +from kafka.coordinator.subscription import Subscription import kafka.errors as Errors from kafka.future import Future from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.commit import ( OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse) +from kafka.protocol.group import GroupMember from kafka.protocol.metadata import MetadataResponse from kafka.structs import OffsetAndMetadata, TopicPartition from kafka.util import WeakMethod @@ -73,15 +75,15 @@ def test_group_protocols(coordinator): coordinator._subscription.subscribe(topics=['foobar']) assert coordinator.group_protocols() == [ - ('range', ConsumerProtocolMemberMetadata( + ('range', ConsumerProtocolMemberMetadata_v0( RangePartitionAssignor.version, ['foobar'], b'')), - ('roundrobin', ConsumerProtocolMemberMetadata( + ('roundrobin', ConsumerProtocolMemberMetadata_v0( RoundRobinPartitionAssignor.version, ['foobar'], b'')), - ('sticky', ConsumerProtocolMemberMetadata( + ('sticky', ConsumerProtocolMemberMetadata_v0( StickyPartitionAssignor.version, ['foobar'], b'')), @@ -134,7 +136,7 @@ def test_join_complete(mocker, coordinator): coordinator.config['assignors'] = (assignor,) mocker.spy(assignor, 'on_assignment') assert assignor.on_assignment.call_count == 0 - assignment = ConsumerProtocolMemberAssignment(0, [('foobar', [0, 1])], b'') + assignment = ConsumerProtocolMemberAssignment_v0(0, [('foobar', [0, 1])], b'') coordinator._on_join_complete(0, 'member-foo', 'roundrobin', assignment.encode()) assert assignor.on_assignment.call_count == 1 assignor.on_assignment.assert_called_with(assignment) @@ -148,7 +150,7 @@ def test_join_complete_with_sticky_assignor(mocker, coordinator): mocker.spy(assignor, 'on_generation_assignment') assert assignor.on_assignment.call_count == 0 assert assignor.on_generation_assignment.call_count == 0 - assignment = ConsumerProtocolMemberAssignment(0, [('foobar', [0, 1])], b'') + assignment = ConsumerProtocolMemberAssignment_v0(0, [('foobar', [0, 1])], b'') coordinator._on_join_complete(0, 'member-foo', 'sticky', assignment.encode()) assert assignor.on_assignment.call_count == 1 assert assignor.on_generation_assignment.call_count == 1 @@ -166,7 +168,7 @@ def test_subscription_listener(mocker, coordinator): assert listener.on_partitions_revoked.call_count == 1 listener.on_partitions_revoked.assert_called_with(set([])) - assignment = ConsumerProtocolMemberAssignment(0, [('foobar', [0, 1])], b'') + assignment = ConsumerProtocolMemberAssignment_v0(0, [('foobar', [0, 1])], b'') coordinator._on_join_complete( 0, 'member-foo', 'roundrobin', assignment.encode()) assert listener.on_partitions_assigned.call_count == 1 @@ -184,7 +186,7 @@ def test_subscription_listener_failure(mocker, coordinator): coordinator._on_join_prepare(0, 'member-foo') assert listener.on_partitions_revoked.call_count == 1 - assignment = ConsumerProtocolMemberAssignment(0, [('foobar', [0, 1])], b'') + assignment = ConsumerProtocolMemberAssignment_v0(0, [('foobar', [0, 1])], b'') coordinator._on_join_complete( 0, 'member-foo', 'roundrobin', assignment.encode()) assert listener.on_partitions_assigned.call_count == 1 @@ -192,14 +194,14 @@ def test_subscription_listener_failure(mocker, coordinator): def test_perform_assignment(mocker, coordinator): coordinator._subscription.subscribe(topics=['foo1']) - member_metadata = { - 'member-foo': ConsumerProtocolMemberMetadata(0, ['foo1'], b''), - 'member-bar': ConsumerProtocolMemberMetadata(0, ['foo1'], b'') + group_subscriptions = { + 'member-foo': Subscription(ConsumerProtocolMemberMetadata_v0(0, ['foo1'], b''), None), + 'member-bar': Subscription(ConsumerProtocolMemberMetadata_v0(0, ['foo1'], b''), None), } assignments = { - 'member-foo': ConsumerProtocolMemberAssignment( + 'member-foo': ConsumerProtocolMemberAssignment_v0( 0, [('foo1', [0])], b''), - 'member-bar': ConsumerProtocolMemberAssignment( + 'member-bar': ConsumerProtocolMemberAssignment_v0( 0, [('foo1', [1])], b'') } @@ -208,12 +210,12 @@ def test_perform_assignment(mocker, coordinator): ret = coordinator._perform_assignment( 'member-foo', 'roundrobin', - [(member, metadata.encode()) - for member, metadata in member_metadata.items()]) + [GroupMember(member, None, subscription.encode()) + for member, subscription in group_subscriptions.items()]) assert RoundRobinPartitionAssignor.assign.call_count == 1 RoundRobinPartitionAssignor.assign.assert_called_with( - coordinator._client.cluster, member_metadata) + coordinator._client.cluster, group_subscriptions) assert ret == assignments From 46319a363ecb777b7978e06bd89d002f25c70caf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 16:19:21 -0700 Subject: [PATCH 06/51] cli fixups: producer py2 compat; consumer no stacktrace on KafkaError --- kafka/cli/consumer/__init__.py | 6 +++++- kafka/cli/producer/__init__.py | 7 ++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/kafka/cli/consumer/__init__.py b/kafka/cli/consumer/__init__.py index 7b1991075..801ffb4e7 100644 --- a/kafka/cli/consumer/__init__.py +++ b/kafka/cli/consumer/__init__.py @@ -4,6 +4,7 @@ import logging from kafka import KafkaConsumer +from kafka.errors import KafkaError def main_parser(): @@ -78,7 +79,10 @@ def run_cli(args=None): except KeyboardInterrupt: logger.info('Bye!') return 0 - except Exception: + except KafkaError as e: + logger.error(e) + return 1 + except Exception as e: logger.exception('Error!') return 1 finally: diff --git a/kafka/cli/producer/__init__.py b/kafka/cli/producer/__init__.py index 186eae1c3..9a45d3388 100644 --- a/kafka/cli/producer/__init__.py +++ b/kafka/cli/producer/__init__.py @@ -66,9 +66,14 @@ def log_result(res_or_err): else: logger.info("Message produced: %s", res_or_err) + try: + input_py23 = raw_input + except NameError: + input_py23 = input + try: while True: - value = input() + value = input_py23() producer.send(config.topic, value=value.encode(config.encoding)).add_both(log_result) except KeyboardInterrupt: logger.info('Bye!') From c8d6cdac9e18c4252a12516fe84105bb169fced1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 23 Jun 2025 16:31:03 -0700 Subject: [PATCH 07/51] cli fixups: producer stdin/eof support; consumer unused e var --- kafka/cli/consumer/__init__.py | 2 +- kafka/cli/producer/__init__.py | 8 +++++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/kafka/cli/consumer/__init__.py b/kafka/cli/consumer/__init__.py index 801ffb4e7..fb5b232c9 100644 --- a/kafka/cli/consumer/__init__.py +++ b/kafka/cli/consumer/__init__.py @@ -82,7 +82,7 @@ def run_cli(args=None): except KafkaError as e: logger.error(e) return 1 - except Exception as e: + except Exception: logger.exception('Error!') return 1 finally: diff --git a/kafka/cli/producer/__init__.py b/kafka/cli/producer/__init__.py index 9a45d3388..597235517 100644 --- a/kafka/cli/producer/__init__.py +++ b/kafka/cli/producer/__init__.py @@ -2,6 +2,7 @@ import argparse import logging +import sys from kafka import KafkaProducer @@ -73,7 +74,12 @@ def log_result(res_or_err): try: while True: - value = input_py23() + try: + value = input_py23() + except EOFError: + value = sys.stdin.read().rstrip('\n') + if not value: + return 0 producer.send(config.topic, value=value.encode(config.encoding)).add_both(log_result) except KeyboardInterrupt: logger.info('Bye!') From 8c6c9da820268adea9767a55fb81c9b46db62828 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 24 Jun 2025 15:12:42 -0700 Subject: [PATCH 08/51] Switch protocol code to getattr/setattr from __dict__ (#2654) --- kafka/admin/client.py | 2 +- kafka/protocol/struct.py | 13 ++++++------- test/test_object_conversion.py | 5 ++--- 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 82edbc868..8266c7bfb 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -1236,7 +1236,7 @@ def _describe_consumer_groups_process_response(self, response): for response_field, response_name in zip(response.SCHEMA.fields, response.SCHEMA.names): if isinstance(response_field, Array): described_groups_field_schema = response_field.array_of - described_group = response.__dict__[response_name][0] + described_group = getattr(response, 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): diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index e9da6e6c1..d4adb8832 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -14,12 +14,12 @@ class Struct(AbstractType): def __init__(self, *args, **kwargs): if len(args) == len(self.SCHEMA.fields): for i, name in enumerate(self.SCHEMA.names): - self.__dict__[name] = args[i] + setattr(self, name, args[i]) elif len(args) > 0: raise ValueError('Args must be empty or mirror schema') else: for name in self.SCHEMA.names: - self.__dict__[name] = kwargs.pop(name, None) + setattr(self, name, kwargs.pop(name, None)) if kwargs: raise ValueError('Keyword(s) not in schema %s: %s' % (list(self.SCHEMA.names), @@ -30,7 +30,6 @@ def __init__(self, *args, **kwargs): # causes instances to "leak" to garbage self.encode = WeakMethod(self._encode_self) - @classmethod def encode(cls, item): # pylint: disable=E0202 bits = [] @@ -40,7 +39,7 @@ def encode(cls, item): # pylint: disable=E0202 def _encode_self(self): return self.SCHEMA.encode( - [self.__dict__[name] for name in self.SCHEMA.names] + [getattr(self, name) for name in self.SCHEMA.names] ) @classmethod @@ -52,12 +51,12 @@ def decode(cls, data): def get_item(self, name): if name not in self.SCHEMA.names: raise KeyError("%s is not in the schema" % name) - return self.__dict__[name] + return getattr(self, name) def __repr__(self): key_vals = [] for name, field in zip(self.SCHEMA.names, self.SCHEMA.fields): - key_vals.append('%s=%s' % (name, field.repr(self.__dict__[name]))) + key_vals.append('%s=%s' % (name, field.repr(getattr(self, name)))) return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' def __hash__(self): @@ -67,6 +66,6 @@ def __eq__(self, other): if self.SCHEMA != other.SCHEMA: return False for attr in self.SCHEMA.names: - if self.__dict__[attr] != other.__dict__[attr]: + if getattr(self, attr) != getattr(other, attr): return False return True diff --git a/test/test_object_conversion.py b/test/test_object_conversion.py index a48eb0601..f1d584091 100644 --- a/test/test_object_conversion.py +++ b/test/test_object_conversion.py @@ -21,7 +21,7 @@ class TestClass(superclass): assert tc.get_item('myobject') == 0 with pytest.raises(KeyError): tc.get_item('does-not-exist') - + def test_with_empty_schema(self, superclass): class TestClass(superclass): API_KEY = 0 @@ -86,7 +86,7 @@ class TestClass(superclass): ('subobject', Int16), ('othersubobject', String('utf-8')))), ('notarray', Int16)) - + tc = TestClass( myarray=[[10, 'hello']], notarray=42 @@ -185,7 +185,6 @@ def test_with_metadata_response(): ]] ) tc.encode() # Make sure this object encodes successfully - obj = tc.to_object() From 223c3207cdf3202f8150cb8068780e6d95c70331 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 24 Jun 2025 15:12:58 -0700 Subject: [PATCH 09/51] Remove old/unused kafka.protocol.pickle (#2653) --- kafka/protocol/pickle.py | 35 ----------------------------------- 1 file changed, 35 deletions(-) delete mode 100644 kafka/protocol/pickle.py diff --git a/kafka/protocol/pickle.py b/kafka/protocol/pickle.py deleted file mode 100644 index d6e5fa74f..000000000 --- a/kafka/protocol/pickle.py +++ /dev/null @@ -1,35 +0,0 @@ -from __future__ import absolute_import - -try: - import copyreg # pylint: disable=import-error -except ImportError: - import copy_reg as copyreg # pylint: disable=import-error - -import types - - -def _pickle_method(method): - try: - func_name = method.__func__.__name__ - obj = method.__self__ - cls = method.__self__.__class__ - except AttributeError: - func_name = method.im_func.__name__ - obj = method.im_self - cls = method.im_class - - return _unpickle_method, (func_name, obj, cls) - - -def _unpickle_method(func_name, obj, cls): - for cls in cls.mro(): - try: - func = cls.__dict__[func_name] - except KeyError: - pass - else: - break - return func.__get__(obj, cls) - -# https://bytes.com/topic/python/answers/552476-why-cant-you-pickle-instancemethods -copyreg.pickle(types.MethodType, _pickle_method, _unpickle_method) From 8fb3f961317ec502fb4e9d13454186ee5674c7e8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 26 Jun 2025 16:40:11 -0700 Subject: [PATCH 10/51] python2 fixups (#2655) --- kafka/consumer/fetcher.py | 3 ++- kafka/sasl/gssapi.py | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 1888d38bf..1689b23f1 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -613,7 +613,8 @@ def _fetchable_partitions(self): fetchable = self._subscriptions.fetchable_partitions() # do not fetch a partition if we have a pending fetch response to process # use copy.copy to avoid runtimeerror on mutation from different thread - discard = {fetch.topic_partition for fetch in self._completed_fetches.copy()} + # TODO: switch to deque.copy() with py3 + discard = {fetch.topic_partition for fetch in copy.copy(self._completed_fetches)} current = self._next_partition_records if current: discard.add(current.topic_partition) diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index c8e4f7cac..4785b1b75 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -68,10 +68,10 @@ def receive(self, auth_bytes): # simply set QoP to 'auth' only (first octet). We reuse the max message size proposed # by the server client_flags = self.SASL_QOP_AUTH - server_flags = msg[0] + server_flags = struct.Struct('>b').unpack(msg[0:1])[0] message_parts = [ struct.Struct('>b').pack(client_flags & server_flags), - msg[1:], + msg[1:], # always agree to max message size from server self.auth_id.encode('utf-8'), ] # add authorization identity to the response, and GSS-wrap From 93bcddeb25db36b1bc0d334937ce64bc730bdcf2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 27 Jun 2025 09:48:47 -0700 Subject: [PATCH 11/51] KIP-430: Return Authorized Operations in Describe Responses (#2656) --- kafka/admin/acl_resource.py | 14 ++- kafka/admin/client.py | 189 +++++++++++++++++++----------------- kafka/client_async.py | 8 +- kafka/cluster.py | 4 +- kafka/protocol/admin.py | 8 +- kafka/protocol/metadata.py | 50 +++++++++- kafka/protocol/types.py | 31 ++++++ kafka/util.py | 1 + test/test_protocol.py | 12 ++- test/test_util.py | 1 + 10 files changed, 213 insertions(+), 105 deletions(-) diff --git a/kafka/admin/acl_resource.py b/kafka/admin/acl_resource.py index fd997a10a..8ae1e978d 100644 --- a/kafka/admin/acl_resource.py +++ b/kafka/admin/acl_resource.py @@ -1,5 +1,4 @@ from __future__ import absolute_import -from kafka.errors import IllegalArgumentError # enum in stdlib as of py3.4 try: @@ -8,6 +7,8 @@ # vendored backport module from kafka.vendor.enum34 import IntEnum +from kafka.errors import IllegalArgumentError + class ResourceType(IntEnum): """Type of kafka resource to set ACL for @@ -30,6 +31,7 @@ class ACLOperation(IntEnum): The ANY value is only valid in a filter context """ + UNKNOWN = 0, ANY = 1, ALL = 2, READ = 3, @@ -41,7 +43,9 @@ class ACLOperation(IntEnum): CLUSTER_ACTION = 9, DESCRIBE_CONFIGS = 10, ALTER_CONFIGS = 11, - IDEMPOTENT_WRITE = 12 + IDEMPOTENT_WRITE = 12, + CREATE_TOKENS = 13, + DESCRIBE_TOKENS = 13 class ACLPermissionType(IntEnum): @@ -50,6 +54,7 @@ class ACLPermissionType(IntEnum): The ANY value is only valid in a filter context """ + UNKNOWN = 0, ANY = 1, DENY = 2, ALLOW = 3 @@ -63,6 +68,7 @@ class ACLResourcePatternType(IntEnum): https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs """ + UNKNOWN = 0, ANY = 1, MATCH = 2, LITERAL = 3, @@ -242,3 +248,7 @@ def validate(self): raise IllegalArgumentError( "pattern_type cannot be {} on a concrete ResourcePattern".format(self.pattern_type.name) ) + + +def valid_acl_operations(int_vals): + return set([ACLOperation(v) for v in int_vals if v not in (0, 1, 2)]) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 8266c7bfb..f21ac97f9 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -11,7 +11,7 @@ from kafka.vendor import six from kafka.admin.acl_resource import ACLOperation, ACLPermissionType, ACLFilter, ACL, ResourcePattern, ResourceType, \ - ACLResourcePatternType + ACLResourcePatternType, valid_acl_operations from kafka.client_async import KafkaClient, selectors from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0, ConsumerProtocol_v0 import kafka.errors as Errors @@ -252,30 +252,32 @@ def _validate_timeout(self, timeout_ms): def _refresh_controller_id(self, timeout_ms=30000): """Determine the Kafka cluster controller.""" - version = self._client.api_version(MetadataRequest, max_version=6) - if 1 <= version <= 6: - timeout_at = time.time() + timeout_ms / 1000 - while time.time() < timeout_at: - response = self.send_request(MetadataRequest[version]()) - controller_id = response.controller_id - if controller_id == -1: - log.warning("Controller ID not available, got -1") - time.sleep(1) - continue - # verify the controller is new enough to support our requests - controller_version = self._client.check_version(node_id=controller_id) - if controller_version < (0, 10, 0): - raise IncompatibleBrokerVersion( - "The controller appears to be running Kafka {}. KafkaAdminClient requires brokers >= 0.10.0.0." - .format(controller_version)) - self._controller_id = controller_id - return - else: - raise Errors.NodeNotReadyError('controller') - else: + version = self._client.api_version(MetadataRequest, max_version=8) + if version == 0: raise UnrecognizedBrokerVersion( "Kafka Admin interface cannot determine the controller using MetadataRequest_v{}." .format(version)) + # use defaults for allow_auto_topic_creation / include_authorized_operations in v6+ + request = MetadataRequest[version]() + + timeout_at = time.time() + timeout_ms / 1000 + while time.time() < timeout_at: + response = self.send_request(request) + controller_id = response.controller_id + if controller_id == -1: + log.warning("Controller ID not available, got -1") + time.sleep(1) + continue + # verify the controller is new enough to support our requests + controller_version = self._client.check_version(node_id=controller_id) + if controller_version < (0, 10, 0): + raise IncompatibleBrokerVersion( + "The controller appears to be running Kafka {}. KafkaAdminClient requires brokers >= 0.10.0.0." + .format(controller_version)) + self._controller_id = controller_id + return + else: + raise Errors.NodeNotReadyError('controller') def _find_coordinator_id_request(self, group_id): """Send a FindCoordinatorRequest to a broker. @@ -540,11 +542,20 @@ def delete_topics(self, topics, timeout_ms=None): ) ) + def _process_metadata_response(self, metadata_response): + obj = metadata_response.to_object() + if 'authorized_operations' in obj: + obj['authorized_operations'] = list(map(lambda acl: acl.name, valid_acl_operations(obj['authorized_operations']))) + for t in obj['topics']: + if 'authorized_operations' in t: + t['authorized_operations'] = list(map(lambda acl: acl.name, valid_acl_operations(t['authorized_operations']))) + return obj + def _get_cluster_metadata(self, topics=None, auto_topic_creation=False): """ topics == None means "get all topics" """ - version = self._client.api_version(MetadataRequest, max_version=5) + version = self._client.api_version(MetadataRequest, max_version=8) if version <= 3: if auto_topic_creation: raise IncompatibleBrokerVersion( @@ -553,13 +564,20 @@ def _get_cluster_metadata(self, topics=None, auto_topic_creation=False): .format(self.config['api_version'])) request = MetadataRequest[version](topics=topics) - elif version <= 5: + elif version <= 7: request = MetadataRequest[version]( topics=topics, allow_auto_topic_creation=auto_topic_creation ) + else: + request = MetadataRequest[version]( + topics=topics, + allow_auto_topic_creation=auto_topic_creation, + include_cluster_authorized_operations=True, + include_topic_authorized_operations=True, + ) - return self.send_request(request) + return self._process_metadata_response(self.send_request(request)) def list_topics(self): """Retrieve a list of all topic names in the cluster. @@ -568,8 +586,7 @@ def list_topics(self): A list of topic name strings. """ metadata = self._get_cluster_metadata(topics=None) - obj = metadata.to_object() - return [t['topic'] for t in obj['topics']] + return [t['topic'] for t in metadata['topics']] def describe_topics(self, topics=None): """Fetch metadata for the specified topics or all topics if None. @@ -582,8 +599,7 @@ def describe_topics(self, topics=None): A list of dicts describing each topic (including partition info). """ metadata = self._get_cluster_metadata(topics=topics) - obj = metadata.to_object() - return obj['topics'] + return metadata['topics'] def describe_cluster(self): """ @@ -595,9 +611,8 @@ def describe_cluster(self): A dict with cluster-wide metadata, excluding topic details. """ metadata = self._get_cluster_metadata() - obj = metadata.to_object() - obj.pop('topics') # We have 'describe_topics' for this - return obj + metadata.pop('topics') # We have 'describe_topics' for this + return metadata @staticmethod def _convert_describe_acls_response_to_acls(describe_response): @@ -1094,11 +1109,11 @@ def _get_leader_for_partitions(self, partitions, timeout_ms=None): partitions = set(partitions) topics = set(tp.topic for tp in partitions) - response = self._get_cluster_metadata(topics=topics).to_object() + metadata = self._get_cluster_metadata(topics=topics) leader2partitions = defaultdict(list) valid_partitions = set() - for topic in response.get("topics", ()): + for topic in metadata.get("topics", ()): for partition in topic.get("partitions", ()): t2p = TopicPartition(topic=topic["topic"], partition=partition["partition"]) if t2p in partitions: @@ -1199,7 +1214,7 @@ def delete_records(self, records_to_delete, timeout_ms=None, partition_leader_id # describe delegation_token protocol not yet implemented # Note: send the request to the least_loaded_node() - def _describe_consumer_groups_request(self, group_id, include_authorized_operations=False): + def _describe_consumer_groups_request(self, group_id): """Send a DescribeGroupsRequest to the group's coordinator. Arguments: @@ -1210,74 +1225,69 @@ def _describe_consumer_groups_request(self, group_id, include_authorized_operati """ version = self._client.api_version(DescribeGroupsRequest, max_version=3) if version <= 2: - if include_authorized_operations: - raise IncompatibleBrokerVersion( - "include_authorized_operations requests " - "DescribeGroupsRequest >= v3, which is not " - "supported by Kafka {}".format(version) - ) # Note: KAFKA-6788 A potential optimization is to group the # request per coordinator and send one request with a list of # all consumer groups. Java still hasn't implemented this # because the error checking is hard to get right when some # groups error and others don't. request = DescribeGroupsRequest[version](groups=(group_id,)) - elif version <= 3: + else: request = DescribeGroupsRequest[version]( groups=(group_id,), - include_authorized_operations=include_authorized_operations + include_authorized_operations=True ) return request 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 - for response_field, response_name in zip(response.SCHEMA.fields, response.SCHEMA.names): - if isinstance(response_field, Array): - described_groups_field_schema = response_field.array_of - described_group = getattr(response, 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_v0.PROTOCOL_TYPE or not protocol_type) - if isinstance(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_v0.decode(member)) - elif member_name == 'member_assignment' and member: - member_information.append(ConsumerProtocolMemberAssignment_v0.decode(member)) - else: - member_information.append(member) - member_info_tuple = MemberInformation._make(member_information) - member_information_list.append(member_info_tuple) - described_group_information_list.append(member_information_list) - else: - 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 - if error_type is not Errors.NoError: - raise error_type( - "DescribeGroupsResponse failed with response '{}'." - .format(response)) - else: + if response.API_VERSION > 3: raise NotImplementedError( "Support for DescribeGroupsResponse_v{} has not yet been added to KafkaAdminClient." .format(response.API_VERSION)) + + assert len(response.groups) == 1 + for response_field, response_name in zip(response.SCHEMA.fields, response.SCHEMA.names): + if isinstance(response_field, Array): + described_groups_field_schema = response_field.array_of + described_group = getattr(response, 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_v0.PROTOCOL_TYPE or not protocol_type) + if isinstance(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_v0.decode(member)) + elif member_name == 'member_assignment' and member: + member_information.append(ConsumerProtocolMemberAssignment_v0.decode(member)) + else: + member_information.append(member) + member_info_tuple = MemberInformation._make(member_information) + member_information_list.append(member_info_tuple) + described_group_information_list.append(member_information_list) + else: + described_group_information_list.append(described_group_information) + # Version 3 of the DescribeGroups API introduced the "authorized_operations" field. + if response.API_VERSION >= 3: + described_group_information_list[-1] = list(map(lambda acl: acl.name, valid_acl_operations(described_group_information_list[-1]))) + else: + # TODO: Fix GroupInformation defaults + 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)) return group_description def describe_consumer_groups(self, group_ids, group_coordinator_id=None, include_authorized_operations=False): @@ -1296,9 +1306,6 @@ def describe_consumer_groups(self, group_ids, group_coordinator_id=None, include useful for avoiding extra network round trips if you already know the group coordinator. This is only useful when all the group_ids have the same coordinator, otherwise it will error. Default: None. - include_authorized_operations (bool, optional): Whether or not to include - information about the operations a group is allowed to perform. - Only supported on API version >= v3. Default: False. Returns: A list of group descriptions. For now the group descriptions @@ -1312,7 +1319,7 @@ def describe_consumer_groups(self, group_ids, group_coordinator_id=None, include groups_coordinators = self._find_coordinator_ids(group_ids) requests = [ - (self._describe_consumer_groups_request(group_id, include_authorized_operations), coordinator_id) + (self._describe_consumer_groups_request(group_id), coordinator_id) for group_id, coordinator_id in groups_coordinators.items() ] return self.send_requests(requests, response_fn=self._describe_consumer_groups_process_response) diff --git a/kafka/client_async.py b/kafka/client_async.py index 7d466574f..de20c218d 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -978,15 +978,17 @@ def _maybe_refresh_metadata(self, wakeup=False): if not topics and self.cluster.is_bootstrap(node_id): topics = list(self.config['bootstrap_topics_filter']) - api_version = self.api_version(MetadataRequest, max_version=7) + api_version = self.api_version(MetadataRequest, max_version=8) if self.cluster.need_all_topic_metadata: topics = MetadataRequest[api_version].ALL_TOPICS elif not topics: topics = MetadataRequest[api_version].NO_TOPICS - if api_version >= 4: + if api_version <= 3: + request = MetadataRequest[api_version](topics) + elif api_version <= 7: request = MetadataRequest[api_version](topics, self.config['allow_auto_create_topics']) else: - request = MetadataRequest[api_version](topics) + request = MetadataRequest[api_version](topics, self.config['allow_auto_create_topics'], False, False) log.debug("Sending metadata request %s to node %s", request, node_id) future = self.send(node_id, request, wakeup=wakeup) future.add_callback(self.cluster.update_metadata) diff --git a/kafka/cluster.py b/kafka/cluster.py index ded8c6f96..9e819246e 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -279,8 +279,10 @@ def update_metadata(self, metadata): if metadata.API_VERSION == 0: error_code, topic, partitions = topic_data is_internal = False - else: + elif metadata.API_VERSION <= 7: error_code, topic, is_internal, partitions = topic_data + else: + error_code, topic, is_internal, partitions, _authorized_operations = topic_data if is_internal: _new_internal_topics.add(topic) error_type = Errors.for_code(error_code) diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index 255166801..32b75df4b 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -8,7 +8,7 @@ from kafka.vendor.enum34 import IntEnum from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String, Float64, CompactString, CompactArray, TaggedFields +from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String, Float64, CompactString, CompactArray, TaggedFields, BitField class CreateTopicsResponse_v0(Response): @@ -337,8 +337,8 @@ class DescribeGroupsResponse_v3(Response): ('client_id', String('utf-8')), ('client_host', String('utf-8')), ('member_metadata', Bytes), - ('member_assignment', Bytes)))), - ('authorized_operations', Int32)) + ('member_assignment', Bytes))), + ('authorized_operations', BitField))) ) @@ -368,7 +368,7 @@ class DescribeGroupsRequest_v2(Request): class DescribeGroupsRequest_v3(Request): API_KEY = 15 API_VERSION = 3 - RESPONSE_TYPE = DescribeGroupsResponse_v2 + RESPONSE_TYPE = DescribeGroupsResponse_v3 SCHEMA = Schema( ('groups', Array(String('utf-8'))), ('include_authorized_operations', Boolean) diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py index bb22ba997..eb632371c 100644 --- a/kafka/protocol/metadata.py +++ b/kafka/protocol/metadata.py @@ -1,7 +1,7 @@ from __future__ import absolute_import from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Boolean, Int16, Int32, Schema, String +from kafka.protocol.types import Array, Boolean, Int16, Int32, Schema, String, BitField class MetadataResponse_v0(Response): @@ -164,6 +164,36 @@ class MetadataResponse_v7(Response): ) +class MetadataResponse_v8(Response): + """v8 adds authorized_operations fields""" + API_KEY = 3 + API_VERSION = 8 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('brokers', Array( + ('node_id', Int32), + ('host', String('utf-8')), + ('port', Int32), + ('rack', String('utf-8')))), + ('cluster_id', String('utf-8')), + ('controller_id', Int32), + ('topics', Array( + ('error_code', Int16), + ('topic', String('utf-8')), + ('is_internal', Boolean), + ('partitions', Array( + ('error_code', Int16), + ('partition', Int32), + ('leader', Int32), + ('leader_epoch', Int32), + ('replicas', Array(Int32)), + ('isr', Array(Int32)), + ('offline_replicas', Array(Int32)))), + ('authorized_operations', BitField))), + ('authorized_operations', BitField) + ) + + class MetadataRequest_v0(Request): API_KEY = 3 API_VERSION = 0 @@ -245,13 +275,27 @@ class MetadataRequest_v7(Request): NO_TOPICS = [] +class MetadataRequest_v8(Request): + API_KEY = 3 + API_VERSION = 8 + RESPONSE_TYPE = MetadataResponse_v8 + SCHEMA = Schema( + ('topics', Array(String('utf-8'))), + ('allow_auto_topic_creation', Boolean), + ('include_cluster_authorized_operations', Boolean), + ('include_topic_authorized_operations', Boolean) + ) + ALL_TOPICS = None + NO_TOPICS = [] + + MetadataRequest = [ MetadataRequest_v0, MetadataRequest_v1, MetadataRequest_v2, MetadataRequest_v3, MetadataRequest_v4, MetadataRequest_v5, - MetadataRequest_v6, MetadataRequest_v7, + MetadataRequest_v6, MetadataRequest_v7, MetadataRequest_v8, ] MetadataResponse = [ MetadataResponse_v0, MetadataResponse_v1, MetadataResponse_v2, MetadataResponse_v3, MetadataResponse_v4, MetadataResponse_v5, - MetadataResponse_v6, MetadataResponse_v7, + MetadataResponse_v6, MetadataResponse_v7, MetadataResponse_v8, ] diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 0e3685d73..8949ce471 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -363,3 +363,34 @@ def decode(self, data): return None return [self.array_of.decode(data) for _ in range(length)] + +class BitField(AbstractType): + @classmethod + def decode(cls, data): + return cls.from_32_bit_field(Int32.decode(data)) + + @classmethod + def encode(cls, vals): + # to_32_bit_field returns unsigned val, so we need to + # encode >I to avoid crash if/when byte 31 is set + # (note that decode as signed still works fine) + return struct.Struct('>I').pack(cls.to_32_bit_field(vals)) + + @classmethod + def to_32_bit_field(cls, vals): + value = 0 + for b in vals: + assert 0 <= b < 32 + value |= 1 << b + return value + + @classmethod + def from_32_bit_field(cls, value): + result = set() + count = 0 + while value != 0: + if (value & 1) != 0: + result.add(count) + count += 1 + value = (value & 0xFFFFFFFF) >> 1 + return result diff --git a/kafka/util.py b/kafka/util.py index 658c17d59..6bc4c7051 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -138,3 +138,4 @@ def wrapper(self, *args, **kwargs): return func(self, *args, **kwargs) functools.update_wrapper(wrapper, func) return wrapper + diff --git a/test/test_protocol.py b/test/test_protocol.py index d0cc7ed0a..35ca938e1 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -2,12 +2,14 @@ import io import struct +import pytest + from kafka.protocol.api import RequestHeader from kafka.protocol.fetch import FetchRequest, FetchResponse from kafka.protocol.find_coordinator import FindCoordinatorRequest from kafka.protocol.message import Message, MessageSet, PartialMessage from kafka.protocol.metadata import MetadataRequest -from kafka.protocol.types import Int16, Int32, Int64, String, UnsignedVarInt32, CompactString, CompactArray, CompactBytes +from kafka.protocol.types import Int16, Int32, Int64, String, UnsignedVarInt32, CompactString, CompactArray, CompactBytes, BitField def test_create_message(): @@ -332,3 +334,11 @@ def test_compact_data_structs(): assert CompactBytes.decode(io.BytesIO(b'\x01')) == b'' enc = CompactBytes.encode(b'foo') assert CompactBytes.decode(io.BytesIO(enc)) == b'foo' + + +@pytest.mark.parametrize(('test_set',), [ + (set([0, 1, 5, 10, 31]),), + (set(range(32)),), +]) +def test_bit_field(test_set): + assert BitField.decode(io.BytesIO(BitField.encode(test_set))) == test_set diff --git a/test/test_util.py b/test/test_util.py index 875b252aa..f9e8a2b51 100644 --- a/test/test_util.py +++ b/test/test_util.py @@ -22,3 +22,4 @@ def test_topic_name_validation(topic_name, expectation): with expectation: ensure_valid_topic_name(topic_name) + From 2abf2cd6458dcb069940c04da2711aa41683f4f4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 29 Jun 2025 16:36:14 -0700 Subject: [PATCH 12/51] KIP-207: Add ListOffsetsRequest v5 / handle OffsetNotAvailableError (#2657) --- kafka/consumer/fetcher.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 1689b23f1..d57bc4786 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -508,7 +508,7 @@ def _group_list_offset_requests(self, timestamps): return dict(timestamps_by_node) def _send_list_offsets_request(self, node_id, timestamps_and_epochs): - version = self._client.api_version(ListOffsetsRequest, max_version=4) + version = self._client.api_version(ListOffsetsRequest, max_version=5) if self.config['isolation_level'] == 'read_committed' and version < 2: raise Errors.UnsupportedVersionError('read_committed isolation level requires ListOffsetsRequest >= v2') by_topic = collections.defaultdict(list) @@ -521,14 +521,14 @@ def _send_list_offsets_request(self, node_id, timestamps_and_epochs): data = (tp.partition, timestamp, 1) by_topic[tp.topic].append(data) - if version <= 1: + if version >= 2: request = ListOffsetsRequest[version]( -1, + self._isolation_level, list(six.iteritems(by_topic))) else: request = ListOffsetsRequest[version]( -1, - self._isolation_level, list(six.iteritems(by_topic))) # Client returns a future that only fails on network issues @@ -588,7 +588,9 @@ def _handle_list_offsets_response(self, future, response): " message format version is before 0.10.0", partition) elif error_type in (Errors.NotLeaderForPartitionError, Errors.ReplicaNotAvailableError, - Errors.KafkaStorageError): + Errors.KafkaStorageError, + Errors.OffsetNotAvailableError, + Errors.LeaderNotAvailableError): log.debug("Attempt to fetch offsets for partition %s failed due" " to %s, retrying.", error_type.__name__, partition) partitions_to_retry.add(partition) From 2297fa156a0b9b284a30b1d888be601cd3323bad Mon Sep 17 00:00:00 2001 From: Xeus-CC Date: Wed, 2 Jul 2025 05:30:08 +1200 Subject: [PATCH 13/51] Fix spelling mistake in KafkaConsumer docs (#2659) --- kafka/consumer/group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 4b688bc76..dcb322b84 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -132,7 +132,7 @@ class KafkaConsumer(six.Iterator): be disabled in cases seeking extreme performance. Default: True isolation_level (str): Configure KIP-98 transactional consumer by setting to 'read_committed'. This will cause the consumer to - skip records from aborted tranactions. Default: 'read_uncommitted' + skip records from aborted transactions. Default: 'read_uncommitted' allow_auto_create_topics (bool): Enable/disable auto topic creation on metadata request. Only available with api_version >= (0, 11). Default: True From e91b036abb2333ef2e28395a02eb95e2d15042ee Mon Sep 17 00:00:00 2001 From: llk89 Date: Wed, 2 Jul 2025 01:32:09 +0800 Subject: [PATCH 14/51] Fix KafkaProducer broken method names (#2660) --- kafka/producer/record_accumulator.py | 4 ++-- kafka/producer/transaction_manager.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 77d48d84f..3a4e60146 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -430,7 +430,7 @@ def ready(self, cluster, now=None): expired = bool(waited_time >= time_to_wait) sendable = (full or expired or self._closed or - self._flush_in_progress()) + self.flush_in_progress()) if sendable and not backing_off: ready_nodes.add(leader) @@ -563,7 +563,7 @@ def deallocate(self, batch): """Deallocate the record batch.""" self._incomplete.remove(batch) - def _flush_in_progress(self): + def flush_in_progress(self): """Are there any threads currently waiting on a flush?""" return self._flushes_in_progress.get() > 0 diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index 7302eb00e..5d69ddc97 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -553,11 +553,11 @@ def producer_epoch(self): return self.transaction_manager.producer_id_and_epoch.epoch def fatal_error(self, exc): - self.transaction_manager._transition_to_fatal_error(exc) + self.transaction_manager.transition_to_fatal_error(exc) self._result.done(error=exc) def abortable_error(self, exc): - self.transaction_manager._transition_to_abortable_error(exc) + self.transaction_manager.transition_to_abortable_error(exc) self._result.done(error=exc) def fail(self, exc): From 0a8713003a53048e8732a1ce5b9f6cd383f54b6f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 1 Jul 2025 10:41:53 -0700 Subject: [PATCH 15/51] Update changelog for 2.2 patch releases + bump version for 2.3.x releases --- CHANGES.md | 12 ++++++++++++ docs/changelog.rst | 17 +++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 30 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 372aebfc6..9cab2c837 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,15 @@ +# 2.2.15 (July 1, 2025) + +Fixes +* Fix KafkaProducer broken method names (llk89 / #2660) +* Fix spelling mistake in KafkaConsumer docs (Xeus-CC / #2659) + +# 2.2.14 (June 27, 2025) + +Fixes +* python2 fixups (#2655) +* Fix coordinator lock contention during close() (#2652) + # 2.2.13 (June 20, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 430f8a512..844bdd9b6 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,23 @@ Changelog ========= +2.2.15 (July 1, 2025) +##################### + +Fixes +----- +* Fix KafkaProducer broken method names (llk89 / #2660) +* Fix spelling mistake in KafkaConsumer docs (Xeus-CC / #2659) + + +2.2.14 (June 27, 2025) +###################### + +Fixes +* python2 fixups (#2655) +* Fix coordinator lock contention during close() (#2652) + + 2.2.13 (June 20, 2025) ###################### diff --git a/kafka/version.py b/kafka/version.py index 298979870..29e5e08ea 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.13' +__version__ = '2.3.0.dev' From 922767440b670094b8d0c7066d4b1b16f95d1478 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 3 Jul 2025 08:32:14 -0700 Subject: [PATCH 16/51] KIP-467: Augment ProduceResponse error messaging for specific culprit records (#2661) --- kafka/errors.py | 3 + kafka/producer/future.py | 35 ++--- kafka/producer/producer_batch.py | 184 +++++++++++++++++++++++++++ kafka/producer/record_accumulator.py | 148 +-------------------- kafka/producer/sender.py | 101 ++++++++++----- kafka/protocol/produce.py | 4 +- test/test_producer_batch.py | 136 ++++++++++++++++++++ test/test_record_accumulator.py | 102 +-------------- test/test_sender.py | 69 ++++++---- 9 files changed, 456 insertions(+), 326 deletions(-) create mode 100644 kafka/producer/producer_batch.py create mode 100644 test/test_producer_batch.py diff --git a/kafka/errors.py b/kafka/errors.py index ac4eadfec..351e07375 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -15,6 +15,9 @@ def __str__(self): return '{0}: {1}'.format(self.__class__.__name__, super(KafkaError, self).__str__()) + def __eq__(self, other): + return self.__class__ == other.__class__ and self.args == other.args + class Cancelled(KafkaError): retriable = True diff --git a/kafka/producer/future.py b/kafka/producer/future.py index f67db0979..13392a96e 100644 --- a/kafka/producer/future.py +++ b/kafka/producer/future.py @@ -29,32 +29,35 @@ def wait(self, timeout=None): class FutureRecordMetadata(Future): - def __init__(self, produce_future, relative_offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size): + def __init__(self, produce_future, batch_index, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size): super(FutureRecordMetadata, self).__init__() self._produce_future = produce_future # packing args as a tuple is a minor speed optimization - self.args = (relative_offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size) + self.args = (batch_index, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size) produce_future.add_callback(self._produce_success) produce_future.add_errback(self.failure) - def _produce_success(self, offset_and_timestamp): - offset, produce_timestamp_ms = offset_and_timestamp + def _produce_success(self, result): + offset, produce_timestamp_ms, record_exceptions_fn = result # Unpacking from args tuple is minor speed optimization - (relative_offset, timestamp_ms, checksum, + (batch_index, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size) = self.args - # None is when Broker does not support the API (<0.10) and - # -1 is when the broker is configured for CREATE_TIME timestamps - if produce_timestamp_ms is not None and produce_timestamp_ms != -1: - timestamp_ms = produce_timestamp_ms - if offset != -1 and relative_offset is not None: - offset += relative_offset - tp = self._produce_future.topic_partition - metadata = RecordMetadata(tp[0], tp[1], tp, offset, timestamp_ms, - checksum, serialized_key_size, - serialized_value_size, serialized_header_size) - self.success(metadata) + if record_exceptions_fn is not None: + self.failure(record_exceptions_fn(batch_index)) + else: + # None is when Broker does not support the API (<0.10) and + # -1 is when the broker is configured for CREATE_TIME timestamps + if produce_timestamp_ms is not None and produce_timestamp_ms != -1: + timestamp_ms = produce_timestamp_ms + if offset != -1 and batch_index is not None: + offset += batch_index + tp = self._produce_future.topic_partition + metadata = RecordMetadata(tp[0], tp[1], tp, offset, timestamp_ms, + checksum, serialized_key_size, + serialized_value_size, serialized_header_size) + self.success(metadata) def get(self, timeout=None): if not self.is_done and not self._produce_future.wait(timeout): diff --git a/kafka/producer/producer_batch.py b/kafka/producer/producer_batch.py new file mode 100644 index 000000000..8be08f575 --- /dev/null +++ b/kafka/producer/producer_batch.py @@ -0,0 +1,184 @@ +from __future__ import absolute_import, division + +import logging +import time + +try: + # enum in stdlib as of py3.4 + from enum import IntEnum # pylint: disable=import-error +except ImportError: + # vendored backport module + from kafka.vendor.enum34 import IntEnum + +import kafka.errors as Errors +from kafka.producer.future import FutureRecordMetadata, FutureProduceResult + + +log = logging.getLogger(__name__) + + +class FinalState(IntEnum): + ABORTED = 0 + FAILED = 1 + SUCCEEDED = 2 + + +class ProducerBatch(object): + def __init__(self, tp, records, now=None): + now = time.time() if now is None else now + self.max_record_size = 0 + self.created = now + self.drained = None + self.attempts = 0 + self.last_attempt = now + self.last_append = now + self.records = records + self.topic_partition = tp + self.produce_future = FutureProduceResult(tp) + self._retry = False + self._final_state = None + + @property + def final_state(self): + return self._final_state + + @property + def record_count(self): + return self.records.next_offset() + + @property + def producer_id(self): + return self.records.producer_id if self.records else None + + @property + def producer_epoch(self): + return self.records.producer_epoch if self.records else None + + @property + def has_sequence(self): + return self.records.has_sequence if self.records else False + + def try_append(self, timestamp_ms, key, value, headers, now=None): + metadata = self.records.append(timestamp_ms, key, value, headers) + if metadata is None: + return None + + now = time.time() if now is None else now + self.max_record_size = max(self.max_record_size, metadata.size) + self.last_append = now + future = FutureRecordMetadata( + self.produce_future, + metadata.offset, + metadata.timestamp, + metadata.crc, + len(key) if key is not None else -1, + len(value) if value is not None else -1, + sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1) + return future + + def abort(self, exception): + """Abort the batch and complete the future and callbacks.""" + if self._final_state is not None: + raise Errors.IllegalStateError("Batch has already been completed in final state: %s" % self._final_state) + self._final_state = FinalState.ABORTED + + log.debug("Aborting batch for partition %s: %s", self.topic_partition, exception) + self._complete_future(-1, -1, lambda _: exception) + + def complete(self, base_offset, log_append_time): + """Complete the batch successfully. + + Arguments: + base_offset (int): The base offset of the messages assigned by the server + log_append_time (int): The log append time or -1 if CreateTime is being used + + Returns: True if the batch was completed as a result of this call, and False + if it had been completed previously. + """ + return self.done(base_offset=base_offset, timestamp_ms=log_append_time) + + def complete_exceptionally(self, top_level_exception, record_exceptions_fn): + """ + Complete the batch exceptionally. The provided top-level exception will be used + for each record future contained in the batch. + + Arguments: + top_level_exception (Exception): top-level partition error. + record_exceptions_fn (callable int -> Exception): Record exception function mapping + batch_index to the respective record exception. + Returns: True if the batch was completed as a result of this call, and False + if it had been completed previously. + """ + assert isinstance(top_level_exception, Exception) + assert callable(record_exceptions_fn) + return self.done(top_level_exception=top_level_exception, record_exceptions_fn=record_exceptions_fn) + + def done(self, base_offset=None, timestamp_ms=None, top_level_exception=None, record_exceptions_fn=None): + """ + Finalize the state of a batch. Final state, once set, is immutable. This function may be called + once or twice on a batch. It may be called twice if + 1. An inflight batch expires before a response from the broker is received. The batch's final + state is set to FAILED. But it could succeed on the broker and second time around batch.done() may + try to set SUCCEEDED final state. + + 2. If a transaction abortion happens or if the producer is closed forcefully, the final state is + ABORTED but again it could succeed if broker responds with a success. + + Attempted transitions from [FAILED | ABORTED] --> SUCCEEDED are logged. + Attempted transitions from one failure state to the same or a different failed state are ignored. + Attempted transitions from SUCCEEDED to the same or a failed state throw an exception. + """ + final_state = FinalState.SUCCEEDED if top_level_exception is None else FinalState.FAILED + if self._final_state is None: + self._final_state = final_state + if final_state is FinalState.SUCCEEDED: + log.debug("Successfully produced messages to %s with base offset %s", self.topic_partition, base_offset) + else: + log.warning("Failed to produce messages to topic-partition %s with base offset %s: %s", + self.topic_partition, base_offset, top_level_exception) + self._complete_future(base_offset, timestamp_ms, record_exceptions_fn) + return True + + elif self._final_state is not FinalState.SUCCEEDED: + if final_state is FinalState.SUCCEEDED: + # Log if a previously unsuccessful batch succeeded later on. + log.debug("ProduceResponse returned %s for %s after batch with base offset %s had already been %s.", + final_state, self.topic_partition, base_offset, self._final_state) + else: + # FAILED --> FAILED and ABORTED --> FAILED transitions are ignored. + log.debug("Ignored state transition %s -> %s for %s batch with base offset %s", + self._final_state, final_state, self.topic_partition, base_offset) + else: + # A SUCCESSFUL batch must not attempt another state change. + raise Errors.IllegalStateError("A %s batch must not attempt another state change to %s" % (self._final_state, final_state)) + return False + + def _complete_future(self, base_offset, timestamp_ms, record_exceptions_fn): + if self.produce_future.is_done: + raise Errors.IllegalStateError('Batch is already closed!') + self.produce_future.success((base_offset, timestamp_ms, record_exceptions_fn)) + + def has_reached_delivery_timeout(self, delivery_timeout_ms, now=None): + now = time.time() if now is None else now + return delivery_timeout_ms / 1000 <= now - self.created + + def in_retry(self): + return self._retry + + def retry(self, now=None): + now = time.time() if now is None else now + self._retry = True + self.attempts += 1 + self.last_attempt = now + self.last_append = now + + @property + def is_done(self): + return self.produce_future.is_done + + def __str__(self): + return 'ProducerBatch(topic_partition=%s, record_count=%d)' % ( + self.topic_partition, self.records.next_offset()) + + + diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 3a4e60146..1add95a3b 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -6,15 +6,8 @@ import threading import time -try: - # enum in stdlib as of py3.4 - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum - import kafka.errors as Errors -from kafka.producer.future import FutureRecordMetadata, FutureProduceResult +from kafka.producer.producer_batch import ProducerBatch from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition @@ -41,145 +34,6 @@ def get(self): return self._val -class FinalState(IntEnum): - ABORTED = 0 - FAILED = 1 - SUCCEEDED = 2 - - -class ProducerBatch(object): - def __init__(self, tp, records, now=None): - now = time.time() if now is None else now - self.max_record_size = 0 - self.created = now - self.drained = None - self.attempts = 0 - self.last_attempt = now - self.last_append = now - self.records = records - self.topic_partition = tp - self.produce_future = FutureProduceResult(tp) - self._retry = False - self._final_state = None - - @property - def final_state(self): - return self._final_state - - @property - def record_count(self): - return self.records.next_offset() - - @property - def producer_id(self): - return self.records.producer_id if self.records else None - - @property - def producer_epoch(self): - return self.records.producer_epoch if self.records else None - - @property - def has_sequence(self): - return self.records.has_sequence if self.records else False - - def try_append(self, timestamp_ms, key, value, headers, now=None): - metadata = self.records.append(timestamp_ms, key, value, headers) - if metadata is None: - return None - - now = time.time() if now is None else now - self.max_record_size = max(self.max_record_size, metadata.size) - self.last_append = now - future = FutureRecordMetadata( - self.produce_future, - metadata.offset, - metadata.timestamp, - metadata.crc, - len(key) if key is not None else -1, - len(value) if value is not None else -1, - sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1) - return future - - def abort(self, exception): - """Abort the batch and complete the future and callbacks.""" - if self._final_state is not None: - raise Errors.IllegalStateError("Batch has already been completed in final state: %s" % self._final_state) - self._final_state = FinalState.ABORTED - - log.debug("Aborting batch for partition %s: %s", self.topic_partition, exception) - self._complete_future(-1, -1, exception) - - def done(self, base_offset=None, timestamp_ms=None, exception=None): - """ - Finalize the state of a batch. Final state, once set, is immutable. This function may be called - once or twice on a batch. It may be called twice if - 1. An inflight batch expires before a response from the broker is received. The batch's final - state is set to FAILED. But it could succeed on the broker and second time around batch.done() may - try to set SUCCEEDED final state. - - 2. If a transaction abortion happens or if the producer is closed forcefully, the final state is - ABORTED but again it could succeed if broker responds with a success. - - Attempted transitions from [FAILED | ABORTED] --> SUCCEEDED are logged. - Attempted transitions from one failure state to the same or a different failed state are ignored. - Attempted transitions from SUCCEEDED to the same or a failed state throw an exception. - """ - final_state = FinalState.SUCCEEDED if exception is None else FinalState.FAILED - if self._final_state is None: - self._final_state = final_state - if final_state is FinalState.SUCCEEDED: - log.debug("Successfully produced messages to %s with base offset %s", self.topic_partition, base_offset) - else: - log.warning("Failed to produce messages to topic-partition %s with base offset %s: %s", - self.topic_partition, base_offset, exception) - self._complete_future(base_offset, timestamp_ms, exception) - return True - - elif self._final_state is not FinalState.SUCCEEDED: - if final_state is FinalState.SUCCEEDED: - # Log if a previously unsuccessful batch succeeded later on. - log.debug("ProduceResponse returned %s for %s after batch with base offset %s had already been %s.", - final_state, self.topic_partition, base_offset, self._final_state) - else: - # FAILED --> FAILED and ABORTED --> FAILED transitions are ignored. - log.debug("Ignored state transition %s -> %s for %s batch with base offset %s", - self._final_state, final_state, self.topic_partition, base_offset) - else: - # A SUCCESSFUL batch must not attempt another state change. - raise Errors.IllegalStateError("A %s batch must not attempt another state change to %s" % (self._final_state, final_state)) - return False - - def _complete_future(self, base_offset, timestamp_ms, exception): - if self.produce_future.is_done: - raise Errors.IllegalStateError('Batch is already closed!') - elif exception is None: - self.produce_future.success((base_offset, timestamp_ms)) - else: - self.produce_future.failure(exception) - - def has_reached_delivery_timeout(self, delivery_timeout_ms, now=None): - now = time.time() if now is None else now - return delivery_timeout_ms / 1000 <= now - self.created - - def in_retry(self): - return self._retry - - def retry(self, now=None): - now = time.time() if now is None else now - self._retry = True - self.attempts += 1 - self.last_attempt = now - self.last_append = now - - @property - def is_done(self): - return self.produce_future.is_done - - def __str__(self): - return 'ProducerBatch(topic_partition=%s, record_count=%d)' % ( - self.topic_partition, self.records.next_offset()) - - class RecordAccumulator(object): """ This class maintains a dequeue per TopicPartition that accumulates messages diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 7a4c557c8..09b9a0f10 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -21,6 +21,11 @@ log = logging.getLogger(__name__) +PartitionResponse = collections.namedtuple("PartitionResponse", + ["error", "base_offset", "last_offset", "log_append_time", "log_start_offset", "record_errors", "error_message", "current_leader"]) +PartitionResponse.__new__.__defaults__ = (Errors.NoError, -1, -1, -1, -1, (), None, (-1, -1)) + + class Sender(threading.Thread): """ The background thread that handles the sending of produce requests to the @@ -225,11 +230,10 @@ def _send_producer_data(self, now=None): needs_transaction_state_reset = False for expired_batch in expired_batches: - error = Errors.KafkaTimeoutError( - "Expiring %d record(s) for %s: %s ms has passed since batch creation" % ( - expired_batch.record_count, expired_batch.topic_partition, - int((time.time() - expired_batch.created) * 1000))) - self._fail_batch(expired_batch, error, base_offset=-1) + error_message = "Expiring %d record(s) for %s: %s ms has passed since batch creation" % ( + expired_batch.record_count, expired_batch.topic_partition, + int((time.time() - expired_batch.created) * 1000)) + self._fail_batch(expired_batch, PartitionResponse(error=Errors.KafkaTimeoutError, error_message=error_message)) if self._sensors: self._sensors.update_produce_request_metrics(batches_by_node) @@ -391,7 +395,7 @@ def _maybe_wait_for_producer_id(self): def _failed_produce(self, batches, node_id, error): log.error("%s: Error sending produce request to node %d: %s", str(self), node_id, error) # trace for batch in batches: - self._complete_batch(batch, error, -1) + self._complete_batch(batch, PartitionResponse(error=error)) def _handle_produce_response(self, node_id, send_time, batches, response): """Handle a produce response.""" @@ -403,35 +407,67 @@ def _handle_produce_response(self, node_id, send_time, batches, response): for topic, partitions in response.topics: for partition_info in partitions: + log_append_time = -1 + log_start_offset = -1 + record_errors = () + error_message = None if response.API_VERSION < 2: - partition, error_code, offset = partition_info - ts = None + partition, error_code, base_offset = partition_info elif 2 <= response.API_VERSION <= 4: - partition, error_code, offset, ts = partition_info + partition, error_code, base_offset, log_append_time = partition_info elif 5 <= response.API_VERSION <= 7: - partition, error_code, offset, ts, _log_start_offset = partition_info + partition, error_code, base_offset, log_append_time, log_start_offset = partition_info else: - # Currently unused / TODO: KIP-467 - partition, error_code, offset, ts, _log_start_offset, _record_errors, _global_error = partition_info + partition, error_code, base_offset, log_append_time, log_start_offset, record_errors, error_message = partition_info tp = TopicPartition(topic, partition) - error = Errors.for_code(error_code) batch = batches_by_partition[tp] - self._complete_batch(batch, error, offset, timestamp_ms=ts) - + partition_response = PartitionResponse( + error=Errors.for_code(error_code), + base_offset=base_offset, + last_offset=-1, + log_append_time=log_append_time, + log_start_offset=log_start_offset, + record_errors=record_errors, + error_message=error_message, + ) + self._complete_batch(batch, partition_response) else: # this is the acks = 0 case, just complete all requests for batch in batches: - self._complete_batch(batch, None, -1) + self._complete_batch(batch, PartitionResponse()) + + def _record_exceptions_fn(self, top_level_exception, record_errors, error_message): + """Returns a fn mapping batch_index to exception""" + # When no record_errors, all batches resolve to top-level exception + if not record_errors: + return lambda _: top_level_exception + + record_errors_dict = dict(record_errors) + def record_exceptions_fn(batch_index): + if batch_index not in record_errors_dict: + return Errors.KafkaError( + "Failed to append record because it was part of a batch which had one more more invalid records") + record_error = record_errors_dict[batch_index] + err_msg = record_error or error_message or top_level_exception.description + exc = top_level_exception.__class__ if len(record_errors) == 1 else Errors.InvalidRecordError + return exc(err_msg) + return record_exceptions_fn + + def _fail_batch(self, batch, partition_response): + if partition_response.error is Errors.TopicAuthorizationFailedError: + exception = Errors.TopicAuthorizationFailedError(batch.topic_partition.topic) + elif partition_response.error is Errors.ClusterAuthorizationFailedError: + exception = Errors.ClusterAuthorizationFailedError("The producer is not authorized to do idempotent sends") + else: + exception = partition_response.error(partition_response.error_message) - def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None): - exception = exception if type(exception) is not type else exception() if self._transaction_manager: if isinstance(exception, Errors.OutOfOrderSequenceNumberError) and \ not self._transaction_manager.is_transactional() and \ self._transaction_manager.has_producer_id(batch.producer_id): log.error("%s: The broker received an out of order sequence number for topic-partition %s" " at offset %s. This indicates data loss on the broker, and should be investigated.", - str(self), batch.topic_partition, base_offset) + str(self), batch.topic_partition, partition_response.base_offset) # Reset the transaction state since we have hit an irrecoverable exception and cannot make any guarantees # about the previously committed message. Note that this will discard the producer id and sequence @@ -448,31 +484,31 @@ def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None): if self._sensors: self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) - if batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, exception=exception): + record_exceptions_fn = self._record_exceptions_fn(exception, partition_response.record_errors, partition_response.error_message) + if batch.complete_exceptionally(exception, record_exceptions_fn): self._maybe_remove_from_inflight_batches(batch) self._accumulator.deallocate(batch) - def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): + def _complete_batch(self, batch, partition_response): """Complete or retry the given batch of records. Arguments: batch (ProducerBatch): The record batch - error (Exception): The error (or None if none) - base_offset (int): The base offset assigned to the records if successful - timestamp_ms (int, optional): The timestamp returned by the broker for this batch + partition_response (PartitionResponse): Response details for partition """ # Standardize no-error to None + error = partition_response.error if error is Errors.NoError: error = None if error is not None: if self._can_retry(batch, error): # retry - log.warning("%s: Got error produce response on topic-partition %s," - " retrying (%s attempts left). Error: %s", + log.warning("%s: Got error produce response on topic-partition %s, retrying (%s attempts left): %s%s", str(self), batch.topic_partition, self.config['retries'] - batch.attempts - 1, - error) + error.__class__.__name__, + (". Error Message: %s" % partition_response.error_message) if partition_response.error_message else "") # If idempotence is enabled only retry the request if the batch matches our current producer id and epoch if not self._transaction_manager or self._transaction_manager.producer_id_and_epoch.match(batch): @@ -488,13 +524,10 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): str(self), batch.producer_id, batch.producer_epoch, self._transaction_manager.producer_id_and_epoch.producer_id, self._transaction_manager.producer_id_and_epoch.epoch) - self._fail_batch(batch, error, base_offset=base_offset, timestamp_ms=timestamp_ms) + self._fail_batch(batch, partition_response) else: - if error is Errors.TopicAuthorizationFailedError: - error = error(batch.topic_partition.topic) - # tell the user the result of their request - self._fail_batch(batch, error, base_offset=base_offset, timestamp_ms=timestamp_ms) + self._fail_batch(batch, partition_response) if error is Errors.UnknownTopicOrPartitionError: log.warning("%s: Received unknown topic or partition error in produce request on partition %s." @@ -505,7 +538,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): self._metadata.request_update() else: - if batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms): + if batch.complete(partition_response.base_offset, partition_response.log_append_time): self._maybe_remove_from_inflight_batches(batch) self._accumulator.deallocate(batch) @@ -561,7 +594,7 @@ def _produce_request(self, node_id, acks, timeout, batches): buf = batch.records.buffer() produce_records_by_partition[topic][partition] = buf - version = self._client.api_version(ProduceRequest, max_version=7) + version = self._client.api_version(ProduceRequest, max_version=8) topic_partition_data = [ (topic, list(partition_info.items())) for topic, partition_info in six.iteritems(produce_records_by_partition)] diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py index 3076a2810..94edd0f80 100644 --- a/kafka/protocol/produce.py +++ b/kafka/protocol/produce.py @@ -112,12 +112,12 @@ class ProduceResponse_v8(Response): ('error_code', Int16), ('offset', Int64), ('timestamp', Int64), - ('log_start_offset', Int64)), + ('log_start_offset', Int64), ('record_errors', (Array( ('batch_index', Int32), ('batch_index_error_message', String('utf-8')) ))), - ('error_message', String('utf-8')) + ('error_message', String('utf-8'))) ))), ('throttle_time_ms', Int32) ) diff --git a/test/test_producer_batch.py b/test/test_producer_batch.py new file mode 100644 index 000000000..bffa79fcb --- /dev/null +++ b/test/test_producer_batch.py @@ -0,0 +1,136 @@ +# pylint: skip-file +from __future__ import absolute_import, division + +import pytest + +from kafka.errors import IllegalStateError, KafkaError +from kafka.producer.future import FutureRecordMetadata, RecordMetadata +from kafka.producer.producer_batch import ProducerBatch +from kafka.record.memory_records import MemoryRecordsBuilder +from kafka.structs import TopicPartition + + +@pytest.fixture +def tp(): + return TopicPartition('foo', 0) + + +@pytest.fixture +def memory_records_builder(): + return MemoryRecordsBuilder(magic=2, compression_type=0, batch_size=100000) + + +@pytest.fixture +def batch(tp, memory_records_builder): + return ProducerBatch(tp, memory_records_builder) + + +def test_producer_batch_producer_id(tp, memory_records_builder): + batch = ProducerBatch(tp, memory_records_builder) + assert batch.producer_id == -1 + batch.records.set_producer_state(123, 456, 789, False) + assert batch.producer_id == 123 + memory_records_builder.close() + assert batch.producer_id == 123 + + +@pytest.mark.parametrize("magic", [0, 1, 2]) +def test_producer_batch_try_append(magic): + tp = TopicPartition('foo', 0) + records = MemoryRecordsBuilder( + magic=magic, compression_type=0, batch_size=100000) + batch = ProducerBatch(tp, records) + assert batch.record_count == 0 + future = batch.try_append(0, b'key', b'value', []) + assert isinstance(future, FutureRecordMetadata) + assert not future.is_done + batch.complete(123, 456) + assert future.is_done + # record-level checksum only provided in v0/v1 formats; payload includes magic-byte + if magic == 0: + checksum = 592888119 + elif magic == 1: + checksum = 213653215 + else: + checksum = None + + expected_metadata = RecordMetadata( + topic=tp[0], partition=tp[1], topic_partition=tp, + offset=123, timestamp=456, checksum=checksum, + serialized_key_size=3, serialized_value_size=5, serialized_header_size=-1) + assert future.value == expected_metadata + + +def test_producer_batch_retry(batch): + assert not batch.in_retry() + batch.retry() + assert batch.in_retry() + + +def test_batch_abort(batch): + future = batch.try_append(123, None, b'msg', []) + batch.abort(KafkaError()) + assert future.is_done + + # subsequent completion should be ignored + assert not batch.complete(500, 2342342341) + assert not batch.complete_exceptionally(KafkaError('top_level'), lambda _: KafkaError('record')) + + assert future.is_done + with pytest.raises(KafkaError): + future.get() + + +def test_batch_cannot_abort_twice(batch): + future = batch.try_append(123, None, b'msg', []) + batch.abort(KafkaError()) + with pytest.raises(IllegalStateError): + batch.abort(KafkaError()) + assert future.is_done + with pytest.raises(KafkaError): + future.get() + + +def test_batch_cannot_complete_twice(batch): + future = batch.try_append(123, None, b'msg', []) + batch.complete(500, 10) + with pytest.raises(IllegalStateError): + batch.complete(1000, 20) + record_metadata = future.get() + assert record_metadata.offset == 500 + assert record_metadata.timestamp == 10 + + +def _test_complete_exceptionally(batch, record_count, top_level_exception, record_exceptions_fn): + futures = [] + for i in range(record_count): + futures.append(batch.try_append(0, b'key', b'value', [])) + + assert record_count == batch.record_count + + batch.complete_exceptionally(top_level_exception, record_exceptions_fn) + assert batch.is_done + + for i, future in enumerate(futures): + assert future.is_done + assert future.failed() + assert isinstance(future.exception, RuntimeError) + assert record_exceptions_fn(i) == future.exception + + +def test_complete_exceptionally_with_record_errors(batch): + record_count = 5 + top_level_exception = RuntimeError() + + record_exceptions_map = {0: RuntimeError(), 3: RuntimeError()} + record_exceptions_fn = lambda i: record_exceptions_map.get(i, top_level_exception) + + _test_complete_exceptionally(batch, record_count, top_level_exception, record_exceptions_fn) + + +def test_complete_exceptionally_with_null_record_errors(batch): + record_count = 5 + top_level_exception = RuntimeError() + + with pytest.raises(AssertionError): + _test_complete_exceptionally(batch, record_count, top_level_exception, None) diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py index 5c7134e5c..0f61c21cf 100644 --- a/test/test_record_accumulator.py +++ b/test/test_record_accumulator.py @@ -4,11 +4,8 @@ import pytest from kafka.cluster import ClusterMetadata -from kafka.errors import IllegalStateError, KafkaError -from kafka.producer.future import FutureRecordMetadata, RecordMetadata -from kafka.producer.record_accumulator import RecordAccumulator, ProducerBatch +from kafka.producer.record_accumulator import RecordAccumulator from kafka.record.default_records import DefaultRecordBatchBuilder -from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition @@ -23,103 +20,6 @@ def cluster(tp, mocker): mocker.patch.object(metadata, 'partitions_for_broker', return_value=[tp]) return metadata -def test_producer_batch_producer_id(): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=2, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - assert batch.producer_id == -1 - batch.records.set_producer_state(123, 456, 789, False) - assert batch.producer_id == 123 - records.close() - assert batch.producer_id == 123 - -@pytest.mark.parametrize("magic", [0, 1, 2]) -def test_producer_batch_try_append(magic): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=magic, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - assert batch.record_count == 0 - future = batch.try_append(0, b'key', b'value', []) - assert isinstance(future, FutureRecordMetadata) - assert not future.is_done - batch.done(base_offset=123, timestamp_ms=456) - assert future.is_done - # record-level checksum only provided in v0/v1 formats; payload includes magic-byte - if magic == 0: - checksum = 592888119 - elif magic == 1: - checksum = 213653215 - else: - checksum = None - - expected_metadata = RecordMetadata( - topic=tp[0], partition=tp[1], topic_partition=tp, - offset=123, timestamp=456, checksum=checksum, - serialized_key_size=3, serialized_value_size=5, serialized_header_size=-1) - assert future.value == expected_metadata - -def test_producer_batch_retry(): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=2, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - assert not batch.in_retry() - batch.retry() - assert batch.in_retry() - -def test_batch_abort(): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=2, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - future = batch.try_append(123, None, b'msg', []) - - batch.abort(KafkaError()) - assert future.is_done - - # subsequent completion should be ignored - batch.done(500, 2342342341) - batch.done(exception=KafkaError()) - - assert future.is_done - with pytest.raises(KafkaError): - future.get() - -def test_batch_cannot_abort_twice(): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=2, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - future = batch.try_append(123, None, b'msg', []) - - batch.abort(KafkaError()) - - with pytest.raises(IllegalStateError): - batch.abort(KafkaError()) - - assert future.is_done - with pytest.raises(KafkaError): - future.get() - -def test_batch_cannot_complete_twice(): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=2, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - future = batch.try_append(123, None, b'msg', []) - - batch.done(500, 10, None) - - with pytest.raises(IllegalStateError): - batch.done(1000, 20, None) - - record_metadata = future.get() - - assert record_metadata.offset == 500 - assert record_metadata.timestamp == 10 - def test_linger(tp, cluster): now = 0 accum = RecordAccumulator(linger_ms=10) diff --git a/test/test_sender.py b/test/test_sender.py index 6d29c1e44..567f1b2ad 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -19,8 +19,10 @@ from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.producer.kafka import KafkaProducer from kafka.protocol.produce import ProduceRequest -from kafka.producer.record_accumulator import RecordAccumulator, ProducerBatch -from kafka.producer.sender import Sender +from kafka.producer.future import FutureRecordMetadata +from kafka.producer.producer_batch import ProducerBatch +from kafka.producer.record_accumulator import RecordAccumulator +from kafka.producer.sender import PartitionResponse, Sender from kafka.producer.transaction_manager import TransactionManager from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition @@ -92,11 +94,11 @@ def test_complete_batch_success(sender): assert not batch.produce_future.is_done # No error, base_offset 0 - sender._complete_batch(batch, None, 0, timestamp_ms=123) + sender._complete_batch(batch, PartitionResponse(base_offset=0, log_append_time=123)) assert batch.is_done assert batch.produce_future.is_done assert batch.produce_future.succeeded() - assert batch.produce_future.value == (0, 123) + assert batch.produce_future.value == (0, 123, None) def test_complete_batch_transaction(sender, transaction_manager): @@ -106,7 +108,7 @@ def test_complete_batch_transaction(sender, transaction_manager): assert sender._transaction_manager.producer_id_and_epoch.producer_id == batch.producer_id # No error, base_offset 0 - sender._complete_batch(batch, None, 0) + sender._complete_batch(batch, PartitionResponse(base_offset=0)) assert batch.is_done assert sender._transaction_manager.sequence_number(batch.topic_partition) == batch.record_count @@ -134,14 +136,15 @@ def test_complete_batch_error(sender, error, refresh_metadata): sender.config['retries'] = 0 assert sender._client.cluster.ttl() > 0 batch = producer_batch() - sender._complete_batch(batch, error, -1) + future = FutureRecordMetadata(batch.produce_future, -1, -1, -1, -1, -1, -1) + sender._complete_batch(batch, PartitionResponse(error=error)) if refresh_metadata: assert sender._client.cluster.ttl() == 0 else: assert sender._client.cluster.ttl() > 0 assert batch.is_done - assert batch.produce_future.failed() - assert isinstance(batch.produce_future.exception, error) + assert future.failed() + assert isinstance(future.exception, error) @pytest.mark.parametrize(("error", "retry"), [ @@ -163,37 +166,40 @@ def test_complete_batch_error(sender, error, refresh_metadata): ]) def test_complete_batch_retry(sender, accumulator, mocker, error, retry): sender.config['retries'] = 1 - mocker.spy(sender, '_fail_batch') mocker.patch.object(accumulator, 'reenqueue') batch = producer_batch() - sender._complete_batch(batch, error, -1) + future = FutureRecordMetadata(batch.produce_future, -1, -1, -1, -1, -1, -1) + sender._complete_batch(batch, PartitionResponse(error=error)) if retry: assert not batch.is_done accumulator.reenqueue.assert_called_with(batch) batch.attempts += 1 # normally handled by accumulator.reenqueue, but it's mocked - sender._complete_batch(batch, error, -1) + sender._complete_batch(batch, PartitionResponse(error=error)) assert batch.is_done - assert isinstance(batch.produce_future.exception, error) + assert future.failed() + assert isinstance(future.exception, error) else: assert batch.is_done - assert isinstance(batch.produce_future.exception, error) + assert future.failed() + assert isinstance(future.exception, error) def test_complete_batch_producer_id_changed_no_retry(sender, accumulator, transaction_manager, mocker): sender._transaction_manager = transaction_manager sender.config['retries'] = 1 - mocker.spy(sender, '_fail_batch') mocker.patch.object(accumulator, 'reenqueue') error = Errors.NotLeaderForPartitionError batch = producer_batch() - sender._complete_batch(batch, error, -1) + future = FutureRecordMetadata(batch.produce_future, -1, -1, -1, -1, -1, -1) + sender._complete_batch(batch, PartitionResponse(error=error)) assert not batch.is_done accumulator.reenqueue.assert_called_with(batch) batch.records._producer_id = 123 # simulate different producer_id assert batch.producer_id != sender._transaction_manager.producer_id_and_epoch.producer_id - sender._complete_batch(batch, error, -1) + sender._complete_batch(batch, PartitionResponse(error=error)) assert batch.is_done - assert isinstance(batch.produce_future.exception, error) + assert future.failed() + assert isinstance(future.exception, error) def test_fail_batch(sender, accumulator, transaction_manager, mocker): @@ -201,9 +207,9 @@ def test_fail_batch(sender, accumulator, transaction_manager, mocker): batch = producer_batch() mocker.patch.object(batch, 'done') assert sender._transaction_manager.producer_id_and_epoch.producer_id == batch.producer_id - error = Exception('error') - sender._fail_batch(batch, base_offset=0, timestamp_ms=None, exception=error) - batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error) + error = Errors.KafkaError + sender._fail_batch(batch, PartitionResponse(error=error)) + batch.done.assert_called_with(top_level_exception=error(None), record_exceptions_fn=mocker.ANY) def test_out_of_order_sequence_number_reset_producer_id(sender, accumulator, transaction_manager, mocker): @@ -213,10 +219,10 @@ def test_out_of_order_sequence_number_reset_producer_id(sender, accumulator, tra batch = producer_batch() mocker.patch.object(batch, 'done') assert sender._transaction_manager.producer_id_and_epoch.producer_id == batch.producer_id - error = Errors.OutOfOrderSequenceNumberError() - sender._fail_batch(batch, base_offset=0, timestamp_ms=None, exception=error) + error = Errors.OutOfOrderSequenceNumberError + sender._fail_batch(batch, PartitionResponse(base_offset=0, log_append_time=None, error=error)) sender._transaction_manager.reset_producer_id.assert_called_once() - batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error) + batch.done.assert_called_with(top_level_exception=error(None), record_exceptions_fn=mocker.ANY) def test_handle_produce_response(): @@ -228,9 +234,9 @@ def test_failed_produce(sender, mocker): mock_batches = ['foo', 'bar', 'fizzbuzz'] sender._failed_produce(mock_batches, 0, 'error') sender._complete_batch.assert_has_calls([ - call('foo', 'error', -1), - call('bar', 'error', -1), - call('fizzbuzz', 'error', -1), + call('foo', PartitionResponse(error='error')), + call('bar', PartitionResponse(error='error')), + call('fizzbuzz', PartitionResponse(error='error')), ]) @@ -253,3 +259,14 @@ def test__send_producer_data_expiry_time_reset(sender, accumulator, mocker): now += accumulator.config['delivery_timeout_ms'] poll_timeout_ms = sender._send_producer_data(now=now) assert poll_timeout_ms > 0 + + +def test__record_exceptions_fn(sender): + record_exceptions_fn = sender._record_exceptions_fn(Errors.KafkaError('top-level'), [(0, 'err-0'), (3, 'err-3')], 'message') + assert record_exceptions_fn(0) == Errors.InvalidRecordError('err-0') + assert record_exceptions_fn(1) == Errors.KafkaError('Failed to append record because it was part of a batch which had one more more invalid records') + assert record_exceptions_fn(2) == Errors.KafkaError('Failed to append record because it was part of a batch which had one more more invalid records') + assert record_exceptions_fn(3) == Errors.InvalidRecordError('err-3') + + record_exceptions_fn = sender._record_exceptions_fn(Errors.KafkaError('top-level'), [(0, 'err-0')], 'message') + assert record_exceptions_fn(0) == Errors.KafkaError('err-0') From ee5436f0d0e10a6249db4ec17652e3af502e291d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 3 Jul 2025 13:02:17 -0700 Subject: [PATCH 17/51] KafkaProducer: Handle UnknownProducerIdError (#2663) --- kafka/producer/sender.py | 7 +++++++ kafka/producer/transaction_manager.py | 4 ++++ 2 files changed, 11 insertions(+) diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 09b9a0f10..869689e9b 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -473,6 +473,13 @@ def _fail_batch(self, batch, partition_response): # about the previously committed message. Note that this will discard the producer id and sequence # numbers for all existing partitions. self._transaction_manager.reset_producer_id() + elif isinstance(exception, Errors.UnknownProducerIdError): + # If we get an UnknownProducerId for a partition, then the broker has no state for that producer. It will + # therefore accept a write with sequence number 0. We reset the sequence number for the partition here so + # that the producer can continue after aborting the transaction. All inflight-requests to this partition + # will also fail with an UnknownProducerId error, so the sequence will remain at 0. Note that if the + # broker supports bumping the epoch, we will later reset all sequence numbers after calling InitProducerId + self._transaction_manager.reset_sequence_for_partition(batch.topic_partition) elif isinstance(exception, (Errors.ClusterAuthorizationFailedError, Errors.TransactionalIdAuthorizationFailedError, Errors.ProducerFencedError, diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index 5d69ddc97..a44d7d9b3 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -330,6 +330,10 @@ def increment_sequence_number(self, tp, increment): else: self._sequence_numbers[tp] += increment + def reset_sequence_for_partition(self, tp): + with self._lock: + self._sequence_numbers.pop(tp, None) + def next_request_handler(self, has_incomplete_batches): with self._lock: if self._new_partitions_in_transaction: From 20e2d52ff2c337e02f8bac7af6c6e8d00ebcc63f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 3 Jul 2025 13:03:07 -0700 Subject: [PATCH 18/51] KIP-654: Abort transaction with pending data with TransactionAbortedError (#2662) --- kafka/errors.py | 4 ++++ kafka/producer/sender.py | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/kafka/errors.py b/kafka/errors.py index 351e07375..dffa35f35 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -102,6 +102,10 @@ class UnsupportedCodecError(KafkaError): pass +class TransactionAbortedError(KafkaError): + pass + + class BrokerResponseError(KafkaError): errno = None message = None diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 869689e9b..b7c38a4f5 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -166,7 +166,11 @@ def run_once(self): self._client.poll(timeout_ms=self.config['retry_backoff_ms']) return elif self._transaction_manager.has_abortable_error(): - self._accumulator.abort_undrained_batches(self._transaction_manager.last_error) + # Attempt to get the last error that caused this abort. + # If there was no error, but we are still aborting, + # then this is most likely a case where there was no fatal error. + exception = self._transaction_manager.last_error or Errors.TransactionAbortedError() + self._accumulator.abort_undrained_batches(exception) except Errors.SaslAuthenticationFailedError as e: # This is already logged as error, but propagated here to perform any clean ups. From 512d0a0b8d71cf7f34f1b23f8a42d52c28af3266 Mon Sep 17 00:00:00 2001 From: Alexandre Feblot Date: Sat, 19 Jul 2025 16:10:16 +0200 Subject: [PATCH 19/51] Fetcher: Add missing argument in debug log (#2665) Co-authored-by: Alexandre Feblot --- kafka/consumer/fetcher.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index d57bc4786..8e524790c 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -661,7 +661,8 @@ def _create_fetch_requests(self): elif not self._client.ready(node_id): # Until we support send request queues, any attempt to send to a not-ready node will be # immediately failed with NodeNotReadyError. - log.debug("Skipping fetch for partition %s because connection to leader node is not ready yet") + log.debug("Skipping fetch for partition %s because connection to leader node is not ready yet", + partition) elif node_id in self._nodes_with_pending_fetch_requests: log.debug("Skipping fetch for partition %s because there is a pending fetch request to node %s", From f0fc1f2b873124a9f9c17145633ec4b26414fb6c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 18 Nov 2025 08:21:24 -0800 Subject: [PATCH 20/51] NixOS helpers --- .gitignore | 2 ++ Makefile | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/.gitignore b/.gitignore index f3cd082fa..8541aae23 100644 --- a/.gitignore +++ b/.gitignore @@ -15,3 +15,5 @@ docs/_build integration-test/ tests-env/ .pytest_cache/ +.envrc +shell.nix diff --git a/Makefile b/Makefile index 30da9cf91..cf0b387e8 100644 --- a/Makefile +++ b/Makefile @@ -97,6 +97,7 @@ servers/%/kafka-bin: servers/dist/$$(call kafka_artifact_name,$$*) | servers/dis mkdir -p $@ tar xzvf $< -C $@ --strip-components 1 if [[ "$*" < "1" ]]; then make servers/patch-libs/$*; fi + if [[ -f "/etc/NIXOS" ]]; then make servers/patch-nixos-shebang/$*; fi servers/%/api_versions: servers/$$*/kafka-bin KAFKA_VERSION=$* python -m test.integration.fixtures get_api_versions >$@ @@ -109,6 +110,9 @@ servers/%/messages: servers/$$*/kafka-bin servers/patch-libs/%: servers/dist/jakarta.xml.bind-api-2.3.3.jar | servers/$$*/kafka-bin cp $< servers/$*/kafka-bin/libs/ +servers/patch-nixos-shebang/%: + for f in $$(ls servers/$*/kafka-bin/bin/*.sh); do if (head -1 $$f | grep '#!/bin/bash' >/dev/null); then echo $$f; sed -i '1s|/bin/bash|/run/current-system/sw/bin/bash|' $$f; fi; done; + servers/download/%: servers/dist/$$(call kafka_artifact_name,$$*) | servers/dist ; # Avoid removing any pattern match targets as intermediates (without this, .tgz artifacts are removed by make after extraction) From 1784b1ca5b78a80b50fe91cbb361d69b9479794e Mon Sep 17 00:00:00 2001 From: Konstantin Kholostov Date: Tue, 18 Nov 2025 22:46:05 +0100 Subject: [PATCH 21/51] Ensure timeout is checked after each fetch position update in `Consumer.position()` (#2668) --- kafka/consumer/group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index dcb322b84..c7ad0f59b 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -781,7 +781,7 @@ def position(self, partition, timeout_ms=None): # batch update fetch positions for any partitions without a valid position if self._update_fetch_positions(timeout_ms=timer.timeout_ms): position = self._subscription.assignment[partition].position - elif timer.expired: + if timer.expired: return None else: return position.offset From 625a9d713193933250ab3890f13b0a3eff19fdf2 Mon Sep 17 00:00:00 2001 From: Guilherme Quentel Melo Date: Tue, 18 Nov 2025 23:28:59 +0100 Subject: [PATCH 22/51] Fix thread not waking up when there is still data to be sent (#2670) --- kafka/conn.py | 7 +++++++ test/test_conn.py | 52 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+) diff --git a/kafka/conn.py b/kafka/conn.py index 64445fab0..c1d42a0b6 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -1075,6 +1075,13 @@ def send_pending_requests_v2(self): total_bytes = self._send_bytes(self._send_buffer) self._send_buffer = self._send_buffer[total_bytes:] + # If all data was sent, we need to get the new data from the protocol now, otherwise + # this function would return True, indicating that there are no more pending + # requests. This could cause the calling thread to wait indefinitely as it won't + # know that there is still buffered data to send. + if not self._send_buffer: + self._send_buffer = self._protocol.send_bytes() + if self._sensors: self._sensors.bytes_sent.record(total_bytes) # Return True iff send buffer is empty diff --git a/test/test_conn.py b/test/test_conn.py index 8d56668c5..a5761442c 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -12,6 +12,9 @@ from kafka.conn import BrokerConnection, ConnectionStates from kafka.future import Future +from kafka.conn import BrokerConnection, ConnectionStates, SSLWantWriteError +from kafka.metrics.metrics import Metrics +from kafka.metrics.stats.sensor import Sensor from kafka.protocol.api import RequestHeader from kafka.protocol.group import HeartbeatResponse from kafka.protocol.metadata import MetadataRequest @@ -43,6 +46,15 @@ def _socket(mocker): mocker.patch('socket.socket', return_value=socket) return socket +def metrics(mocker): + metrics = mocker.MagicMock(Metrics) + metrics.mocked_sensors = {} + def sensor(name, **kwargs): + if name not in metrics.mocked_sensors: + metrics.mocked_sensors[name] = mocker.MagicMock(Sensor) + return metrics.mocked_sensors[name] + metrics.sensor.side_effect = sensor + return metrics @pytest.fixture def conn(_socket, dns_lookup, mocker): @@ -228,6 +240,46 @@ def test_send_response(_socket, conn): assert len(conn.in_flight_requests) == 1 +def test_send_async_request_while_other_request_is_already_in_buffer(_socket, conn, metrics): + conn.connect() + assert conn.state is ConnectionStates.CONNECTED + assert 'node-0.bytes-sent' in metrics.mocked_sensors + bytes_sent_sensor = metrics.mocked_sensors['node-0.bytes-sent'] + + req1 = MetadataRequest[0](topics='foo') + header1 = RequestHeader(req1, client_id=conn.config['client_id']) + payload_bytes1 = len(header1.encode()) + len(req1.encode()) + req2 = MetadataRequest[0]([]) + header2 = RequestHeader(req2, client_id=conn.config['client_id']) + payload_bytes2 = len(header2.encode()) + len(req2.encode()) + + # The first call to the socket will raise a transient SSL exception. This will make the first + # request to be kept in the internal buffer to be sent in the next call of + # send_pending_requests_v2. + _socket.send.side_effect = [SSLWantWriteError, 4 + payload_bytes1, 4 + payload_bytes2] + + conn.send(req1, blocking=False) + # This won't send any bytes because of the SSL exception and the request bytes will be kept in + # the buffer. + assert conn.send_pending_requests_v2() is False + assert bytes_sent_sensor.record.call_args_list[0].args == (0,) + + conn.send(req2, blocking=False) + # This will send the remaining bytes in the buffer from the first request, but should notice + # that the second request was queued, therefore it should return False. + bytes_sent_sensor.record.reset_mock() + assert conn.send_pending_requests_v2() is False + bytes_sent_sensor.record.assert_called_once_with(4 + payload_bytes1) + + bytes_sent_sensor.record.reset_mock() + assert conn.send_pending_requests_v2() is True + bytes_sent_sensor.record.assert_called_once_with(4 + payload_bytes2) + + bytes_sent_sensor.record.reset_mock() + assert conn.send_pending_requests_v2() is True + bytes_sent_sensor.record.assert_called_once_with(0) + + def test_send_error(_socket, conn): conn.connect() assert conn.state is ConnectionStates.CONNECTED From de55dde654cd7eedbc2db267495fe02478ffb839 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 18 Nov 2025 14:47:47 -0800 Subject: [PATCH 23/51] Fixup new conn test --- test/test_conn.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/test/test_conn.py b/test/test_conn.py index a5761442c..3fa64066e 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -46,6 +46,7 @@ def _socket(mocker): mocker.patch('socket.socket', return_value=socket) return socket +@pytest.fixture def metrics(mocker): metrics = mocker.MagicMock(Metrics) metrics.mocked_sensors = {} @@ -57,8 +58,8 @@ def sensor(name, **kwargs): return metrics @pytest.fixture -def conn(_socket, dns_lookup, mocker): - conn = BrokerConnection('localhost', 9092, socket.AF_INET) +def conn(_socket, dns_lookup, metrics, mocker): + conn = BrokerConnection('localhost', 9092, socket.AF_INET, metrics=metrics) mocker.patch.object(conn, '_try_api_versions_check', return_value=True) return conn From a75edb75932e490d7aeef1183ada9722a6f4e07a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Nov 2025 14:52:01 -0800 Subject: [PATCH 24/51] Bump actions/checkout from 4 to 5 (#2669) --- .github/workflows/codeql-analysis.yml | 2 +- .github/workflows/python-package.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 4f6360b71..e0a113db4 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -36,7 +36,7 @@ jobs: # https://docs.github.com/en/free-pro-team@latest/github/finding-security-vulnerabilities-and-errors-in-your-code/configuring-code-scanning#changing-the-languages-that-are-analyzed steps: - name: Checkout repository - uses: actions/checkout@v4 + uses: actions/checkout@v5 # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index df790120a..b8bca1cdd 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -52,7 +52,7 @@ jobs: kafka: "4.0.0" steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v5 - name: Set up Python ${{ matrix.python }} uses: actions/setup-python@v5 with: From e9673314782c7e9a47701acd3fe0d71eb56a195f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Nov 2025 15:00:10 -0800 Subject: [PATCH 25/51] Bump actions/setup-java from 4 to 5 (#2673) --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index b8bca1cdd..07f86c233 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -68,7 +68,7 @@ jobs: - name: Pylint run: pylint --recursive=y --errors-only --exit-zero kafka test - name: Setup java - uses: actions/setup-java@v4 + uses: actions/setup-java@v5 with: distribution: temurin java-version: 23 From edd74554adfefb5c2acebf4449c6753b98367522 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 18 Nov 2025 15:00:36 -0800 Subject: [PATCH 26/51] Add transactional_id to KafkaProducer Keyword Arguments docstring --- kafka/producer/kafka.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 9401bd814..005274c82 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -134,10 +134,14 @@ class KafkaProducer(object): value_serializer (callable): used to convert user-supplied message values to bytes. If not None, called as f(value), should return bytes. Default: None. + transactional_id (str): Enable transactional producer with a unique + identifier. This will be used to identify the same producer + instance across process restarts. Default: None. enable_idempotence (bool): When set to True, the producer will ensure that exactly one copy of each message is written in the stream. If False, producer retries due to broker failures, etc., may write - duplicates of the retried message in the stream. Default: False. + duplicates of the retried message in the stream. + Default: True if `transactional_id` is provided, otherwise False. Note that enabling idempotence requires `max_in_flight_requests_per_connection` to be set to 1 and `retries` From 894ad8e1aa4eb3650e6b8487b44d73a016a6c77a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Nov 2025 15:11:44 -0800 Subject: [PATCH 27/51] Bump actions/setup-python from 5 to 6 (#2674) --- .github/workflows/python-package.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 07f86c233..e9fbd108a 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -54,7 +54,7 @@ jobs: steps: - uses: actions/checkout@v5 - name: Set up Python ${{ matrix.python }} - uses: actions/setup-python@v5 + uses: actions/setup-python@v6 with: python-version: ${{ matrix.python }} cache: pip From befd1810db36928e6b25e352e1fdf7d440c02668 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Nov 2025 16:05:18 -0800 Subject: [PATCH 28/51] Bump github/codeql-action from 3 to 4 (#2678) --- .github/workflows/codeql-analysis.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index e0a113db4..a909f7cf5 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -40,7 +40,7 @@ jobs: # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL - uses: github/codeql-action/init@v3 + uses: github/codeql-action/init@v4 with: languages: ${{ matrix.language }} # If you wish to specify custom queries, you can do so here or in a config file. @@ -51,7 +51,7 @@ jobs: # Autobuild attempts to build any compiled languages (C/C++, C#, or Java). # If this step fails, then you should remove it and run the build manually (see below) - name: Autobuild - uses: github/codeql-action/autobuild@v3 + uses: github/codeql-action/autobuild@v4 # ℹ️ Command-line programs to run using the OS shell. # 📚 https://git.io/JvXDl @@ -64,4 +64,4 @@ jobs: # make bootstrap # make release - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@v3 + uses: github/codeql-action/analyze@v4 From 38a105bbe07fe2eb1beff74d80371f54cd5d7d8a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 18 Nov 2025 16:04:53 -0800 Subject: [PATCH 29/51] Changelog updates --- CHANGES.md | 38 ++++++++++++++++++++++++++++++++++++++ docs/changelog.rst | 41 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 79 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 9cab2c837..58d42472b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,41 @@ +# 2.3.0 (unreleased) + +Producer +* KIP-654: Abort transaction with pending data with TransactionAbortedError (#2662) +* KafkaProducer: Handle UnknownProducerIdError (#2663) +* KIP-467: Augment ProduceResponse error messaging for specific culprit records (#2661) +* Add `transactional_id` to KafkaProducer Keyword Arguments docstring + +Consumer +* KIP-345: Consumer group static membership (#2625) +* KIP-207: Add ListOffsetsRequest v5 / handle OffsetNotAvailableError (#2657) +* Fetcher: Add missing argument in debug log (#2665) + +AdminClient +* KIP-430: Return Authorized Operations in Describe Responses (#2656) +* Add `send_request()` and `send_requests()` to KafkaAdminClient (#2649) + +CLI +* python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) + +Maintenance +* Remove old/unused kafka.protocol.pickle (#2653) +* Switch protocol code to getattr/setattr from __dict__ (#2654) +* Drop unused imports (#2651) + +Project Infra +* Bump github/codeql-action from 3 to 4 (#2678) +* Bump actions/setup-python from 5 to 6 (#2674) +* Bump actions/setup-java from 4 to 5 (#2673) +* Bump actions/checkout from 4 to 5 (#2669) +* NixOS helpers + +# 2.2.16 (Nov 18, 2025) + +Fixes +* Fix thread not waking up when there is still data to be sent (gqmelo / #2670) +* Ensure timeout is checked after each fetch position update in `Consumer.position()` (k61n / #2668) + # 2.2.15 (July 1, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 844bdd9b6..383a777df 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,46 @@ Changelog ========= +2.3.0 (unreleased) +################## + +Producer +-------- +* KIP-654: Abort transaction with pending data with TransactionAbortedError (#2662) +* KafkaProducer: Handle UnknownProducerIdError (#2663) +* KIP-467: Augment ProduceResponse error messaging for specific culprit records (#2661) +* Add transactional_id to KafkaProducer Keyword Arguments docstring + +Consumer +-------- +* KIP-345: Consumer group static membership (#2625) +* KIP-207: Add ListOffsetsRequest v5 / handle OffsetNotAvailableError (#2657) +* Fetcher: Add missing argument in debug log (#2665) + +AdminClient +----------- +* KIP-430: Return Authorized Operations in Describe Responses (#2656) +* Add send_request() and send_requests() to KafkaAdminClient (#2649) + +CLI +--- +* python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) + +Maintenance +----------- +* Remove old/unused kafka.protocol.pickle (#2653) +* Switch protocol code to getattr/setattr from __dict__ (#2654) +* Drop unused imports (#2651) + +Project Infra +------------- +* Bump github/codeql-action from 3 to 4 (#2678) +* Bump actions/setup-python from 5 to 6 (#2674) +* Bump actions/setup-java from 4 to 5 (#2673) +* Bump actions/checkout from 4 to 5 (#2669) +* NixOS helpers + + 2.2.15 (July 1, 2025) ##################### @@ -14,6 +54,7 @@ Fixes ###################### Fixes +----- * python2 fixups (#2655) * Fix coordinator lock contention during close() (#2652) From 7e38cf2b8d60531deda3d2e65c43e304aee07726 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 20 Nov 2025 10:39:14 -0800 Subject: [PATCH 30/51] Bump actions/checkout from 5 to 6 (#2694) --- .github/workflows/codeql-analysis.yml | 2 +- .github/workflows/python-package.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index a909f7cf5..967d82c07 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -36,7 +36,7 @@ jobs: # https://docs.github.com/en/free-pro-team@latest/github/finding-security-vulnerabilities-and-errors-in-your-code/configuring-code-scanning#changing-the-languages-that-are-analyzed steps: - name: Checkout repository - uses: actions/checkout@v5 + uses: actions/checkout@v6 # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index e9fbd108a..79dcebc22 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -52,7 +52,7 @@ jobs: kafka: "4.0.0" steps: - - uses: actions/checkout@v5 + - uses: actions/checkout@v6 - name: Set up Python ${{ matrix.python }} uses: actions/setup-python@v6 with: From 8d38aa7b8da354902b36e8a98d57218831c67cd1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 11:55:22 -0800 Subject: [PATCH 31/51] Initiate Coordinator Reconnect w/ Backoff from Heartbeat Thread (#2695) --- kafka/consumer/group.py | 4 +++- kafka/coordinator/base.py | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index c7ad0f59b..36df78326 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -747,7 +747,9 @@ def _poll_once(self, timer, max_records, update_offsets=True): # We do not want to be stuck blocking in poll if we are missing some positions # since the offset lookup may be backing off after a failure - poll_timeout_ms = min(timer.timeout_ms, self._coordinator.time_to_next_poll() * 1000) + poll_timeout_ms = timer.timeout_ms + if self.config['group_id'] is not None: + poll_timeout_ms = min(poll_timeout_ms, self._coordinator.time_to_next_poll() * 1000) if not has_all_fetch_positions: log.debug('poll: do not have all fetch positions...') poll_timeout_ms = min(poll_timeout_ms, self.config['retry_backoff_ms']) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index d13ce4abb..e1d8d8336 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -1198,6 +1198,7 @@ def _run_once(self): self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) elif not self.coordinator.connected(): + self.coordinator._client.maybe_connect(self.coordinator.coordinator_id) self.coordinator._client._lock.release() self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) From 41973022190b79a998f615762ffbcb3ee1b0787f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 12:15:34 -0800 Subject: [PATCH 32/51] Add internal poll to consumer.position() (#2696) --- kafka/consumer/group.py | 11 +++++----- test/integration/test_consumer_integration.py | 22 +++++++++++++++++++ 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 36df78326..49fa3e261 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -779,13 +779,12 @@ def position(self, partition, timeout_ms=None): timer = Timer(timeout_ms) position = self._subscription.assignment[partition].position - while position is None: + while position is None and not timer.expired: # batch update fetch positions for any partitions without a valid position - if self._update_fetch_positions(timeout_ms=timer.timeout_ms): - position = self._subscription.assignment[partition].position - if timer.expired: - return None - else: + self._update_fetch_positions(timeout_ms=timer.timeout_ms) + self._client.poll(timeout_ms=timer.timeout_ms) + position = self._subscription.assignment[partition].position + if position is not None: return position.offset def highwater(self, partition): diff --git a/test/integration/test_consumer_integration.py b/test/integration/test_consumer_integration.py index 71cf2642d..6060dc830 100644 --- a/test/integration/test_consumer_integration.py +++ b/test/integration/test_consumer_integration.py @@ -302,3 +302,25 @@ def test_kafka_consumer_offsets_for_times_errors(kafka_consumer_factory, topic): with pytest.raises(KafkaTimeoutError): consumer.offsets_for_times({bad_tp: 0}) + + +@pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") +def test_kafka_consumer_position_after_seek_to_end(kafka_consumer_factory, topic, send_messages): + send_messages(range(0, 10), partition=0) + + # Start a consumer with manual partition assignment. + consumer = kafka_consumer_factory( + topics=(), + group_id=None, + enable_auto_commit=False, + ) + tp = TopicPartition(topic, 0) + consumer.assign([tp]) + + # Seek to the end of the partition, and call position() to synchronize the + # partition's offset without calling poll(). + consumer.seek_to_end(tp) + position = consumer.position(tp, timeout_ms=1000) + + # Verify we got the expected position + assert position == 10, f"Expected position 10, got {position}" From c75f97c5e4f1dd4ef61a70caf04d5cf3a0b00836 Mon Sep 17 00:00:00 2001 From: Julius Schwartzenberg Date: Thu, 20 Nov 2025 21:35:41 +0100 Subject: [PATCH 33/51] SOCKS5: support looking up names remotely (#2666) --- kafka/conn.py | 7 +++++++ kafka/socks5_wrapper.py | 36 +++++++++++++++++++++++++++++++----- 2 files changed, 38 insertions(+), 5 deletions(-) mode change 100644 => 100755 kafka/conn.py mode change 100644 => 100755 kafka/socks5_wrapper.py diff --git a/kafka/conn.py b/kafka/conn.py old mode 100644 new mode 100755 index c1d42a0b6..9c79184aa --- a/kafka/conn.py +++ b/kafka/conn.py @@ -326,6 +326,9 @@ def _dns_lookup(self): return True def _next_afi_sockaddr(self): + if self.config["socks5_proxy"] and Socks5Wrapper.use_remote_lookup(self.config["socks5_proxy"]): + return (socket.AF_UNSPEC, (self.host, self.port)) + if not self._gai: if not self._dns_lookup(): return @@ -379,6 +382,7 @@ def connect(self): self._sock_afi, self._sock_addr = next_lookup try: if self.config["socks5_proxy"] is not None: + log.debug('%s: initializing Socks5 proxy at %s', self, self.config["socks5_proxy"]) self._socks5_proxy = Socks5Wrapper(self.config["socks5_proxy"], self.afi) self._sock = self._socks5_proxy.socket(self._sock_afi, socket.SOCK_STREAM) else: @@ -864,6 +868,8 @@ def connection_delay(self): if self.disconnected() or self.connecting(): if len(self._gai) > 0: return 0 + elif self.config["socks5_proxy"] and Socks5Wrapper.use_remote_lookup(self.config["socks5_proxy"]): + return 0 else: time_waited = time.time() - self.last_attempt return max(self._reconnect_backoff - time_waited, 0) * 1000 @@ -964,6 +970,7 @@ def close(self, error=None): # the socket fd from selectors cleanly. sock = self._sock self._sock = None + self._socks5_proxy = None # drop lock before state change callback and processing futures self.config['state_change_callback'](self.node_id, sock, self) diff --git a/kafka/socks5_wrapper.py b/kafka/socks5_wrapper.py old mode 100644 new mode 100755 index 18bea7c8d..6715f2093 --- a/kafka/socks5_wrapper.py +++ b/kafka/socks5_wrapper.py @@ -64,6 +64,15 @@ def dns_lookup(cls, host, port, afi=socket.AF_UNSPEC): log.warning("DNS lookup failed for proxy %s:%d, %r", host, port, ex) return [] + @classmethod + def use_remote_lookup(cls, proxy_url): + if proxy_url is None: + return False + return urlparse(proxy_url).scheme == 'socks5h' + + def _use_remote_lookup(self): + return self._proxy_url.scheme == 'socks5h' + def socket(self, family, sock_type): """Open and record a socket. @@ -187,7 +196,10 @@ def connect_ex(self, addr): return errno.ECONNREFUSED if self._state == ProxyConnectionStates.REQUEST_SUBMIT: - if self._target_afi == socket.AF_INET: + if self._use_remote_lookup(): + addr_type = 3 + addr_len = len(addr[0]) + elif self._target_afi == socket.AF_INET: addr_type = 1 addr_len = 4 elif self._target_afi == socket.AF_INET6: @@ -200,14 +212,28 @@ def connect_ex(self, addr): return errno.ECONNREFUSED self._buffer_out = struct.pack( - "!bbbb{}sh".format(addr_len), + "!bbbb", 5, # version 1, # command: connect 0, # reserved - addr_type, # 1 for ipv4, 4 for ipv6 address - socket.inet_pton(self._target_afi, addr[0]), # either 4 or 16 bytes of actual address - addr[1], # port + addr_type, # 1 for ipv4, 4 for ipv6 address, 3 for domain name ) + # Addr format depends on type + if addr_type == 3: + # len + domain name (no null terminator) + self._buffer_out += struct.pack( + "!b{}s".format(addr_len), + addr_len, + addr[0].encode('ascii'), + ) + else: + # either 4 (type 1) or 16 (type 4) bytes of actual address + self._buffer_out += struct.pack( + "!{}s".format(addr_len), + socket.inet_pton(self._target_afi, addr[0]), + ) + self._buffer_out += struct.pack("!H", addr[1]) # port + self._state = ProxyConnectionStates.REQUESTING if self._state == ProxyConnectionStates.REQUESTING: From dc470dade907c47657ef78fc82006376230cded8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 12:51:02 -0800 Subject: [PATCH 34/51] Update changelog --- CHANGES.md | 14 ++++++++++++-- docs/changelog.rst | 26 ++++++++++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 58d42472b..69e922346 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -4,7 +4,6 @@ Producer * KIP-654: Abort transaction with pending data with TransactionAbortedError (#2662) * KafkaProducer: Handle UnknownProducerIdError (#2663) * KIP-467: Augment ProduceResponse error messaging for specific culprit records (#2661) -* Add `transactional_id` to KafkaProducer Keyword Arguments docstring Consumer * KIP-345: Consumer group static membership (#2625) @@ -30,6 +29,18 @@ Project Infra * Bump actions/checkout from 4 to 5 (#2669) * NixOS helpers +# 2.2.17 (Nov 20, 2025) + +Fixes +* Add internal poll to consumer.position() (#2696) +* Initiate Coordinator Reconnect w/ Backoff from Heartbeat Thread (#2695) + +Networking +* SOCKS5: support looking up names remotely (jschwartzenberg / #2666) + +Documentation +* Add `transactional_id` to KafkaProducer Keyword Arguments docstring + # 2.2.16 (Nov 18, 2025) Fixes @@ -64,7 +75,6 @@ Fixes * Avoid RuntimeError on mutated `_completed_fetches` deque in consumer fetcher (#2646) * Throw exception on invalid bucket type (#2642) - # 2.2.11 (June 5, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 383a777df..bacdadd75 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -41,6 +41,32 @@ Project Infra * NixOS helpers +2.2.17 (Nov 20, 2025) +##################### + +Fixes +----- +* Add internal poll to consumer.position() (#2696) +* Initiate Coordinator Reconnect w/ Backoff from Heartbeat Thread (#2695) + +Networking +---------- +* SOCKS5: support looking up names remotely (jschwartzenberg / #2666) + +Documentation +------------- +* Add `transactional_id` to KafkaProducer Keyword Arguments docstring + + +2.2.16 (Nov 18, 2025) +##################### + +Fixes +----- +* Fix thread not waking up when there is still data to be sent (gqmelo / #2670) +* Ensure timeout is checked after each fetch position update in `Consumer.position()` (k61n / #2668) + + 2.2.15 (July 1, 2025) ##################### From 84d1394e42141294249d3acc764e76a46d60da82 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 13:30:43 -0800 Subject: [PATCH 35/51] Add cli modules to docs/; add top level kafka.__main__ with help message --- docs/index.rst | 8 +++ docs/usage.rst | 89 +++++++++++++++++++++++++++++++++- kafka/__main__.py | 5 ++ kafka/cli/admin/__init__.py | 4 -- kafka/cli/consumer/__init__.py | 5 -- kafka/cli/producer/__init__.py | 5 -- 6 files changed, 101 insertions(+), 15 deletions(-) create mode 100644 kafka/__main__.py diff --git a/docs/index.rst b/docs/index.rst index 0e9aa5f7b..8ea89d2d2 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -180,6 +180,14 @@ client. See `KafkaProducer `_ for more details. metrics = producer.metrics() +Module CLI Interface +******************** + +kafka-python also provides simple command-line interfaces for consumer, producer, and admin clients. +Access via ``python -m kafka.consumer``, ``python -m kafka.producer``, and ``python -m kafka.admin``. +See `Usage `_ for more details. + + Thread safety ************* diff --git a/docs/usage.rst b/docs/usage.rst index c001ec049..ee1ac7371 100644 --- a/docs/usage.rst +++ b/docs/usage.rst @@ -2,8 +2,95 @@ Usage ***** +CLI +=== + +The kafka module provides a simple command-line interface for consumer, producer, +and admin apis. + +python -m kafka.consumer +------------------------ + +.. code:: bash + + ❯ python -m kafka.consumer --help + usage: python -m kafka.consumer [-h] -b BOOTSTRAP_SERVERS -t TOPICS -g GROUP [-c EXTRA_CONFIG] [-l LOG_LEVEL] [-f FORMAT] [--encoding ENCODING] + + Kafka console consumer + + options: + -h, --help show this help message and exit + -b BOOTSTRAP_SERVERS, --bootstrap-servers BOOTSTRAP_SERVERS + host:port for cluster bootstrap servers + -t TOPICS, --topic TOPICS + subscribe to topic + -g GROUP, --group GROUP + consumer group + -c EXTRA_CONFIG, --extra-config EXTRA_CONFIG + additional configuration properties for kafka consumer + -l LOG_LEVEL, --log-level LOG_LEVEL + logging level, passed to logging.basicConfig + -f FORMAT, --format FORMAT + output format: str|raw|full + --encoding ENCODING encoding to use for str output decode() + + +python -m kafka.producer +------------------------ + +.. code:: bash + + ❯ python -m kafka.producer --help + usage: python -m kafka.producer [-h] -b BOOTSTRAP_SERVERS -t TOPIC [-c EXTRA_CONFIG] [-l LOG_LEVEL] [--encoding ENCODING] + + Kafka console producer + + options: + -h, --help show this help message and exit + -b BOOTSTRAP_SERVERS, --bootstrap-servers BOOTSTRAP_SERVERS + host:port for cluster bootstrap servers + -t TOPIC, --topic TOPIC + publish to topic + -c EXTRA_CONFIG, --extra-config EXTRA_CONFIG + additional configuration properties for kafka producer + -l LOG_LEVEL, --log-level LOG_LEVEL + logging level, passed to logging.basicConfig + --encoding ENCODING byte encoding for produced messages + + +python -m kafka.admin +--------------------- + +.. code:: bash + + ❯ python -m kafka.admin --help + usage: python -m kafka.admin [-h] -b BOOTSTRAP_SERVERS [-c EXTRA_CONFIG] [-l LOG_LEVEL] [-f FORMAT] {cluster,configs,log-dirs,topics,consumer-groups} ... + + Kafka admin client + + positional arguments: + {cluster,configs,log-dirs,topics,consumer-groups} + subcommands + cluster Manage Kafka Cluster + configs Manage Kafka Configuration + log-dirs Manage Kafka Topic/Partition Log Directories + topics List/Describe/Create/Delete Kafka Topics + consumer-groups Manage Kafka Consumer Groups + + options: + -h, --help show this help message and exit + -b BOOTSTRAP_SERVERS, --bootstrap-servers BOOTSTRAP_SERVERS + host:port for cluster bootstrap servers + -c EXTRA_CONFIG, --extra-config EXTRA_CONFIG + additional configuration properties for admin client + -l LOG_LEVEL, --log-level LOG_LEVEL + logging level, passed to logging.basicConfig + -f FORMAT, --format FORMAT + output format: raw|json + + KafkaConsumer -============= +============== .. code:: python diff --git a/kafka/__main__.py b/kafka/__main__.py new file mode 100644 index 000000000..9e464ae2c --- /dev/null +++ b/kafka/__main__.py @@ -0,0 +1,5 @@ +import sys + +print("Available module interfaces: kafka.consumer, kafka.producer, kafka.admin") + +sys.exit(1) diff --git a/kafka/cli/admin/__init__.py b/kafka/cli/admin/__init__.py index 02971e333..d16aab802 100644 --- a/kafka/cli/admin/__init__.py +++ b/kafka/cli/admin/__init__.py @@ -83,10 +83,6 @@ def run_cli(args=None): logger.exception('Error!') return 1 -if __name__ == '__main__': - import sys - sys.exit(run_cli()) - # Commands TODO: # [acls] diff --git a/kafka/cli/consumer/__init__.py b/kafka/cli/consumer/__init__.py index fb5b232c9..32be19665 100644 --- a/kafka/cli/consumer/__init__.py +++ b/kafka/cli/consumer/__init__.py @@ -87,8 +87,3 @@ def run_cli(args=None): return 1 finally: consumer.close() - - -if __name__ == '__main__': - import sys - sys.exit(run_cli()) diff --git a/kafka/cli/producer/__init__.py b/kafka/cli/producer/__init__.py index 597235517..01d94b666 100644 --- a/kafka/cli/producer/__init__.py +++ b/kafka/cli/producer/__init__.py @@ -89,8 +89,3 @@ def log_result(res_or_err): return 1 finally: producer.close() - - -if __name__ == '__main__': - import sys - sys.exit(run_cli()) From 5904e5da352d1a318b605e1d0cf5b18cfc7e92bf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 13:45:29 -0800 Subject: [PATCH 36/51] minor api docs formatting cleanups --- docs/apidoc/BrokerConnection.rst | 3 --- docs/apidoc/ClusterMetadata.rst | 3 --- docs/apidoc/KafkaAdminClient.rst | 3 --- docs/apidoc/KafkaClient.rst | 3 --- docs/apidoc/KafkaConsumer.rst | 3 --- docs/apidoc/KafkaProducer.rst | 3 --- docs/apidoc/modules.rst | 1 + 7 files changed, 1 insertion(+), 18 deletions(-) diff --git a/docs/apidoc/BrokerConnection.rst b/docs/apidoc/BrokerConnection.rst index c56cf4271..c5366faf5 100644 --- a/docs/apidoc/BrokerConnection.rst +++ b/docs/apidoc/BrokerConnection.rst @@ -1,5 +1,2 @@ -BrokerConnection -================ - .. autoclass:: kafka.BrokerConnection :members: diff --git a/docs/apidoc/ClusterMetadata.rst b/docs/apidoc/ClusterMetadata.rst index 4b575b376..a55940329 100644 --- a/docs/apidoc/ClusterMetadata.rst +++ b/docs/apidoc/ClusterMetadata.rst @@ -1,5 +1,2 @@ -ClusterMetadata -=========== - .. autoclass:: kafka.cluster.ClusterMetadata :members: diff --git a/docs/apidoc/KafkaAdminClient.rst b/docs/apidoc/KafkaAdminClient.rst index 837b00cab..1ea2ad19e 100644 --- a/docs/apidoc/KafkaAdminClient.rst +++ b/docs/apidoc/KafkaAdminClient.rst @@ -1,5 +1,2 @@ -KafkaAdminClient -=========== - .. autoclass:: kafka.KafkaAdminClient :members: diff --git a/docs/apidoc/KafkaClient.rst b/docs/apidoc/KafkaClient.rst index 5c9d736a2..5a1630149 100644 --- a/docs/apidoc/KafkaClient.rst +++ b/docs/apidoc/KafkaClient.rst @@ -1,5 +1,2 @@ -KafkaClient -=========== - .. autoclass:: kafka.KafkaClient :members: diff --git a/docs/apidoc/KafkaConsumer.rst b/docs/apidoc/KafkaConsumer.rst index 39062c684..c8dc4cbd7 100644 --- a/docs/apidoc/KafkaConsumer.rst +++ b/docs/apidoc/KafkaConsumer.rst @@ -1,5 +1,2 @@ -KafkaConsumer -============= - .. autoclass:: kafka.KafkaConsumer :members: diff --git a/docs/apidoc/KafkaProducer.rst b/docs/apidoc/KafkaProducer.rst index 1b71c4114..e4991daec 100644 --- a/docs/apidoc/KafkaProducer.rst +++ b/docs/apidoc/KafkaProducer.rst @@ -1,5 +1,2 @@ -KafkaProducer -============= - .. autoclass:: kafka.KafkaProducer :members: diff --git a/docs/apidoc/modules.rst b/docs/apidoc/modules.rst index 066fc6523..357fb5038 100644 --- a/docs/apidoc/modules.rst +++ b/docs/apidoc/modules.rst @@ -2,6 +2,7 @@ kafka-python API **************** .. toctree:: + :maxdepth: 1 KafkaConsumer KafkaProducer From 1569334715c546250df92c44bcbc1d6f2eefb63b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 16:10:06 -0800 Subject: [PATCH 37/51] Add ProducerBatch.__lt__ for heapq (#2698) --- kafka/producer/producer_batch.py | 5 +++-- test/test_producer_batch.py | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/kafka/producer/producer_batch.py b/kafka/producer/producer_batch.py index 8be08f575..198a3a0c5 100644 --- a/kafka/producer/producer_batch.py +++ b/kafka/producer/producer_batch.py @@ -180,5 +180,6 @@ def __str__(self): return 'ProducerBatch(topic_partition=%s, record_count=%d)' % ( self.topic_partition, self.records.next_offset()) - - + # for heapq + def __lt__(self, other): + return self.created < other.created diff --git a/test/test_producer_batch.py b/test/test_producer_batch.py index bffa79fcb..7d959cbe9 100644 --- a/test/test_producer_batch.py +++ b/test/test_producer_batch.py @@ -134,3 +134,18 @@ def test_complete_exceptionally_with_null_record_errors(batch): with pytest.raises(AssertionError): _test_complete_exceptionally(batch, record_count, top_level_exception, None) + + +def test_producer_batch_lt(tp, memory_records_builder): + b1 = ProducerBatch(tp, memory_records_builder, now=1) + b2 = ProducerBatch(tp, memory_records_builder, now=2) + + assert b1 < b2 + assert not b1 < b1 + + import heapq + q = [] + heapq.heappush(q, b2) + heapq.heappush(q, b1) + assert q[0] == b1 + assert q[1] == b2 From 63d96344e0ad9b342d911f28bb5041c440130565 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 16:36:36 -0800 Subject: [PATCH 38/51] Release 2.3.0 --- CHANGES.md | 14 ++++++++++---- docs/changelog.rst | 21 +++++++++++++++------ kafka/version.py | 2 +- 3 files changed, 26 insertions(+), 11 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 69e922346..934afcacb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,4 +1,7 @@ -# 2.3.0 (unreleased) +# 2.3.0 (Nov 20, 2025) + +CLI +* python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) Producer * KIP-654: Abort transaction with pending data with TransactionAbortedError (#2662) @@ -14,9 +17,6 @@ AdminClient * KIP-430: Return Authorized Operations in Describe Responses (#2656) * Add `send_request()` and `send_requests()` to KafkaAdminClient (#2649) -CLI -* python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) - Maintenance * Remove old/unused kafka.protocol.pickle (#2653) * Switch protocol code to getattr/setattr from __dict__ (#2654) @@ -27,8 +27,14 @@ Project Infra * Bump actions/setup-python from 5 to 6 (#2674) * Bump actions/setup-java from 4 to 5 (#2673) * Bump actions/checkout from 4 to 5 (#2669) +* Bump actions/checkout from 5 to 6 (#2694) * NixOS helpers +# 2.2.18 (Nov 20, 2025) + +Fixes +* Add ProducerBatch.__lt__ for heapq (#2698) + # 2.2.17 (Nov 20, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index bacdadd75..9370da67e 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,8 +1,12 @@ Changelog ========= -2.3.0 (unreleased) -################## +2.3.0 (Nov 20, 2025) +#################### + +CLI +--- +* python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) Producer -------- @@ -22,10 +26,6 @@ AdminClient * KIP-430: Return Authorized Operations in Describe Responses (#2656) * Add send_request() and send_requests() to KafkaAdminClient (#2649) -CLI ---- -* python -m cli interfaces for kafka.admin, kafka.consumer, kafka.producer (#2650) - Maintenance ----------- * Remove old/unused kafka.protocol.pickle (#2653) @@ -38,9 +38,18 @@ Project Infra * Bump actions/setup-python from 5 to 6 (#2674) * Bump actions/setup-java from 4 to 5 (#2673) * Bump actions/checkout from 4 to 5 (#2669) +* Bump actions/checkout from 5 to 6 (#2694) * NixOS helpers +2.2.18 (Nov 20, 2025) +##################### + +Fixes +----- +* Add ProducerBatch.__lt__ for heapq (#2698) + + 2.2.17 (Nov 20, 2025) ##################### diff --git a/kafka/version.py b/kafka/version.py index 29e5e08ea..82190396f 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.3.0.dev' +__version__ = '2.3.0' From b7ef5adacee432cf5d74a862be374608f5dcdb35 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 20 Nov 2025 16:46:54 -0800 Subject: [PATCH 39/51] README / docs updates re compatibility and CLI release --- README.rst | 23 ++++++++++------------- docs/compatibility.rst | 4 ++-- docs/index.rst | 14 ++------------ 3 files changed, 14 insertions(+), 27 deletions(-) diff --git a/README.rst b/README.rst index b11868241..735257a96 100644 --- a/README.rst +++ b/README.rst @@ -22,22 +22,11 @@ Python client for the Apache Kafka distributed stream processing system. kafka-python is designed to function much like the official java client, with a sprinkling of pythonic interfaces (e.g., consumer iterators). -kafka-python is best used with newer brokers (0.9+), but is backwards-compatible with -older versions (to 0.8.0). Some features will only be enabled on newer brokers. -For example, fully coordinated consumer groups -- i.e., dynamic partition -assignment to multiple consumers in the same group -- requires use of 0.9+ kafka -brokers. Supporting this feature for earlier broker releases would require -writing and maintaining custom leadership election and membership / health -check code (perhaps using zookeeper or consul). For older brokers, you can -achieve something similar by manually assigning different partitions to each -consumer instance with config management tools like chef, ansible, etc. This -approach will work fine, though it does not support rebalancing on failures. -See https://kafka-python.readthedocs.io/en/master/compatibility.html -for more details. - Please note that the master branch may contain unreleased features. For release documentation, please see readthedocs and/or python's inline help. +New in 2.3 release: python -m kafka.* interfaces for quick scripts and testing. + .. code-block:: bash $ pip install kafka-python @@ -188,6 +177,14 @@ for more details. metrics = producer.metrics() +Module CLI Interface +******************** + +kafka-python also provides simple command-line interfaces for consumer, producer, and admin clients. +Access via ``python -m kafka.consumer``, ``python -m kafka.producer``, and ``python -m kafka.admin``. +See https://kafka-python.readthedocs.io/en/master/usage.html for more details. + + Thread safety ************* diff --git a/docs/compatibility.rst b/docs/compatibility.rst index 353273114..a3bd706f2 100644 --- a/docs/compatibility.rst +++ b/docs/compatibility.rst @@ -13,8 +13,8 @@ Because the kafka server protocol is backwards compatible, kafka-python is expected to work with newer broker releases as well. Although kafka-python is tested and expected to work on recent broker versions, -not all features are supported. Specifically, transactional producer/consumer -support is not fully implemented. PRs welcome! +not all features are supported. Please see github open issues for feature tracking. +PRs welcome! kafka-python is tested on python 2.7, and 3.8-3.13. diff --git a/docs/index.rst b/docs/index.rst index 8ea89d2d2..3b39bb965 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -16,21 +16,11 @@ Python client for the Apache Kafka distributed stream processing system. kafka-python is designed to function much like the official java client, with a sprinkling of pythonic interfaces (e.g., consumer iterators). -kafka-python is best used with newer brokers (0.9+), but is backwards-compatible with -older versions (to 0.8.0). Some features will only be enabled on newer brokers. -For example, fully coordinated consumer groups -- i.e., dynamic -partition assignment to multiple consumers in the same group -- requires use of -0.9 kafka brokers. Supporting this feature for earlier broker releases would -require writing and maintaining custom leadership election and membership / -health check code (perhaps using zookeeper or consul). For older brokers, you -can achieve something similar by manually assigning different partitions to -each consumer instance with config management tools like chef, ansible, etc. -This approach will work fine, though it does not support rebalancing on -failures. See `Compatibility `_ for more details. - Please note that the master branch may contain unreleased features. For release documentation, please see readthedocs and/or python's inline help. +New in 2.3 release: python -m kafka.* interfaces for quick scripts and testing. + .. code:: bash pip install kafka-python From 5c9be1c1207abf6afb3a69f491becf4b58348c30 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Nov 2025 09:10:26 -0800 Subject: [PATCH 40/51] Minor py2 fix for consumer integration test --- test/integration/test_consumer_integration.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/integration/test_consumer_integration.py b/test/integration/test_consumer_integration.py index 6060dc830..9f103c237 100644 --- a/test/integration/test_consumer_integration.py +++ b/test/integration/test_consumer_integration.py @@ -323,4 +323,4 @@ def test_kafka_consumer_position_after_seek_to_end(kafka_consumer_factory, topic position = consumer.position(tp, timeout_ms=1000) # Verify we got the expected position - assert position == 10, f"Expected position 10, got {position}" + assert position == 10, "Expected position 10, got {}".format(position) From c2658110e1d4fc99686a2a1d425ef7e8961cf561 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Nov 2025 10:38:14 -0800 Subject: [PATCH 41/51] .gitignore .venvs --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 8541aae23..a0a54714f 100644 --- a/.gitignore +++ b/.gitignore @@ -17,3 +17,4 @@ tests-env/ .pytest_cache/ .envrc shell.nix +.venv*/ From 9c20267de781e91558fa5f9e390df7b8e938075e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Dec 2025 04:37:24 -0800 Subject: [PATCH 42/51] Docs: Add note re debug logging; add OffsetAndMetadata/TopicPartition --- README.rst | 12 ++++++++++++ docs/apidoc/OffsetAndMetadata.rst | 2 ++ docs/apidoc/TopicPartition.rst | 2 ++ docs/index.rst | 12 ++++++++++++ 4 files changed, 28 insertions(+) create mode 100644 docs/apidoc/OffsetAndMetadata.rst create mode 100644 docs/apidoc/TopicPartition.rst diff --git a/README.rst b/README.rst index 735257a96..c48ed9a63 100644 --- a/README.rst +++ b/README.rst @@ -228,3 +228,15 @@ testing, probing, and general experimentation. The protocol support is leveraged to enable a KafkaClient.check_version() method that probes a kafka broker and attempts to identify which version it is running (0.8.0 to 2.6+). + + +Debugging +********* + +Use python's `logging` module to view internal operational events. +See https://docs.python.org/3/howto/logging.html for overview / howto. + +.. code-block:: python + + import logging + logging.basicConfig(level=logging.DEBUG) diff --git a/docs/apidoc/OffsetAndMetadata.rst b/docs/apidoc/OffsetAndMetadata.rst new file mode 100644 index 000000000..88a1eb936 --- /dev/null +++ b/docs/apidoc/OffsetAndMetadata.rst @@ -0,0 +1,2 @@ +.. autoclass:: kafka.OffsetAndMetadata + :members: diff --git a/docs/apidoc/TopicPartition.rst b/docs/apidoc/TopicPartition.rst new file mode 100644 index 000000000..bbb5d8456 --- /dev/null +++ b/docs/apidoc/TopicPartition.rst @@ -0,0 +1,2 @@ +.. autoclass:: kafka.TopicPartition + :members: diff --git a/docs/index.rst b/docs/index.rst index 3b39bb965..df019351c 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -223,6 +223,18 @@ method that probes a kafka broker and attempts to identify which version it is running (0.8.0 to 2.6+). +Debugging +********* + +Use python's `logging` module to view internal operational events. +See https://docs.python.org/3/howto/logging.html for overview / howto. + +.. code-block:: python + + import logging + logging.basicConfig(level=logging.DEBUG) + + .. toctree:: :hidden: :maxdepth: 2 From c650a77ce7a3e65e5201de30264d4806b6348b6d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Dec 2025 05:35:39 -0800 Subject: [PATCH 43/51] Drop python2 support after kafka-python 2.3.x (#2699) --- .github/workflows/python-package.yml | 4 +- docs/compatibility.rst | 3 +- kafka/__init__.py | 10 +- kafka/admin/__init__.py | 2 - kafka/admin/__main__.py | 2 - kafka/admin/acl_resource.py | 9 +- kafka/admin/client.py | 20 +- kafka/admin/config_resource.py | 9 +- kafka/admin/new_partitions.py | 3 - kafka/admin/new_topic.py | 3 - kafka/benchmarks/consumer_performance.py | 2 - kafka/benchmarks/load_example.py | 1 - kafka/benchmarks/producer_performance.py | 4 - kafka/benchmarks/record_batch_compose.py | 1 - kafka/benchmarks/record_batch_read.py | 1 - kafka/benchmarks/varint_speed.py | 30 +- kafka/cli/admin/__init__.py | 2 - kafka/cli/admin/cluster/__init__.py | 2 - kafka/cli/admin/cluster/describe.py | 3 - kafka/cli/admin/configs/__init__.py | 2 - kafka/cli/admin/configs/describe.py | 2 - kafka/cli/admin/consumer_groups/__init__.py | 2 - kafka/cli/admin/consumer_groups/delete.py | 3 - kafka/cli/admin/consumer_groups/describe.py | 3 - kafka/cli/admin/consumer_groups/list.py | 3 - .../cli/admin/consumer_groups/list_offsets.py | 3 - kafka/cli/admin/log_dirs/__init__.py | 2 - kafka/cli/admin/log_dirs/describe.py | 3 - kafka/cli/admin/topics/__init__.py | 2 - kafka/cli/admin/topics/create.py | 2 - kafka/cli/admin/topics/delete.py | 3 - kafka/cli/admin/topics/describe.py | 3 - kafka/cli/admin/topics/list.py | 3 - kafka/cli/consumer/__init__.py | 2 - kafka/cli/producer/__init__.py | 9 +- kafka/client_async.py | 40 +- kafka/cluster.py | 8 +- kafka/codec.py | 8 - kafka/conn.py | 31 +- kafka/consumer/__init__.py | 2 - kafka/consumer/__main__.py | 2 - kafka/consumer/fetcher.py | 33 +- kafka/consumer/group.py | 15 +- kafka/consumer/subscription_state.py | 39 +- kafka/coordinator/assignors/abstract.py | 2 - kafka/coordinator/assignors/range.py | 8 +- kafka/coordinator/assignors/roundrobin.py | 8 +- .../assignors/sticky/partition_movements.py | 4 +- .../assignors/sticky/sticky_assignor.py | 43 +- kafka/coordinator/base.py | 6 +- kafka/coordinator/consumer.py | 34 +- kafka/coordinator/heartbeat.py | 2 - kafka/coordinator/protocol.py | 2 - kafka/coordinator/subscription.py | 3 - kafka/errors.py | 2 - kafka/future.py | 2 - kafka/metrics/__init__.py | 2 - kafka/metrics/compound_stat.py | 6 +- kafka/metrics/dict_reporter.py | 2 - kafka/metrics/kafka_metric.py | 2 - kafka/metrics/measurable.py | 2 - kafka/metrics/measurable_stat.py | 6 +- kafka/metrics/metric_config.py | 2 - kafka/metrics/metric_name.py | 2 - kafka/metrics/metrics.py | 2 - kafka/metrics/metrics_reporter.py | 7 +- kafka/metrics/quota.py | 3 - kafka/metrics/stat.py | 7 +- kafka/metrics/stats/__init__.py | 2 - kafka/metrics/stats/avg.py | 2 - kafka/metrics/stats/count.py | 2 - kafka/metrics/stats/histogram.py | 2 - kafka/metrics/stats/max_stat.py | 2 - kafka/metrics/stats/min_stat.py | 2 - kafka/metrics/stats/percentile.py | 3 - kafka/metrics/stats/percentiles.py | 2 - kafka/metrics/stats/rate.py | 2 - kafka/metrics/stats/sampled_stat.py | 6 +- kafka/metrics/stats/sensor.py | 2 - kafka/metrics/stats/total.py | 2 - kafka/partitioner/__init__.py | 2 - kafka/partitioner/default.py | 9 - kafka/producer/__init__.py | 2 - kafka/producer/__main__.py | 2 - kafka/producer/future.py | 2 - kafka/producer/kafka.py | 33 +- kafka/producer/producer_batch.py | 10 +- kafka/producer/record_accumulator.py | 2 - kafka/producer/sender.py | 16 +- kafka/producer/transaction_manager.py | 23 +- kafka/protocol/__init__.py | 3 - kafka/protocol/abstract.py | 7 +- kafka/protocol/add_offsets_to_txn.py | 2 - kafka/protocol/add_partitions_to_txn.py | 2 - kafka/protocol/admin.py | 9 +- kafka/protocol/api.py | 10 +- kafka/protocol/api_versions.py | 2 - kafka/protocol/commit.py | 2 - kafka/protocol/end_txn.py | 2 - kafka/protocol/fetch.py | 2 - kafka/protocol/find_coordinator.py | 2 - kafka/protocol/group.py | 2 - kafka/protocol/init_producer_id.py | 2 - kafka/protocol/list_offsets.py | 2 - kafka/protocol/message.py | 2 - kafka/protocol/metadata.py | 2 - kafka/protocol/offset_for_leader_epoch.py | 2 - kafka/protocol/parser.py | 2 - kafka/protocol/produce.py | 2 - kafka/protocol/sasl_authenticate.py | 2 - kafka/protocol/sasl_handshake.py | 2 - kafka/protocol/struct.py | 2 - kafka/protocol/txn_offset_commit.py | 2 - kafka/protocol/types.py | 2 - kafka/record/abc.py | 16 +- kafka/record/memory_records.py | 1 - kafka/sasl/__init__.py | 2 - kafka/sasl/abc.py | 7 +- kafka/sasl/gssapi.py | 2 - kafka/sasl/msk.py | 4 +- kafka/sasl/oauth.py | 2 - kafka/sasl/plain.py | 2 - kafka/sasl/scram.py | 11 +- kafka/sasl/sspi.py | 2 - kafka/serializer/__init__.py | 2 - kafka/serializer/abstract.py | 2 - kafka/socks5_wrapper.py | 6 +- kafka/structs.py | 1 - kafka/util.py | 27 +- kafka/vendor/enum34.py | 841 -------------- kafka/vendor/selectors34.py | 641 ----------- kafka/vendor/six.py | 1004 ----------------- kafka/vendor/socketpair.py | 75 -- pylint.rc | 2 +- pyproject.toml | 7 +- requirements-dev.txt | 1 - test/__init__.py | 2 - test/conftest.py | 2 - test/integration/conftest.py | 4 +- test/integration/fixtures.py | 5 +- test/integration/test_consumer_group.py | 11 +- test/integration/test_consumer_integration.py | 6 +- test/integration/test_producer_integration.py | 2 - test/record/test_default_records.py | 9 +- test/record/test_legacy_records.py | 8 +- test/record/test_records.py | 1 - test/sasl/test_gssapi.py | 7 +- test/sasl/test_msk.py | 6 +- test/service.py | 2 - test/test_assignors.py | 54 +- test/test_client_async.py | 10 +- test/test_cluster.py | 1 - test/test_codec.py | 3 - test/test_conn.py | 13 +- test/test_consumer.py | 2 - test/test_coordinator.py | 1 - test/test_fetcher.py | 1 - test/test_partitioner.py | 2 - test/test_producer.py | 2 - test/test_producer_batch.py | 1 - test/test_record_accumulator.py | 1 - test/test_sender.py | 8 +- test/test_subscription_state.py | 7 +- test/test_util.py | 1 - test/testutil.py | 2 - 165 files changed, 207 insertions(+), 3319 deletions(-) delete mode 100644 kafka/vendor/enum34.py delete mode 100644 kafka/vendor/selectors34.py delete mode 100644 kafka/vendor/six.py delete mode 100644 kafka/vendor/socketpair.py diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 79dcebc22..21acc4a32 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -35,7 +35,7 @@ jobs: - "3.9.0" - "4.0.0" python: - - "3.13" + - "3.14" include: #- python: "pypy3.9" # kafka: "2.6.0" @@ -50,6 +50,8 @@ jobs: kafka: "4.0.0" - python: "3.12" kafka: "4.0.0" + - python: "3.13" + kafka: "4.0.0" steps: - uses: actions/checkout@v6 diff --git a/docs/compatibility.rst b/docs/compatibility.rst index a3bd706f2..58457f17d 100644 --- a/docs/compatibility.rst +++ b/docs/compatibility.rst @@ -16,6 +16,7 @@ Although kafka-python is tested and expected to work on recent broker versions, not all features are supported. Please see github open issues for feature tracking. PRs welcome! -kafka-python is tested on python 2.7, and 3.8-3.13. +kafka-python is tested on python 3.8-3.14. +python 2.7 was supported through kafka-python release 2.3. Builds and tests via Github Actions Workflows. See https://github.com/dpkp/kafka-python/actions diff --git a/kafka/__init__.py b/kafka/__init__.py index 41a014072..e394188f8 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - __title__ = 'kafka' from kafka.version import __version__ __author__ = 'Dana Powers' @@ -8,14 +6,8 @@ # Set default logging handler to avoid "No handler found" warnings. import logging -try: # Python 2.7+ - from logging import NullHandler -except ImportError: - class NullHandler(logging.Handler): - def emit(self, record): - pass -logging.getLogger(__name__).addHandler(NullHandler()) +logging.getLogger(__name__).addHandler(logging.NullHandler()) from kafka.admin import KafkaAdminClient diff --git a/kafka/admin/__init__.py b/kafka/admin/__init__.py index c240fc6d0..c67fb9e6a 100644 --- a/kafka/admin/__init__.py +++ b/kafka/admin/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.admin.config_resource import ConfigResource, ConfigResourceType from kafka.admin.client import KafkaAdminClient from kafka.admin.acl_resource import (ACL, ACLFilter, ResourcePattern, ResourcePatternFilter, ACLOperation, diff --git a/kafka/admin/__main__.py b/kafka/admin/__main__.py index 776063869..74dc8b33b 100644 --- a/kafka/admin/__main__.py +++ b/kafka/admin/__main__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from kafka.cli.admin import run_cli diff --git a/kafka/admin/acl_resource.py b/kafka/admin/acl_resource.py index 8ae1e978d..2ac4b4f43 100644 --- a/kafka/admin/acl_resource.py +++ b/kafka/admin/acl_resource.py @@ -1,11 +1,4 @@ -from __future__ import absolute_import - -# enum in stdlib as of py3.4 -try: - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum +from enum import IntEnum from kafka.errors import IllegalArgumentError diff --git a/kafka/admin/client.py b/kafka/admin/client.py index f21ac97f9..6bdac5613 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - from collections import defaultdict import copy import itertools @@ -8,7 +6,6 @@ import time from . import ConfigResourceType -from kafka.vendor import six from kafka.admin.acl_resource import ACLOperation, ACLPermissionType, ACLFilter, ACL, ResourcePattern, ResourceType, \ ACLResourcePatternType, valid_acl_operations @@ -122,8 +119,7 @@ class KafkaAdminClient(object): ssl_crlfile (str): Optional filename containing the CRL to check for certificate expiration. By default, no CRL check is done. When providing a file, only the leaf certificate will be checked against - this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+. - Default: None. + this CRL. Default: None. api_version (tuple): Specify which Kafka API version to use. If set to None, KafkaClient will attempt to infer the broker version by probing various APIs. Example: (0, 10, 2). Default: None @@ -420,11 +416,7 @@ def _send_request_to_controller(self, request): raise RuntimeError("This should never happen, please file a bug with full stacktrace if encountered") def _parse_topic_request_response(self, topic_error_tuples, request, response, tries): - # Also small py2/py3 compatibility -- py3 can ignore extra values - # during unpack via: for x, y, *rest in list_of_values. py2 cannot. - # So for now we have to map across the list and explicitly drop any - # extra values (usually the error_message) - for topic, error_code in map(lambda e: e[:2], topic_error_tuples): + for topic, error_code, *_ in topic_error_tuples: error_type = Errors.for_code(error_code) if tries and error_type is Errors.NotControllerError: # No need to inspect the rest of the errors for @@ -439,12 +431,8 @@ def _parse_topic_request_response(self, topic_error_tuples, request, response, t return True def _parse_topic_partition_request_response(self, request, response, tries): - # Also small py2/py3 compatibility -- py3 can ignore extra values - # during unpack via: for x, y, *rest in list_of_values. py2 cannot. - # So for now we have to map across the list and explicitly drop any - # extra values (usually the error_message) for topic, partition_results in response.replication_election_results: - for partition_id, error_code in map(lambda e: e[:2], partition_results): + for partition_id, error_code, *_ in partition_results: error_type = Errors.for_code(error_code) if tries and error_type is Errors.NotControllerError: # No need to inspect the rest of the errors for @@ -1418,7 +1406,7 @@ def _list_consumer_group_offsets_request(self, group_id, partitions=None): topics_partitions_dict = defaultdict(set) for topic, partition in partitions: topics_partitions_dict[topic].add(partition) - topics_partitions = list(six.iteritems(topics_partitions_dict)) + topics_partitions = list(topics_partitions_dict.items()) return OffsetFetchRequest[version](group_id, topics_partitions) def _list_consumer_group_offsets_process_response(self, response): diff --git a/kafka/admin/config_resource.py b/kafka/admin/config_resource.py index 06754ba9f..347ff915f 100644 --- a/kafka/admin/config_resource.py +++ b/kafka/admin/config_resource.py @@ -1,11 +1,4 @@ -from __future__ import absolute_import - -# enum in stdlib as of py3.4 -try: - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum +from enum import IntEnum class ConfigResourceType(IntEnum): diff --git a/kafka/admin/new_partitions.py b/kafka/admin/new_partitions.py index 429b2e190..bc3b8172e 100644 --- a/kafka/admin/new_partitions.py +++ b/kafka/admin/new_partitions.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class NewPartitions(object): """A class for new partition creation on existing topics. Note that the length of new_assignments, if specified, must be the difference between the new total number of partitions and the existing number of partitions. diff --git a/kafka/admin/new_topic.py b/kafka/admin/new_topic.py index e43c52226..931e73448 100644 --- a/kafka/admin/new_topic.py +++ b/kafka/admin/new_topic.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class NewTopic(object): """ A class for new topic creation Arguments: diff --git a/kafka/benchmarks/consumer_performance.py b/kafka/benchmarks/consumer_performance.py index c35a164c2..e76264145 100644 --- a/kafka/benchmarks/consumer_performance.py +++ b/kafka/benchmarks/consumer_performance.py @@ -1,8 +1,6 @@ #!/usr/bin/env python # Adapted from https://github.com/mrafayaleem/kafka-jython -from __future__ import absolute_import, print_function - import argparse import pprint import sys diff --git a/kafka/benchmarks/load_example.py b/kafka/benchmarks/load_example.py index 29796a74c..6f24e9b37 100644 --- a/kafka/benchmarks/load_example.py +++ b/kafka/benchmarks/load_example.py @@ -1,5 +1,4 @@ #!/usr/bin/env python -from __future__ import print_function import argparse import logging diff --git a/kafka/benchmarks/producer_performance.py b/kafka/benchmarks/producer_performance.py index 1a1092960..df7c51f29 100644 --- a/kafka/benchmarks/producer_performance.py +++ b/kafka/benchmarks/producer_performance.py @@ -1,8 +1,6 @@ #!/usr/bin/env python # Adapted from https://github.com/mrafayaleem/kafka-jython -from __future__ import absolute_import, print_function - import argparse import pprint import sys @@ -10,8 +8,6 @@ import time import traceback -from kafka.vendor.six.moves import range - from kafka import KafkaProducer diff --git a/kafka/benchmarks/record_batch_compose.py b/kafka/benchmarks/record_batch_compose.py index 5b07fd59a..b7d05e37f 100644 --- a/kafka/benchmarks/record_batch_compose.py +++ b/kafka/benchmarks/record_batch_compose.py @@ -1,5 +1,4 @@ #!/usr/bin/env python3 -from __future__ import print_function import hashlib import itertools import os diff --git a/kafka/benchmarks/record_batch_read.py b/kafka/benchmarks/record_batch_read.py index 2ef32298d..ab5ae5227 100644 --- a/kafka/benchmarks/record_batch_read.py +++ b/kafka/benchmarks/record_batch_read.py @@ -1,5 +1,4 @@ #!/usr/bin/env python -from __future__ import print_function import hashlib import itertools import os diff --git a/kafka/benchmarks/varint_speed.py b/kafka/benchmarks/varint_speed.py index b2628a1b5..de57b6198 100644 --- a/kafka/benchmarks/varint_speed.py +++ b/kafka/benchmarks/varint_speed.py @@ -1,7 +1,5 @@ #!/usr/bin/env python -from __future__ import print_function import pyperf -from kafka.vendor import six test_data = [ @@ -114,7 +112,10 @@ def encode_varint_1(num): return buf[:i + 1] -def encode_varint_2(value, int2byte=six.int2byte): +def int2byte(i): + return bytes((i,)) + +def encode_varint_2(value): value = (value << 1) ^ (value >> 63) bits = value & 0x7f @@ -141,7 +142,7 @@ def encode_varint_3(value, buf): return value -def encode_varint_4(value, int2byte=six.int2byte): +def encode_varint_4(value): value = (value << 1) ^ (value >> 63) if value <= 0x7f: # 1 byte @@ -269,22 +270,13 @@ def size_of_varint_2(value): return 10 -if six.PY3: - def _read_byte(memview, pos): - """ Read a byte from memoryview as an integer - - Raises: - IndexError: if position is out of bounds - """ - return memview[pos] -else: - def _read_byte(memview, pos): - """ Read a byte from memoryview as an integer +def _read_byte(memview, pos): + """ Read a byte from memoryview as an integer - Raises: - IndexError: if position is out of bounds - """ - return ord(memview[pos]) + Raises: + IndexError: if position is out of bounds + """ + return memview[pos] def decode_varint_1(buffer, pos=0): diff --git a/kafka/cli/admin/__init__.py b/kafka/cli/admin/__init__.py index d16aab802..dd20ea6de 100644 --- a/kafka/cli/admin/__init__.py +++ b/kafka/cli/admin/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import argparse import json import logging diff --git a/kafka/cli/admin/cluster/__init__.py b/kafka/cli/admin/cluster/__init__.py index 735228565..7876c0afa 100644 --- a/kafka/cli/admin/cluster/__init__.py +++ b/kafka/cli/admin/cluster/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from .describe import DescribeCluster diff --git a/kafka/cli/admin/cluster/describe.py b/kafka/cli/admin/cluster/describe.py index 6a2ff06e5..40cc4c0ac 100644 --- a/kafka/cli/admin/cluster/describe.py +++ b/kafka/cli/admin/cluster/describe.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class DescribeCluster: @classmethod diff --git a/kafka/cli/admin/configs/__init__.py b/kafka/cli/admin/configs/__init__.py index 7ec6d1042..75b555f40 100644 --- a/kafka/cli/admin/configs/__init__.py +++ b/kafka/cli/admin/configs/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from .describe import DescribeConfigs diff --git a/kafka/cli/admin/configs/describe.py b/kafka/cli/admin/configs/describe.py index 3ff366667..0f32a744f 100644 --- a/kafka/cli/admin/configs/describe.py +++ b/kafka/cli/admin/configs/describe.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.admin.config_resource import ConfigResource diff --git a/kafka/cli/admin/consumer_groups/__init__.py b/kafka/cli/admin/consumer_groups/__init__.py index cfb1bdb4f..dea4513cb 100644 --- a/kafka/cli/admin/consumer_groups/__init__.py +++ b/kafka/cli/admin/consumer_groups/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from .delete import DeleteConsumerGroups diff --git a/kafka/cli/admin/consumer_groups/delete.py b/kafka/cli/admin/consumer_groups/delete.py index 5724ae551..40fc593ac 100644 --- a/kafka/cli/admin/consumer_groups/delete.py +++ b/kafka/cli/admin/consumer_groups/delete.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class DeleteConsumerGroups: @classmethod diff --git a/kafka/cli/admin/consumer_groups/describe.py b/kafka/cli/admin/consumer_groups/describe.py index 02298e9a4..e1f3393e2 100644 --- a/kafka/cli/admin/consumer_groups/describe.py +++ b/kafka/cli/admin/consumer_groups/describe.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class DescribeConsumerGroups: @classmethod diff --git a/kafka/cli/admin/consumer_groups/list.py b/kafka/cli/admin/consumer_groups/list.py index 6c02f3bee..dfeb954f3 100644 --- a/kafka/cli/admin/consumer_groups/list.py +++ b/kafka/cli/admin/consumer_groups/list.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class ListConsumerGroups: @classmethod diff --git a/kafka/cli/admin/consumer_groups/list_offsets.py b/kafka/cli/admin/consumer_groups/list_offsets.py index 7c05c5aae..ff8f33acb 100644 --- a/kafka/cli/admin/consumer_groups/list_offsets.py +++ b/kafka/cli/admin/consumer_groups/list_offsets.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class ListConsumerGroupOffsets: @classmethod diff --git a/kafka/cli/admin/log_dirs/__init__.py b/kafka/cli/admin/log_dirs/__init__.py index 3a4a94351..46f5a254f 100644 --- a/kafka/cli/admin/log_dirs/__init__.py +++ b/kafka/cli/admin/log_dirs/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from .describe import DescribeLogDirs diff --git a/kafka/cli/admin/log_dirs/describe.py b/kafka/cli/admin/log_dirs/describe.py index a0c301f82..6c3c27bea 100644 --- a/kafka/cli/admin/log_dirs/describe.py +++ b/kafka/cli/admin/log_dirs/describe.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class DescribeLogDirs: @classmethod diff --git a/kafka/cli/admin/topics/__init__.py b/kafka/cli/admin/topics/__init__.py index dbf569445..bcb1973af 100644 --- a/kafka/cli/admin/topics/__init__.py +++ b/kafka/cli/admin/topics/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from .create import CreateTopic diff --git a/kafka/cli/admin/topics/create.py b/kafka/cli/admin/topics/create.py index d033f6b30..0844b411b 100644 --- a/kafka/cli/admin/topics/create.py +++ b/kafka/cli/admin/topics/create.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.admin.new_topic import NewTopic diff --git a/kafka/cli/admin/topics/delete.py b/kafka/cli/admin/topics/delete.py index a88400ef2..70a1e749b 100644 --- a/kafka/cli/admin/topics/delete.py +++ b/kafka/cli/admin/topics/delete.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class DeleteTopic: @classmethod diff --git a/kafka/cli/admin/topics/describe.py b/kafka/cli/admin/topics/describe.py index 2e96871d6..b053524b4 100644 --- a/kafka/cli/admin/topics/describe.py +++ b/kafka/cli/admin/topics/describe.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class DescribeTopics: @classmethod diff --git a/kafka/cli/admin/topics/list.py b/kafka/cli/admin/topics/list.py index 2dbf3828e..2c5d48ccc 100644 --- a/kafka/cli/admin/topics/list.py +++ b/kafka/cli/admin/topics/list.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class ListTopics: @classmethod diff --git a/kafka/cli/consumer/__init__.py b/kafka/cli/consumer/__init__.py index 32be19665..89a6fb0f4 100644 --- a/kafka/cli/consumer/__init__.py +++ b/kafka/cli/consumer/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, print_function - import argparse import logging diff --git a/kafka/cli/producer/__init__.py b/kafka/cli/producer/__init__.py index 01d94b666..d9210317d 100644 --- a/kafka/cli/producer/__init__.py +++ b/kafka/cli/producer/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, print_function - import argparse import logging import sys @@ -67,15 +65,10 @@ def log_result(res_or_err): else: logger.info("Message produced: %s", res_or_err) - try: - input_py23 = raw_input - except NameError: - input_py23 = input - try: while True: try: - value = input_py23() + value = input() except EOFError: value = sys.stdin.read().rstrip('\n') if not value: diff --git a/kafka/client_async.py b/kafka/client_async.py index de20c218d..734efb65b 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -1,23 +1,13 @@ -from __future__ import absolute_import, division - import collections import copy import logging import random +import selectors import socket import threading import time import weakref -# selectors in stdlib as of py3.4 -try: - import selectors # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor import selectors34 as selectors - -from kafka.vendor import six - from kafka.cluster import ClusterMetadata from kafka.conn import BrokerConnection, ConnectionStates, get_ip_port_afi from kafka import errors as Errors @@ -28,14 +18,8 @@ from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.metadata import MetadataRequest from kafka.util import Dict, Timer, WeakMethod, ensure_valid_topic_name -# Although this looks unused, it actually monkey-patches socket.socketpair() -# and should be left in as long as we're using socket.socketpair() in this file -from kafka.vendor import socketpair # noqa: F401 from kafka.version import __version__ -if six.PY2: - ConnectionError = None - log = logging.getLogger('kafka.client') @@ -787,7 +771,7 @@ def _poll(self, timeout): if conn not in processed and conn.connected() and conn._sock.pending(): self._pending_completion.extend(conn.recv()) - for conn in six.itervalues(self._conns): + for conn in self._conns.values(): if conn.requests_timed_out(): timed_out = conn.timed_out_ifrs() timeout_ms = (timed_out[0][2] - timed_out[0][1]) * 1000 @@ -926,7 +910,7 @@ def add_topic(self, topic): def _next_ifr_request_timeout_ms(self): if self._conns: - return min([conn.next_ifr_request_timeout_ms() for conn in six.itervalues(self._conns)]) + return min([conn.next_ifr_request_timeout_ms() for conn in self._conns.values()]) else: return float('inf') @@ -1194,14 +1178,6 @@ def send_and_receive(self, node_id, request): return future.value -# OrderedDict requires python2.7+ -try: - from collections import OrderedDict -except ImportError: - # If we dont have OrderedDict, we'll fallback to dict with O(n) priority reads - OrderedDict = dict - - class IdleConnectionManager(object): def __init__(self, connections_max_idle_ms): if connections_max_idle_ms > 0: @@ -1210,7 +1186,7 @@ def __init__(self, connections_max_idle_ms): self.connections_max_idle = float('inf') self.next_idle_close_check_time = None self.update_next_idle_close_check_time(time.time()) - self.lru_connections = OrderedDict() + self.lru_connections = collections.OrderedDict() def update(self, conn_id): # order should reflect last-update @@ -1248,13 +1224,7 @@ def poll_expired_connection(self): oldest_conn_id = None oldest_ts = None - if OrderedDict is dict: - for conn_id, ts in self.lru_connections.items(): - if oldest_conn_id is None or ts < oldest_ts: - oldest_conn_id = conn_id - oldest_ts = ts - else: - (oldest_conn_id, oldest_ts) = next(iter(self.lru_connections.items())) + (oldest_conn_id, oldest_ts) = next(iter(self.lru_connections.items())) self.update_next_idle_close_check_time(oldest_ts) diff --git a/kafka/cluster.py b/kafka/cluster.py index 9e819246e..09832fde2 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import collections import copy import logging @@ -8,8 +6,6 @@ import threading import time -from kafka.vendor import six - from kafka import errors as Errors from kafka.conn import get_ip_port_afi from kafka.future import Future @@ -133,7 +129,7 @@ def available_partitions_for_topic(self, topic): if topic not in self._partitions: return None return set([partition for partition, metadata - in six.iteritems(self._partitions[topic]) + in self._partitions[topic].items() if metadata.leader != -1]) def leader_for_partition(self, partition): @@ -435,7 +431,7 @@ def collect_hosts(hosts, randomize=True): randomize the returned list. """ - if isinstance(hosts, six.string_types): + if isinstance(hosts, str): hosts = hosts.strip().split(',') result = [] diff --git a/kafka/codec.py b/kafka/codec.py index b73df060d..7177a646f 100644 --- a/kafka/codec.py +++ b/kafka/codec.py @@ -1,12 +1,8 @@ -from __future__ import absolute_import - import gzip import io import platform import struct -from kafka.vendor import six -from kafka.vendor.six.moves import range _XERIAL_V1_HEADER = (-126, b'S', b'N', b'A', b'P', b'P', b'Y', 0, 1, 1) _XERIAL_V1_FORMAT = 'bccccccBii' @@ -149,10 +145,6 @@ def snappy_encode(payload, xerial_compatible=True, xerial_blocksize=32*1024): # buffer... likely a python-snappy bug, so just use a slice copy chunker = lambda payload, i, size: payload[i:size+i] - elif six.PY2: - # Sliced buffer avoids additional copies - # pylint: disable-msg=undefined-variable - chunker = lambda payload, i, size: buffer(payload, i, size) else: # snappy.compress does not like raw memoryviews, so we have to convert # tobytes, which is a copy... oh well. it's the thought that counts. diff --git a/kafka/conn.py b/kafka/conn.py index 9c79184aa..a213a4c53 100755 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -1,24 +1,13 @@ -from __future__ import absolute_import, division - import copy import errno import io import logging from random import uniform - -# selectors in stdlib as of py3.4 -try: - import selectors # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor import selectors34 as selectors - +import selectors import socket import threading import time -from kafka.vendor import six - import kafka.errors as Errors from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate @@ -40,11 +29,6 @@ from kafka.version import __version__ -if six.PY2: - ConnectionError = socket.error - TimeoutError = socket.error - BlockingIOError = Exception - log = logging.getLogger(__name__) DEFAULT_KAFKA_PORT = 9092 @@ -720,13 +704,9 @@ def _send_bytes(self, data): except (SSLWantReadError, SSLWantWriteError): break except (ConnectionError, TimeoutError) as e: - if six.PY2 and e.errno == errno.EWOULDBLOCK: - break raise except BlockingIOError: - if six.PY3: - break - raise + break return total_sent def _send_bytes_blocking(self, data): @@ -1183,17 +1163,12 @@ def _recv(self): except (SSLWantReadError, SSLWantWriteError): break except (ConnectionError, TimeoutError) as e: - if six.PY2 and e.errno == errno.EWOULDBLOCK: - break log.exception('%s: Error receiving network data' ' closing socket', self) err = Errors.KafkaConnectionError(e) break except BlockingIOError: - if six.PY3: - break - # For PY2 this is a catchall and should be re-raised - raise + break # Only process bytes if there was no connection exception if err is None: diff --git a/kafka/consumer/__init__.py b/kafka/consumer/__init__.py index e09bcc1b8..5341d5648 100644 --- a/kafka/consumer/__init__.py +++ b/kafka/consumer/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.consumer.group import KafkaConsumer __all__ = [ diff --git a/kafka/consumer/__main__.py b/kafka/consumer/__main__.py index 0356a1aae..c2de706aa 100644 --- a/kafka/consumer/__main__.py +++ b/kafka/consumer/__main__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from kafka.cli.consumer import run_cli diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 8e524790c..8c2a41ba0 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import collections import copy import itertools @@ -7,8 +5,6 @@ import sys import time -from kafka.vendor import six - import kafka.errors as Errors from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate @@ -55,7 +51,7 @@ class RecordTooLargeError(Errors.KafkaError): pass -class Fetcher(six.Iterator): +class Fetcher(object): DEFAULT_CONFIG = { 'key_deserializer': None, 'value_deserializer': None, @@ -147,7 +143,7 @@ def send_fetches(self): List of Futures: each future resolves to a FetchResponse """ futures = [] - for node_id, (request, fetch_offsets) in six.iteritems(self._create_fetch_requests()): + for node_id, (request, fetch_offsets) in self._create_fetch_requests().items(): log.debug("Sending FetchRequest to node %s", node_id) self._nodes_with_pending_fetch_requests.add(node_id) future = self._client.send(node_id, request, wakeup=False) @@ -421,7 +417,7 @@ def _reset_offset_if_needed(self, partition, timestamp, offset): def _reset_offsets_async(self, timestamps): timestamps_by_node = self._group_list_offset_requests(timestamps) - for node_id, timestamps_and_epochs in six.iteritems(timestamps_by_node): + for node_id, timestamps_and_epochs in timestamps_by_node.items(): if not self._client.ready(node_id): continue partitions = set(timestamps_and_epochs.keys()) @@ -434,7 +430,7 @@ def on_success(timestamps_and_epochs, result): self._subscriptions.reset_failed(partitions_to_retry, time.time() + self.config['retry_backoff_ms'] / 1000) self._client.cluster.request_update() - for partition, offset in six.iteritems(fetched_offsets): + for partition, offset in fetched_offsets.items(): ts, _epoch = timestamps_and_epochs[partition] self._reset_offset_if_needed(partition, ts, offset.offset) @@ -484,7 +480,7 @@ def on_fail(err): if not list_offsets_future.is_done: list_offsets_future.failure(err) - for node_id, timestamps in six.iteritems(timestamps_by_node): + for node_id, timestamps in timestamps_by_node.items(): _f = self._send_list_offsets_request(node_id, timestamps) _f.add_callback(on_success, remaining_responses) _f.add_errback(on_fail) @@ -492,7 +488,7 @@ def on_fail(err): def _group_list_offset_requests(self, timestamps): timestamps_by_node = collections.defaultdict(dict) - for partition, timestamp in six.iteritems(timestamps): + for partition, timestamp in timestamps.items(): node_id = self._client.cluster.leader_for_partition(partition) if node_id is None: self._client.add_topic(partition.topic) @@ -512,7 +508,7 @@ def _send_list_offsets_request(self, node_id, timestamps_and_epochs): if self.config['isolation_level'] == 'read_committed' and version < 2: raise Errors.UnsupportedVersionError('read_committed isolation level requires ListOffsetsRequest >= v2') by_topic = collections.defaultdict(list) - for tp, (timestamp, leader_epoch) in six.iteritems(timestamps_and_epochs): + for tp, (timestamp, leader_epoch) in timestamps_and_epochs.items(): if version >= 4: data = (tp.partition, leader_epoch, timestamp) elif version >= 1: @@ -525,11 +521,11 @@ def _send_list_offsets_request(self, node_id, timestamps_and_epochs): request = ListOffsetsRequest[version]( -1, self._isolation_level, - list(six.iteritems(by_topic))) + list(by_topic.items())) else: request = ListOffsetsRequest[version]( -1, - list(six.iteritems(by_topic))) + list(by_topic.items())) # Client returns a future that only fails on network issues # so create a separate future and attach a callback to update it @@ -697,7 +693,7 @@ def _create_fetch_requests(self): partition, position.offset) requests = {} - for node_id, next_partitions in six.iteritems(fetchable): + for node_id, next_partitions in fetchable.items(): if version >= 7 and self.config['enable_incremental_fetch_sessions']: if node_id not in self._session_handlers: self._session_handlers[node_id] = FetchSessionHandler(node_id) @@ -741,7 +737,7 @@ def _create_fetch_requests(self): session.to_forget) fetch_offsets = {} - for tp, partition_data in six.iteritems(next_partitions): + for tp, partition_data in next_partitions.items(): if version <= 8: offset = partition_data[1] else: @@ -953,9 +949,6 @@ def _maybe_skip_record(self, record): def __bool__(self): return self.record_iterator is not None - # py2 - __nonzero__ = __bool__ - def drain(self): if self.record_iterator is not None: self.record_iterator = None @@ -1283,7 +1276,7 @@ def to_send(self): # Return as list of [(topic, [(partition, ...), ...]), ...] # so it can be passed directly to encoder partition_data = collections.defaultdict(list) - for tp, partition_info in six.iteritems(self._to_send): + for tp, partition_info in self._to_send.items(): partition_data[tp.topic].append(partition_info) return list(partition_data.items()) @@ -1334,7 +1327,7 @@ def record(self, partition, num_bytes, num_records): if not self.unrecorded_partitions: self.sensors.bytes_fetched.record(self.fetch_metrics.total_bytes) self.sensors.records_fetched.record(self.fetch_metrics.total_records) - for topic, metrics in six.iteritems(self.topic_fetch_metrics): + for topic, metrics in self.topic_fetch_metrics.items(): self.sensors.record_topic_fetch_metrics(topic, metrics.total_bytes, metrics.total_records) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 49fa3e261..7c8b1cb2f 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import copy import logging import re @@ -8,8 +6,6 @@ from kafka.errors import KafkaConfigurationError, UnsupportedVersionError -from kafka.vendor import six - from kafka.client_async import KafkaClient, selectors from kafka.consumer.fetcher import Fetcher from kafka.consumer.subscription_state import SubscriptionState @@ -25,7 +21,7 @@ log = logging.getLogger(__name__) -class KafkaConsumer(six.Iterator): +class KafkaConsumer(object): """Consume records from a Kafka cluster. The consumer will transparently handle the failure of servers in the Kafka @@ -205,8 +201,7 @@ class KafkaConsumer(six.Iterator): ssl_crlfile (str): Optional filename containing the CRL to check for certificate expiration. By default, no CRL check is done. When providing a file, only the leaf certificate will be checked against - this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+. - Default: None. + this CRL. Default: None. ssl_ciphers (str): optionally set the available ciphers for ssl connections. It should be a string in the OpenSSL cipher list format. If no cipher can be selected (because compile-time options @@ -1027,7 +1022,7 @@ def metrics(self, raw=False): return self._metrics.metrics.copy() metrics = {} - for k, v in six.iteritems(self._metrics.metrics.copy()): + for k, v in self._metrics.metrics.copy().items(): if k.group not in metrics: metrics[k.group] = {} if k.name not in metrics[k.group]: @@ -1072,7 +1067,7 @@ def offsets_for_times(self, timestamps): raise UnsupportedVersionError( "offsets_for_times API not supported for cluster version {}" .format(self.config['api_version'])) - for tp, ts in six.iteritems(timestamps): + for tp, ts in timestamps.items(): timestamps[tp] = int(ts) if ts < 0: raise ValueError( @@ -1183,7 +1178,7 @@ def _update_fetch_positions(self, timeout_ms=None): def _message_generator_v2(self): timeout_ms = 1000 * max(0, self._consumer_timeout - time.time()) record_map = self.poll(timeout_ms=timeout_ms, update_offsets=False) - for tp, records in six.iteritems(record_map): + for tp, records in record_map.items(): # Generators are stateful, and it is possible that the tp / records # here may become stale during iteration -- i.e., we seek to a # different offset, pause consumption, or lose assignment. diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index f99f01615..578e70a07 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -1,25 +1,13 @@ -from __future__ import absolute_import - import abc from collections import OrderedDict -try: - from collections.abc import Sequence -except ImportError: - from collections import Sequence -try: - # enum in stdlib as of py3.4 - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum +from collections.abc import Sequence +from enum import IntEnum import logging import random import re import threading import time -from kafka.vendor import six - import kafka.errors as Errors from kafka.protocol.list_offsets import OffsetResetStrategy from kafka.structs import OffsetAndMetadata @@ -167,7 +155,7 @@ def change_subscription(self, topics): if not self.partitions_auto_assigned(): raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) - if isinstance(topics, six.string_types): + if isinstance(topics, str): topics = [topics] if self.subscription == set(topics): @@ -256,13 +244,13 @@ def assign_from_subscribed(self, assignments): def _set_assignment(self, partition_states, randomize=False): """Batch partition assignment by topic (self.assignment is OrderedDict)""" self.assignment.clear() - topics = [tp.topic for tp in six.iterkeys(partition_states)] + topics = [tp.topic for tp in partition_states] if randomize: random.shuffle(topics) topic_partitions = OrderedDict({topic: [] for topic in topics}) - for tp in six.iterkeys(partition_states): + for tp in partition_states: topic_partitions[tp.topic].append(tp) - for topic in six.iterkeys(topic_partitions): + for topic in topic_partitions: for tp in topic_partitions[topic]: self.assignment[tp] = partition_states[tp] @@ -324,7 +312,7 @@ def paused_partitions(self): def fetchable_partitions(self): """Return ordered list of TopicPartitions that should be Fetched.""" fetchable = list() - for partition, state in six.iteritems(self.assignment): + for partition, state in self.assignment.items(): if state.is_fetchable(): fetchable.append(partition) return fetchable @@ -338,7 +326,7 @@ def partitions_auto_assigned(self): def all_consumed_offsets(self): """Returns consumed offsets as {TopicPartition: OffsetAndMetadata}""" all_consumed = {} - for partition, state in six.iteritems(self.assignment): + for partition, state in self.assignment.items(): if state.has_valid_position: all_consumed[partition] = state.position return all_consumed @@ -371,7 +359,7 @@ def is_offset_reset_needed(self, partition): @synchronized def has_all_fetch_positions(self): - for state in six.itervalues(self.assignment): + for state in self.assignment.values(): if not state.has_valid_position: return False return True @@ -379,7 +367,7 @@ def has_all_fetch_positions(self): @synchronized def missing_fetch_positions(self): missing = set() - for partition, state in six.iteritems(self.assignment): + for partition, state in self.assignment.items(): if state.is_missing_position(): missing.add(partition) return missing @@ -391,7 +379,7 @@ def has_valid_position(self, partition): @synchronized def reset_missing_positions(self): partitions_with_no_offsets = set() - for tp, state in six.iteritems(self.assignment): + for tp, state in self.assignment.items(): if state.is_missing_position(): if self._default_offset_reset_strategy == OffsetResetStrategy.NONE: partitions_with_no_offsets.add(tp) @@ -404,7 +392,7 @@ def reset_missing_positions(self): @synchronized def partitions_needing_reset(self): partitions = set() - for tp, state in six.iteritems(self.assignment): + for tp, state in self.assignment.items(): if state.awaiting_reset and state.is_reset_allowed(): partitions.add(tp) return partitions @@ -509,8 +497,7 @@ def is_fetchable(self): return not self.paused and self.has_valid_position -@six.add_metaclass(abc.ABCMeta) -class ConsumerRebalanceListener(object): +class ConsumerRebalanceListener(object, metaclass=abc.ABCMeta): """ A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the consumer changes. diff --git a/kafka/coordinator/assignors/abstract.py b/kafka/coordinator/assignors/abstract.py index a6fe970d2..3cdc2cace 100644 --- a/kafka/coordinator/assignors/abstract.py +++ b/kafka/coordinator/assignors/abstract.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import abc import logging diff --git a/kafka/coordinator/assignors/range.py b/kafka/coordinator/assignors/range.py index d639d5b75..307ae0b76 100644 --- a/kafka/coordinator/assignors/range.py +++ b/kafka/coordinator/assignors/range.py @@ -1,11 +1,7 @@ -from __future__ import absolute_import - import collections import itertools import logging -from kafka.vendor import six - from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0 @@ -35,7 +31,7 @@ class RangePartitionAssignor(AbstractPartitionAssignor): @classmethod def assign(cls, cluster, group_subscriptions): consumers_per_topic = collections.defaultdict(list) - for member_id, subscription in six.iteritems(group_subscriptions): + for member_id, subscription in group_subscriptions.items(): for topic in subscription.topics: consumers_per_topic[topic].append((subscription.group_instance_id, member_id)) @@ -47,7 +43,7 @@ def assign(cls, cluster, group_subscriptions): grouped = {k: list(g) for k, g in itertools.groupby(consumers_per_topic[topic], key=lambda ids: ids[0] is not None)} consumers_per_topic[topic] = sorted(grouped.get(True, [])) + sorted(grouped.get(False, [])) # sorted static members first, then sorted dynamic - for topic, consumers_for_topic in six.iteritems(consumers_per_topic): + for topic, consumers_for_topic in consumers_per_topic.items(): partitions = cluster.partitions_for_topic(topic) if partitions is None: log.warning('No partition metadata for topic %s', topic) diff --git a/kafka/coordinator/assignors/roundrobin.py b/kafka/coordinator/assignors/roundrobin.py index 8d83972cc..f73a10679 100644 --- a/kafka/coordinator/assignors/roundrobin.py +++ b/kafka/coordinator/assignors/roundrobin.py @@ -1,11 +1,7 @@ -from __future__ import absolute_import - import collections import itertools import logging -from kafka.vendor import six - from kafka.coordinator.assignors.abstract import AbstractPartitionAssignor from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0 from kafka.structs import TopicPartition @@ -51,7 +47,7 @@ class RoundRobinPartitionAssignor(AbstractPartitionAssignor): @classmethod def assign(cls, cluster, group_subscriptions): all_topics = set() - for subscription in six.itervalues(group_subscriptions): + for subscription in group_subscriptions.values(): all_topics.update(subscription.topics) all_topic_partitions = [] @@ -68,7 +64,7 @@ def assign(cls, cluster, group_subscriptions): assignment = collections.defaultdict(lambda: collections.defaultdict(list)) # Sort static and dynamic members separately to maintain stable static assignments - ungrouped = [(subscription.group_instance_id, member_id) for member_id, subscription in six.iteritems(group_subscriptions)] + ungrouped = [(subscription.group_instance_id, member_id) for member_id, subscription in group_subscriptions.items()] grouped = {k: list(g) for k, g in itertools.groupby(ungrouped, key=lambda ids: ids[0] is not None)} member_list = sorted(grouped.get(True, [])) + sorted(grouped.get(False, [])) # sorted static members first, then sorted dynamic member_iter = itertools.cycle(member_list) diff --git a/kafka/coordinator/assignors/sticky/partition_movements.py b/kafka/coordinator/assignors/sticky/partition_movements.py index 8851e4cda..78f2eb22c 100644 --- a/kafka/coordinator/assignors/sticky/partition_movements.py +++ b/kafka/coordinator/assignors/sticky/partition_movements.py @@ -2,8 +2,6 @@ from collections import defaultdict, namedtuple from copy import deepcopy -from kafka.vendor import six - log = logging.getLogger(__name__) @@ -74,7 +72,7 @@ def get_partition_to_be_moved(self, partition, old_consumer, new_consumer): return next(iter(self.partition_movements_by_topic[partition.topic][reverse_pair])) def are_sticky(self): - for topic, movements in six.iteritems(self.partition_movements_by_topic): + for topic, movements in self.partition_movements_by_topic.items(): movement_pairs = set(movements.keys()) if self._has_cycles(movement_pairs): log.error( diff --git a/kafka/coordinator/assignors/sticky/sticky_assignor.py b/kafka/coordinator/assignors/sticky/sticky_assignor.py index 3166356fe..243c26709 100644 --- a/kafka/coordinator/assignors/sticky/sticky_assignor.py +++ b/kafka/coordinator/assignors/sticky/sticky_assignor.py @@ -10,7 +10,6 @@ from kafka.protocol.struct import Struct from kafka.protocol.types import String, Array, Int32 from kafka.structs import TopicPartition -from kafka.vendor import six log = logging.getLogger(__name__) @@ -110,7 +109,7 @@ def balance(self): # narrow down the reassignment scope to only those partitions that can actually be reassigned fixed_partitions = set() - for partition in six.iterkeys(self.partition_to_all_potential_consumers): + for partition in self.partition_to_all_potential_consumers: if not self._can_partition_participate_in_reassignment(partition): fixed_partitions.add(partition) for fixed_partition in fixed_partitions: @@ -119,7 +118,7 @@ def balance(self): # narrow down the reassignment scope to only those consumers that are subject to reassignment fixed_assignments = {} - for consumer in six.iterkeys(self.consumer_to_all_potential_partitions): + for consumer in self.consumer_to_all_potential_partitions: if not self._can_consumer_participate_in_reassignment(consumer): self._remove_consumer_from_current_subscriptions_and_maintain_order(consumer) fixed_assignments[consumer] = self.current_assignment[consumer] @@ -148,7 +147,7 @@ def balance(self): self.current_partition_consumer.update(prebalance_partition_consumers) # add the fixed assignments (those that could not change) back - for consumer, partitions in six.iteritems(fixed_assignments): + for consumer, partitions in fixed_assignments.items(): self.current_assignment[consumer] = partitions self._add_consumer_to_current_subscriptions_and_maintain_order(consumer) @@ -156,8 +155,8 @@ def get_final_assignment(self, member_id): assignment = defaultdict(list) for topic_partition in self.current_assignment[member_id]: assignment[topic_partition.topic].append(topic_partition.partition) - assignment = {k: sorted(v) for k, v in six.iteritems(assignment)} - return six.viewitems(assignment) + assignment = {k: sorted(v) for k, v in assignment.items()} + return assignment.items() def _initialize(self, cluster): self._init_current_assignments(self.members) @@ -170,7 +169,7 @@ def _initialize(self, cluster): for p in partitions: partition = TopicPartition(topic=topic, partition=p) self.partition_to_all_potential_consumers[partition] = [] - for consumer_id, member_metadata in six.iteritems(self.members): + for consumer_id, member_metadata in self.members.items(): self.consumer_to_all_potential_partitions[consumer_id] = [] for topic in member_metadata.subscription: if cluster.partitions_for_topic(topic) is None: @@ -190,7 +189,7 @@ def _init_current_assignments(self, members): # for each partition we create a map of its consumers by generation sorted_partition_consumers_by_generation = {} - for consumer, member_metadata in six.iteritems(members): + for consumer, member_metadata in members.items(): for partitions in member_metadata.partitions: if partitions in sorted_partition_consumers_by_generation: consumers = sorted_partition_consumers_by_generation[partitions] @@ -209,7 +208,7 @@ def _init_current_assignments(self, members): # previous_assignment holds the prior ConsumerGenerationPair (before current) of each partition # current and previous consumers are the last two consumers of each partition in the above sorted map - for partitions, consumers in six.iteritems(sorted_partition_consumers_by_generation): + for partitions, consumers in sorted_partition_consumers_by_generation.items(): generations = sorted(consumers.keys(), reverse=True) self.current_assignment[consumers[generations[0]]].append(partitions) # now update previous assignment if any @@ -220,7 +219,7 @@ def _init_current_assignments(self, members): self.is_fresh_assignment = len(self.current_assignment) == 0 - for consumer_id, partitions in six.iteritems(self.current_assignment): + for consumer_id, partitions in self.current_assignment.items(): for partition in partitions: self.current_partition_consumer[partition] = consumer_id @@ -230,14 +229,14 @@ def _are_subscriptions_identical(self): true, if both potential consumers of partitions and potential partitions that consumers can consume are the same """ - if not has_identical_list_elements(list(six.itervalues(self.partition_to_all_potential_consumers))): + if not has_identical_list_elements(list(self.partition_to_all_potential_consumers.values())): return False - return has_identical_list_elements(list(six.itervalues(self.consumer_to_all_potential_partitions))) + return has_identical_list_elements(list(self.consumer_to_all_potential_partitions.values())) def _populate_sorted_partitions(self): # set of topic partitions with their respective potential consumers all_partitions = set((tp, tuple(consumers)) - for tp, consumers in six.iteritems(self.partition_to_all_potential_consumers)) + for tp, consumers in self.partition_to_all_potential_consumers.items()) partitions_sorted_by_num_of_potential_consumers = sorted(all_partitions, key=partitions_comparator_key) self.sorted_partitions = [] @@ -246,7 +245,7 @@ def _populate_sorted_partitions(self): # then we just need to simply list partitions in a round robin fashion (from consumers with # most assigned partitions to those with least) assignments = deepcopy(self.current_assignment) - for consumer_id, partitions in six.iteritems(assignments): + for consumer_id, partitions in assignments.items(): to_remove = [] for partition in partitions: if partition not in self.partition_to_all_potential_consumers: @@ -255,7 +254,7 @@ def _populate_sorted_partitions(self): partitions.remove(partition) sorted_consumers = SortedSet( - iterable=[(consumer, tuple(partitions)) for consumer, partitions in six.iteritems(assignments)], + iterable=[(consumer, tuple(partitions)) for consumer, partitions in assignments.items()], key=subscriptions_comparator_key, ) # at this point, sorted_consumers contains an ascending-sorted list of consumers based on @@ -267,7 +266,7 @@ def _populate_sorted_partitions(self): remaining_partitions = assignments[consumer] # from partitions that had a different consumer before, # keep only those that are assigned to this consumer now - previous_partitions = set(six.iterkeys(self.previous_assignment)).intersection(set(remaining_partitions)) + previous_partitions = set(self.previous_assignment.keys()).intersection(set(remaining_partitions)) if previous_partitions: # if there is a partition of this consumer that was assigned to another consumer before # mark it as good options for reassignment @@ -292,7 +291,7 @@ def _populate_partitions_to_reassign(self): self.unassigned_partitions = deepcopy(self.sorted_partitions) assignments_to_remove = [] - for consumer_id, partitions in six.iteritems(self.current_assignment): + for consumer_id, partitions in self.current_assignment.items(): if consumer_id not in self.members: # if a consumer that existed before (and had some partition assignments) is now removed, # remove it from current_assignment @@ -325,7 +324,7 @@ def _populate_partitions_to_reassign(self): def _initialize_current_subscriptions(self): self.sorted_current_subscriptions = SortedSet( - iterable=[(consumer, tuple(partitions)) for consumer, partitions in six.iteritems(self.current_assignment)], + iterable=[(consumer, tuple(partitions)) for consumer, partitions in self.current_assignment.items()], key=subscriptions_comparator_key, ) @@ -352,7 +351,7 @@ def _is_balanced(self): # create a mapping from partitions to the consumer assigned to them all_assigned_partitions = {} - for consumer_id, consumer_partitions in six.iteritems(self.current_assignment): + for consumer_id, consumer_partitions in self.current_assignment.items(): for partition in consumer_partitions: if partition in all_assigned_partitions: log.error("{} is assigned to more than one consumer.".format(partition)) @@ -491,11 +490,11 @@ def _get_balance_score(assignment): """ score = 0 consumer_to_assignment = {} - for consumer_id, partitions in six.iteritems(assignment): + for consumer_id, partitions in assignment.items(): consumer_to_assignment[consumer_id] = len(partitions) consumers_to_explore = set(consumer_to_assignment.keys()) - for consumer_id in consumer_to_assignment.keys(): + for consumer_id in consumer_to_assignment: if consumer_id in consumers_to_explore: consumers_to_explore.remove(consumer_id) for other_consumer_id in consumers_to_explore: @@ -593,7 +592,7 @@ def assign(cls, cluster, members): dict: {member_id: MemberAssignment} """ members_metadata = {} - for consumer, member_metadata in six.iteritems(members): + for consumer, member_metadata in members.items(): members_metadata[consumer] = cls.parse_member_metadata(member_metadata) executor = StickyAssignmentExecutor(cluster, members_metadata) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index e1d8d8336..12e7ac40b 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import abc import copy import logging @@ -8,8 +6,6 @@ import warnings import weakref -from kafka.vendor import six - from kafka.coordinator.heartbeat import Heartbeat from kafka import errors as Errors from kafka.future import Future @@ -710,7 +706,7 @@ def _on_join_leader(self, response): group_assignment = self._perform_assignment(response.leader_id, response.group_protocol, members) - for member_id, assignment in six.iteritems(group_assignment): + for member_id, assignment in group_assignment.items(): if not isinstance(assignment, bytes): group_assignment[member_id] = assignment.encode() diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index a7aac4352..0bce86aaf 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -1,13 +1,9 @@ -from __future__ import absolute_import, division - import collections import copy import functools import logging import time -from kafka.vendor import six - from kafka.coordinator.base import BaseCoordinator, Generation from kafka.coordinator.assignors.range import RangePartitionAssignor from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor @@ -367,7 +363,7 @@ def _perform_assignment(self, leader_id, assignment_strategy, members): log.debug("Finished assignment for group %s: %s", self.group_id, assignments) group_assignment = {} - for member_id, assignment in six.iteritems(assignments): + for member_id, assignment in assignments.items(): group_assignment[member_id] = assignment return group_assignment @@ -421,7 +417,7 @@ def refresh_committed_offsets_if_needed(self, timeout_ms=None): offsets = self.fetch_committed_offsets(missing_fetch_positions, timeout_ms=timeout_ms) except Errors.KafkaTimeoutError: return False - for partition, offset in six.iteritems(offsets): + for partition, offset in offsets.items(): log.debug("Setting offset for partition %s to the committed offset %s", partition, offset.offset) self._subscription.seek(partition, offset.offset) return True @@ -640,7 +636,7 @@ def _send_offset_commit_request(self, offsets): # create the offset commit request offset_data = collections.defaultdict(dict) - for tp, offset in six.iteritems(offsets): + for tp, offset in offsets.items(): offset_data[tp.topic][tp.partition] = offset version = self._client.api_version(OffsetCommitRequest, max_version=7) @@ -675,8 +671,8 @@ def _send_offset_commit_request(self, offsets): partition, offset.offset, offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] + ) for partition, offset in partitions.items()] + ) for topic, partitions in offset_data.items()] ) elif version == 1: request = OffsetCommitRequest[version]( @@ -691,8 +687,8 @@ def _send_offset_commit_request(self, offsets): offset.offset, -1, # timestamp, unused offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] + ) for partition, offset in partitions.items()] + ) for topic, partitions in offset_data.items()] ) elif version <= 4: request = OffsetCommitRequest[version]( @@ -705,8 +701,8 @@ def _send_offset_commit_request(self, offsets): partition, offset.offset, offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] + ) for partition, offset in partitions.items()] + ) for topic, partitions in offset_data.items()] ) elif version <= 5: request = OffsetCommitRequest[version]( @@ -718,8 +714,8 @@ def _send_offset_commit_request(self, offsets): partition, offset.offset, offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] + ) for partition, offset in partitions.items()] + ) for topic, partitions in offset_data.items()] ) elif version <= 6: request = OffsetCommitRequest[version]( @@ -732,8 +728,8 @@ def _send_offset_commit_request(self, offsets): offset.offset, offset.leader_epoch, offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] + ) for partition, offset in partitions.items()] + ) for topic, partitions in offset_data.items()] ) else: request = OffsetCommitRequest[version]( @@ -747,8 +743,8 @@ def _send_offset_commit_request(self, offsets): offset.offset, offset.leader_epoch, offset.metadata - ) for partition, offset in six.iteritems(partitions)] - ) for topic, partitions in six.iteritems(offset_data)] + ) for partition, offset in partitions.items()] + ) for topic, partitions in offset_data.items()] ) log.debug("Sending offset-commit request with %s for group %s to %s", diff --git a/kafka/coordinator/heartbeat.py b/kafka/coordinator/heartbeat.py index edc9f4a36..fb40c302b 100644 --- a/kafka/coordinator/heartbeat.py +++ b/kafka/coordinator/heartbeat.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import copy import logging import time diff --git a/kafka/coordinator/protocol.py b/kafka/coordinator/protocol.py index bfa1c4695..51957a468 100644 --- a/kafka/coordinator/protocol.py +++ b/kafka/coordinator/protocol.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.struct import Struct from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String from kafka.structs import TopicPartition diff --git a/kafka/coordinator/subscription.py b/kafka/coordinator/subscription.py index ca49c1bc0..b5c47994d 100644 --- a/kafka/coordinator/subscription.py +++ b/kafka/coordinator/subscription.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class Subscription(object): __slots__ = ('_metadata', '_group_instance_id') def __init__(self, metadata, group_instance_id): diff --git a/kafka/errors.py b/kafka/errors.py index dffa35f35..96fdcb223 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import inspect import sys diff --git a/kafka/future.py b/kafka/future.py index 2af061ee7..5d53c2192 100644 --- a/kafka/future.py +++ b/kafka/future.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import functools import logging import threading diff --git a/kafka/metrics/__init__.py b/kafka/metrics/__init__.py index 2a62d6334..22427e967 100644 --- a/kafka/metrics/__init__.py +++ b/kafka/metrics/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.compound_stat import NamedMeasurable from kafka.metrics.dict_reporter import DictReporter from kafka.metrics.kafka_metric import KafkaMetric diff --git a/kafka/metrics/compound_stat.py b/kafka/metrics/compound_stat.py index f5b482da2..656e1122c 100644 --- a/kafka/metrics/compound_stat.py +++ b/kafka/metrics/compound_stat.py @@ -1,13 +1,9 @@ -from __future__ import absolute_import - import abc from kafka.metrics.stat import AbstractStat -from kafka.vendor.six import add_metaclass -@add_metaclass(abc.ABCMeta) -class AbstractCompoundStat(AbstractStat): +class AbstractCompoundStat(AbstractStat, metaclass=abc.ABCMeta): """ A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the example for a diff --git a/kafka/metrics/dict_reporter.py b/kafka/metrics/dict_reporter.py index 0b98fe1e4..49af60476 100644 --- a/kafka/metrics/dict_reporter.py +++ b/kafka/metrics/dict_reporter.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import logging import threading diff --git a/kafka/metrics/kafka_metric.py b/kafka/metrics/kafka_metric.py index fef684850..97799245b 100644 --- a/kafka/metrics/kafka_metric.py +++ b/kafka/metrics/kafka_metric.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import time diff --git a/kafka/metrics/measurable.py b/kafka/metrics/measurable.py index b06d4d789..ef096f31d 100644 --- a/kafka/metrics/measurable.py +++ b/kafka/metrics/measurable.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import abc diff --git a/kafka/metrics/measurable_stat.py b/kafka/metrics/measurable_stat.py index 08222b144..f16798c5a 100644 --- a/kafka/metrics/measurable_stat.py +++ b/kafka/metrics/measurable_stat.py @@ -1,14 +1,10 @@ -from __future__ import absolute_import - import abc from kafka.metrics.measurable import AbstractMeasurable from kafka.metrics.stat import AbstractStat -from kafka.vendor.six import add_metaclass -@add_metaclass(abc.ABCMeta) -class AbstractMeasurableStat(AbstractStat, AbstractMeasurable): +class AbstractMeasurableStat(AbstractStat, AbstractMeasurable, metaclass=abc.ABCMeta): """ An AbstractMeasurableStat is an AbstractStat that is also an AbstractMeasurable (i.e. can produce a single floating point value). diff --git a/kafka/metrics/metric_config.py b/kafka/metrics/metric_config.py index 7e5ead1fe..008dfa6d8 100644 --- a/kafka/metrics/metric_config.py +++ b/kafka/metrics/metric_config.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys diff --git a/kafka/metrics/metric_name.py b/kafka/metrics/metric_name.py index b8ab2a3ad..db351829c 100644 --- a/kafka/metrics/metric_name.py +++ b/kafka/metrics/metric_name.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import copy diff --git a/kafka/metrics/metrics.py b/kafka/metrics/metrics.py index 41a37db58..f00833837 100644 --- a/kafka/metrics/metrics.py +++ b/kafka/metrics/metrics.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import logging import sys import time diff --git a/kafka/metrics/metrics_reporter.py b/kafka/metrics/metrics_reporter.py index 8df2e9ea6..fd7de8d5e 100644 --- a/kafka/metrics/metrics_reporter.py +++ b/kafka/metrics/metrics_reporter.py @@ -1,12 +1,7 @@ -from __future__ import absolute_import - import abc -from kafka.vendor.six import add_metaclass - -@add_metaclass(abc.ABCMeta) -class AbstractMetricsReporter(object): +class AbstractMetricsReporter(object, metaclass=abc.ABCMeta): """ An abstract class to allow things to listen as new metrics are created so they can be reported. diff --git a/kafka/metrics/quota.py b/kafka/metrics/quota.py index 36a30c44e..2cf6e9089 100644 --- a/kafka/metrics/quota.py +++ b/kafka/metrics/quota.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class Quota(object): """An upper or lower bound for metrics""" __slots__ = ('_bound', '_upper') diff --git a/kafka/metrics/stat.py b/kafka/metrics/stat.py index 8825d2783..2588493a7 100644 --- a/kafka/metrics/stat.py +++ b/kafka/metrics/stat.py @@ -1,12 +1,7 @@ -from __future__ import absolute_import - import abc -from kafka.vendor.six import add_metaclass - -@add_metaclass(abc.ABCMeta) -class AbstractStat(object): +class AbstractStat(object, metaclass=abc.ABCMeta): """ An AbstractStat is a quantity such as average, max, etc that is computed off the stream of updates to a sensor diff --git a/kafka/metrics/stats/__init__.py b/kafka/metrics/stats/__init__.py index a3d535dfd..04afb61ae 100644 --- a/kafka/metrics/stats/__init__.py +++ b/kafka/metrics/stats/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.stats.avg import Avg from kafka.metrics.stats.count import Count from kafka.metrics.stats.histogram import Histogram diff --git a/kafka/metrics/stats/avg.py b/kafka/metrics/stats/avg.py index 906d95573..c7e7dc485 100644 --- a/kafka/metrics/stats/avg.py +++ b/kafka/metrics/stats/avg.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.stats.sampled_stat import AbstractSampledStat diff --git a/kafka/metrics/stats/count.py b/kafka/metrics/stats/count.py index 6cd6d2abe..060bd2d99 100644 --- a/kafka/metrics/stats/count.py +++ b/kafka/metrics/stats/count.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.stats.sampled_stat import AbstractSampledStat diff --git a/kafka/metrics/stats/histogram.py b/kafka/metrics/stats/histogram.py index 2c8afbfb3..019373184 100644 --- a/kafka/metrics/stats/histogram.py +++ b/kafka/metrics/stats/histogram.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import math diff --git a/kafka/metrics/stats/max_stat.py b/kafka/metrics/stats/max_stat.py index 9c5eeb6fd..5e0382b3b 100644 --- a/kafka/metrics/stats/max_stat.py +++ b/kafka/metrics/stats/max_stat.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.stats.sampled_stat import AbstractSampledStat diff --git a/kafka/metrics/stats/min_stat.py b/kafka/metrics/stats/min_stat.py index 6bebe57e0..a728fa2eb 100644 --- a/kafka/metrics/stats/min_stat.py +++ b/kafka/metrics/stats/min_stat.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from kafka.metrics.stats.sampled_stat import AbstractSampledStat diff --git a/kafka/metrics/stats/percentile.py b/kafka/metrics/stats/percentile.py index 75e64ce5e..17cbb1fc1 100644 --- a/kafka/metrics/stats/percentile.py +++ b/kafka/metrics/stats/percentile.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - class Percentile(object): __slots__ = ('_metric_name', '_percentile') diff --git a/kafka/metrics/stats/percentiles.py b/kafka/metrics/stats/percentiles.py index 2cb2d84de..574557a59 100644 --- a/kafka/metrics/stats/percentiles.py +++ b/kafka/metrics/stats/percentiles.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics import AnonMeasurable, NamedMeasurable from kafka.metrics.compound_stat import AbstractCompoundStat from kafka.metrics.stats import Histogram diff --git a/kafka/metrics/stats/rate.py b/kafka/metrics/stats/rate.py index 4d0ba0f27..6005bdb47 100644 --- a/kafka/metrics/stats/rate.py +++ b/kafka/metrics/stats/rate.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.measurable_stat import AbstractMeasurableStat from kafka.metrics.stats.sampled_stat import AbstractSampledStat diff --git a/kafka/metrics/stats/sampled_stat.py b/kafka/metrics/stats/sampled_stat.py index fe8970dbf..b3bbfb046 100644 --- a/kafka/metrics/stats/sampled_stat.py +++ b/kafka/metrics/stats/sampled_stat.py @@ -1,13 +1,9 @@ -from __future__ import absolute_import - import abc from kafka.metrics.measurable_stat import AbstractMeasurableStat -from kafka.vendor.six import add_metaclass -@add_metaclass(abc.ABCMeta) -class AbstractSampledStat(AbstractMeasurableStat): +class AbstractSampledStat(AbstractMeasurableStat, metaclass=abc.ABCMeta): """ An AbstractSampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a configurable diff --git a/kafka/metrics/stats/sensor.py b/kafka/metrics/stats/sensor.py index 9f7ac45f5..b539618ab 100644 --- a/kafka/metrics/stats/sensor.py +++ b/kafka/metrics/stats/sensor.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import threading import time diff --git a/kafka/metrics/stats/total.py b/kafka/metrics/stats/total.py index a78e99733..d43ceee18 100644 --- a/kafka/metrics/stats/total.py +++ b/kafka/metrics/stats/total.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.metrics.measurable_stat import AbstractMeasurableStat diff --git a/kafka/partitioner/__init__.py b/kafka/partitioner/__init__.py index 21a3bbb66..eed1dca69 100644 --- a/kafka/partitioner/__init__.py +++ b/kafka/partitioner/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.partitioner.default import DefaultPartitioner, murmur2 diff --git a/kafka/partitioner/default.py b/kafka/partitioner/default.py index d0914c682..8004020c0 100644 --- a/kafka/partitioner/default.py +++ b/kafka/partitioner/default.py @@ -1,9 +1,5 @@ -from __future__ import absolute_import - import random -from kafka.vendor import six - class DefaultPartitioner(object): """Default partitioner. @@ -43,11 +39,6 @@ def murmur2(data): Returns: MurmurHash2 of data """ - # Python2 bytes is really a str, causing the bitwise operations below to fail - # so convert to bytearray. - if six.PY2: - data = bytearray(bytes(data)) - length = len(data) seed = 0x9747b28c # 'm' and 'r' are mixing constants generated offline. diff --git a/kafka/producer/__init__.py b/kafka/producer/__init__.py index 576c772a0..869dbb3dc 100644 --- a/kafka/producer/__init__.py +++ b/kafka/producer/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.producer.kafka import KafkaProducer __all__ = [ diff --git a/kafka/producer/__main__.py b/kafka/producer/__main__.py index e5fd1b1d1..7b3215fc8 100644 --- a/kafka/producer/__main__.py +++ b/kafka/producer/__main__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import sys from kafka.cli.producer import run_cli diff --git a/kafka/producer/future.py b/kafka/producer/future.py index 13392a96e..50fabbb3c 100644 --- a/kafka/producer/future.py +++ b/kafka/producer/future.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import collections import threading diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 005274c82..a937138a6 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import atexit import copy import logging @@ -8,8 +6,6 @@ import warnings import weakref -from kafka.vendor import six - import kafka.errors as Errors from kafka.client_async import KafkaClient, selectors from kafka.codec import has_gzip, has_snappy, has_lz4, has_zstd @@ -296,21 +292,20 @@ class KafkaProducer(object): will be ignored. Default: None. ssl_check_hostname (bool): flag to configure whether ssl handshake should verify that the certificate matches the brokers hostname. - default: true. + Default: True. ssl_cafile (str): optional filename of ca file to use in certificate - verification. default: none. + verification. Default: None. ssl_certfile (str): optional filename of file in pem format containing the client certificate, as well as any ca certificates needed to - establish the certificate's authenticity. default: none. + establish the certificate's authenticity. Default: None. ssl_keyfile (str): optional filename containing the client private key. - default: none. + Default: None. ssl_password (str): optional password to be used when loading the - certificate chain. default: none. + certificate chain. Default: None. ssl_crlfile (str): optional filename containing the CRL to check for certificate expiration. By default, no CRL check is done. When providing a file, only the leaf certificate will be checked against - this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+. - default: none. + this CRL. Default: None. ssl_ciphers (str): optionally set the available ciphers for ssl connections. It should be a string in the OpenSSL cipher list format. If no cipher can be selected (because compile-time options @@ -583,19 +578,7 @@ def wrapper(): def _unregister_cleanup(self): if getattr(self, '_cleanup', None): - if hasattr(atexit, 'unregister'): - atexit.unregister(self._cleanup) # pylint: disable=no-member - - # py2 requires removing from private attribute... - else: - - # ValueError on list.remove() if the exithandler no longer exists - # but that is fine here - try: - atexit._exithandlers.remove( # pylint: disable=no-member - (self._cleanup, (), {})) - except ValueError: - pass + atexit.unregister(self._cleanup) self._cleanup = None def __del__(self): @@ -1012,7 +995,7 @@ def metrics(self, raw=False): return self._metrics.metrics.copy() metrics = {} - for k, v in six.iteritems(self._metrics.metrics.copy()): + for k, v in self._metrics.metrics.copy().items(): if k.group not in metrics: metrics[k.group] = {} if k.name not in metrics[k.group]: diff --git a/kafka/producer/producer_batch.py b/kafka/producer/producer_batch.py index 198a3a0c5..71dea4fbd 100644 --- a/kafka/producer/producer_batch.py +++ b/kafka/producer/producer_batch.py @@ -1,15 +1,7 @@ -from __future__ import absolute_import, division - +from enum import IntEnum import logging import time -try: - # enum in stdlib as of py3.4 - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum - import kafka.errors as Errors from kafka.producer.future import FutureRecordMetadata, FutureProduceResult diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 1add95a3b..e779b1668 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import collections import copy import logging diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index b7c38a4f5..7df76ea24 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import collections import copy import heapq @@ -7,8 +5,6 @@ import threading import time -from kafka.vendor import six - from kafka import errors as Errors from kafka.metrics.measurable import AnonMeasurable from kafka.metrics.stats import Avg, Max, Rate @@ -86,7 +82,7 @@ def _get_expired_inflight_batches(self, now=None): """Get the in-flight batches that has reached delivery timeout.""" expired_batches = [] to_remove = [] - for tp, queue in six.iteritems(self._in_flight_batches): + for tp, queue in self._in_flight_batches.items(): while queue: _created_at, batch = queue[0] if batch.has_reached_delivery_timeout(self._accumulator.delivery_timeout_ms): @@ -206,7 +202,7 @@ def _send_producer_data(self, now=None): batches_by_node = self._accumulator.drain( self._metadata, ready_nodes, self.config['max_request_size'], now=now) - for batch_list in six.itervalues(batches_by_node): + for batch_list in batches_by_node.values(): for batch in batch_list: item = (batch.created, batch) queue = self._in_flight_batches[batch.topic_partition] @@ -214,7 +210,7 @@ def _send_producer_data(self, now=None): if self.config['guarantee_message_order']: # Mute all the partitions drained - for batch_list in six.itervalues(batches_by_node): + for batch_list in batches_by_node.values(): for batch in batch_list: self._accumulator.muted.add(batch.topic_partition) @@ -272,7 +268,7 @@ def _send_producer_data(self, now=None): # metadata expiry time poll_timeout_ms = 0 - for node_id, request in six.iteritems(requests): + for node_id, request in requests.items(): batches = batches_by_node[node_id] log.debug('%s: Sending Produce Request: %r', str(self), request) (self._client.send(node_id, request, wakeup=False) @@ -584,7 +580,7 @@ def _create_produce_requests(self, collated): dict: {node_id: ProduceRequest} (version depends on client api_versions) """ requests = {} - for node_id, batches in six.iteritems(collated): + for node_id, batches in collated.items(): if batches: requests[node_id] = self._produce_request( node_id, self.config['acks'], @@ -608,7 +604,7 @@ def _produce_request(self, node_id, acks, timeout, batches): version = self._client.api_version(ProduceRequest, max_version=8) topic_partition_data = [ (topic, list(partition_info.items())) - for topic, partition_info in six.iteritems(produce_records_by_partition)] + for topic, partition_info in produce_records_by_partition.items()] transactional_id = self._transaction_manager.transactional_id if self._transaction_manager else None if version >= 3: return ProduceRequest[version]( diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index a44d7d9b3..6c7a5f5b4 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -1,20 +1,10 @@ -from __future__ import absolute_import, division - import abc import collections +from enum import IntEnum import heapq import logging import threading -from kafka.vendor import six - -try: - # enum in stdlib as of py3.4 - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum - import kafka.errors as Errors from kafka.protocol.add_offsets_to_txn import AddOffsetsToTxnRequest from kafka.protocol.add_partitions_to_txn import AddPartitionsToTxnRequest @@ -535,8 +525,7 @@ def exception(self): return self._error -@six.add_metaclass(abc.ABCMeta) -class TxnRequestHandler(object): +class TxnRequestHandler(object, metaclass=abc.ABCMeta): def __init__(self, transaction_manager, result=None): self.transaction_manager = transaction_manager self.retry_backoff_ms = transaction_manager.retry_backoff_ms @@ -687,7 +676,7 @@ def handle_response(self, response): for topic, partition_data in response.results for partition, error_code in partition_data} - for tp, error in six.iteritems(results): + for tp, error in results.items(): if error is Errors.NoError: continue elif error in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): @@ -876,7 +865,7 @@ def handle_response(self, response): log.debug("Successfully added partition for consumer group %s to transaction", self.consumer_group_id) # note the result is not completed until the TxnOffsetCommit returns - for tp, offset in six.iteritems(self.offsets): + for tp, offset in self.offsets.items(): self.transaction_manager._pending_txn_offset_commits[tp] = offset handler = TxnOffsetCommitHandler(self.transaction_manager, self.consumer_group_id, self.transaction_manager._pending_txn_offset_commits, self._result) @@ -914,7 +903,7 @@ def _build_request(self): version = 0 topic_data = collections.defaultdict(list) - for tp, offset in six.iteritems(self.offsets): + for tp, offset in self.offsets.items(): if version >= 2: partition_data = (tp.partition, offset.offset, offset.leader_epoch, offset.metadata) else: @@ -948,7 +937,7 @@ def handle_response(self, response): for topic, partition_data in response.topics for partition, error_code in partition_data} - for tp, error in six.iteritems(errors): + for tp, error in errors.items(): if error is Errors.NoError: log.debug("Successfully added offsets for %s from consumer group %s to transaction.", tp, self.consumer_group_id) diff --git a/kafka/protocol/__init__.py b/kafka/protocol/__init__.py index 025447f99..ff9c68306 100644 --- a/kafka/protocol/__init__.py +++ b/kafka/protocol/__init__.py @@ -1,6 +1,3 @@ -from __future__ import absolute_import - - API_KEYS = { 0: 'Produce', 1: 'Fetch', diff --git a/kafka/protocol/abstract.py b/kafka/protocol/abstract.py index 7ce5fc18f..e0d89433b 100644 --- a/kafka/protocol/abstract.py +++ b/kafka/protocol/abstract.py @@ -1,12 +1,7 @@ -from __future__ import absolute_import - import abc -from kafka.vendor.six import add_metaclass - -@add_metaclass(abc.ABCMeta) -class AbstractType(object): +class AbstractType(object, metaclass=abc.ABCMeta): @abc.abstractmethod def encode(cls, value): # pylint: disable=no-self-argument pass diff --git a/kafka/protocol/add_offsets_to_txn.py b/kafka/protocol/add_offsets_to_txn.py index fa2509330..bc6805b83 100644 --- a/kafka/protocol/add_offsets_to_txn.py +++ b/kafka/protocol/add_offsets_to_txn.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/add_partitions_to_txn.py b/kafka/protocol/add_partitions_to_txn.py index fdf28f4ae..b6a95ceaf 100644 --- a/kafka/protocol/add_partitions_to_txn.py +++ b/kafka/protocol/add_partitions_to_txn.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index 32b75df4b..56ef656e6 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -1,11 +1,4 @@ -from __future__ import absolute_import - -# enum in stdlib as of py3.4 -try: - from enum import IntEnum # pylint: disable=import-error -except ImportError: - # vendored backport module - from kafka.vendor.enum34 import IntEnum +from enum import IntEnum from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String, Float64, CompactString, CompactArray, TaggedFields, BitField diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index 9cd5767c1..c7a477cac 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -1,12 +1,8 @@ -from __future__ import absolute_import - import abc from kafka.protocol.struct import Struct from kafka.protocol.types import Int16, Int32, String, Schema, Array, TaggedFields -from kafka.vendor.six import add_metaclass - class RequestHeader(Struct): SCHEMA = Schema( @@ -51,8 +47,7 @@ class ResponseHeaderV2(Struct): ) -@add_metaclass(abc.ABCMeta) -class Request(Struct): +class Request(Struct, metaclass=abc.ABCMeta): FLEXIBLE_VERSION = False @abc.abstractproperty @@ -88,8 +83,7 @@ def build_header(self, correlation_id, client_id): return RequestHeader(self, correlation_id=correlation_id, client_id=client_id) -@add_metaclass(abc.ABCMeta) -class Response(Struct): +class Response(Struct, metaclass=abc.ABCMeta): FLEXIBLE_VERSION = False @abc.abstractproperty diff --git a/kafka/protocol/api_versions.py b/kafka/protocol/api_versions.py index e7cedd954..7638bf46e 100644 --- a/kafka/protocol/api_versions.py +++ b/kafka/protocol/api_versions.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from io import BytesIO from kafka.protocol.api import Request, Response diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py index 4cbc43afd..74141184b 100644 --- a/kafka/protocol/commit.py +++ b/kafka/protocol/commit.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/end_txn.py b/kafka/protocol/end_txn.py index 96d6cc514..5a4089fa6 100644 --- a/kafka/protocol/end_txn.py +++ b/kafka/protocol/end_txn.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Boolean, Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/fetch.py b/kafka/protocol/fetch.py index 036a37eb8..cc8652774 100644 --- a/kafka/protocol/fetch.py +++ b/kafka/protocol/fetch.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import collections from kafka.protocol.api import Request, Response diff --git a/kafka/protocol/find_coordinator.py b/kafka/protocol/find_coordinator.py index be5b45ded..caad15bfd 100644 --- a/kafka/protocol/find_coordinator.py +++ b/kafka/protocol/find_coordinator.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Int8, Int16, Int32, Schema, String diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index 383f3cd2a..5d35ab219 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import collections from kafka.protocol.api import Request, Response diff --git a/kafka/protocol/init_producer_id.py b/kafka/protocol/init_producer_id.py index 8426fe00b..102ea4770 100644 --- a/kafka/protocol/init_producer_id.py +++ b/kafka/protocol/init_producer_id.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/list_offsets.py b/kafka/protocol/list_offsets.py index 2e36dd660..99f85f12e 100644 --- a/kafka/protocol/list_offsets.py +++ b/kafka/protocol/list_offsets.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py index 4c5c031b8..03d1098c0 100644 --- a/kafka/protocol/message.py +++ b/kafka/protocol/message.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import io import time diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py index eb632371c..715b48595 100644 --- a/kafka/protocol/metadata.py +++ b/kafka/protocol/metadata.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Boolean, Int16, Int32, Schema, String, BitField diff --git a/kafka/protocol/offset_for_leader_epoch.py b/kafka/protocol/offset_for_leader_epoch.py index 8465588a3..64c9d1751 100644 --- a/kafka/protocol/offset_for_leader_epoch.py +++ b/kafka/protocol/offset_for_leader_epoch.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, CompactArray, CompactString, Int16, Int32, Int64, Schema, String, TaggedFields diff --git a/kafka/protocol/parser.py b/kafka/protocol/parser.py index 8683a952a..84d9e8718 100644 --- a/kafka/protocol/parser.py +++ b/kafka/protocol/parser.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import collections import logging diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py index 94edd0f80..4f07ca002 100644 --- a/kafka/protocol/produce.py +++ b/kafka/protocol/produce.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Int16, Int32, Int64, String, Array, Schema, Bytes diff --git a/kafka/protocol/sasl_authenticate.py b/kafka/protocol/sasl_authenticate.py index a2b9b1988..0be8b54a4 100644 --- a/kafka/protocol/sasl_authenticate.py +++ b/kafka/protocol/sasl_authenticate.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Bytes, Int16, Int64, Schema, String diff --git a/kafka/protocol/sasl_handshake.py b/kafka/protocol/sasl_handshake.py index e91c856ca..fa792cf67 100644 --- a/kafka/protocol/sasl_handshake.py +++ b/kafka/protocol/sasl_handshake.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int16, Schema, String diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index d4adb8832..b482326fa 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from io import BytesIO from kafka.protocol.abstract import AbstractType diff --git a/kafka/protocol/txn_offset_commit.py b/kafka/protocol/txn_offset_commit.py index df1b1bd1e..cea96e9f8 100644 --- a/kafka/protocol/txn_offset_commit.py +++ b/kafka/protocol/txn_offset_commit.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int16, Int32, Int64, Schema, String diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 8949ce471..812c5e74d 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import struct from struct import error diff --git a/kafka/record/abc.py b/kafka/record/abc.py index c78f0da69..908ad30b7 100644 --- a/kafka/record/abc.py +++ b/kafka/record/abc.py @@ -1,12 +1,7 @@ -from __future__ import absolute_import - import abc -from kafka.vendor.six import add_metaclass - -@add_metaclass(abc.ABCMeta) -class ABCRecord(object): +class ABCRecord(object, metaclass=abc.ABCMeta): __slots__ = () @abc.abstractproperty @@ -57,8 +52,7 @@ def headers(self): """ -@add_metaclass(abc.ABCMeta) -class ABCRecordBatchBuilder(object): +class ABCRecordBatchBuilder(object, metaclass=abc.ABCMeta): __slots__ = () @abc.abstractmethod @@ -97,8 +91,7 @@ def build(self): """ -@add_metaclass(abc.ABCMeta) -class ABCRecordBatch(object): +class ABCRecordBatch(object, metaclass=abc.ABCMeta): """ For v2 encapsulates a RecordBatch, for v0/v1 a single (maybe compressed) message. """ @@ -126,8 +119,7 @@ def magic(self): """ -@add_metaclass(abc.ABCMeta) -class ABCRecords(object): +class ABCRecords(object, metaclass=abc.ABCMeta): __slots__ = () @abc.abstractmethod diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index 9df733059..d203dd8c7 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -18,7 +18,6 @@ # # So we can iterate over batches just by knowing offsets of Length. Magic is # used to construct the correct class for Batch itself. -from __future__ import division import struct diff --git a/kafka/sasl/__init__.py b/kafka/sasl/__init__.py index 90f05e733..047f3529b 100644 --- a/kafka/sasl/__init__.py +++ b/kafka/sasl/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import platform from kafka.sasl.gssapi import SaslMechanismGSSAPI diff --git a/kafka/sasl/abc.py b/kafka/sasl/abc.py index 0577888a9..30a58d888 100644 --- a/kafka/sasl/abc.py +++ b/kafka/sasl/abc.py @@ -1,12 +1,7 @@ -from __future__ import absolute_import - import abc -from kafka.vendor.six import add_metaclass - -@add_metaclass(abc.ABCMeta) -class SaslMechanism(object): +class SaslMechanism(object, metaclass=abc.ABCMeta): @abc.abstractmethod def __init__(self, **config): pass diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index 4785b1b75..9b10efbe7 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import struct # needed for SASL_GSSAPI authentication: diff --git a/kafka/sasl/msk.py b/kafka/sasl/msk.py index 7ec03215d..10896ce1f 100644 --- a/kafka/sasl/msk.py +++ b/kafka/sasl/msk.py @@ -1,11 +1,10 @@ -from __future__ import absolute_import - import datetime import hashlib import hmac import json import logging import string +import urllib # needed for AWS_MSK_IAM authentication: try: @@ -16,7 +15,6 @@ from kafka.errors import KafkaConfigurationError from kafka.sasl.abc import SaslMechanism -from kafka.vendor.six.moves import urllib log = logging.getLogger(__name__) diff --git a/kafka/sasl/oauth.py b/kafka/sasl/oauth.py index f1e959cb6..b4951744b 100644 --- a/kafka/sasl/oauth.py +++ b/kafka/sasl/oauth.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import abc import logging diff --git a/kafka/sasl/plain.py b/kafka/sasl/plain.py index 81443f5fe..2af75acfb 100644 --- a/kafka/sasl/plain.py +++ b/kafka/sasl/plain.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import logging from kafka.sasl.abc import SaslMechanism diff --git a/kafka/sasl/scram.py b/kafka/sasl/scram.py index d8cd071a7..420d88327 100644 --- a/kafka/sasl/scram.py +++ b/kafka/sasl/scram.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import base64 import hashlib import hmac @@ -8,18 +6,13 @@ from kafka.sasl.abc import SaslMechanism -from kafka.vendor import six log = logging.getLogger(__name__) -if six.PY2: - def xor_bytes(left, right): - return bytearray(ord(lb) ^ ord(rb) for lb, rb in zip(left, right)) -else: - def xor_bytes(left, right): - return bytes(lb ^ rb for lb, rb in zip(left, right)) +def xor_bytes(left, right): + return bytes(lb ^ rb for lb, rb in zip(left, right)) class SaslMechanismScram(SaslMechanism): diff --git a/kafka/sasl/sspi.py b/kafka/sasl/sspi.py index f4c95d037..2aa686479 100644 --- a/kafka/sasl/sspi.py +++ b/kafka/sasl/sspi.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import logging # Windows-only diff --git a/kafka/serializer/__init__.py b/kafka/serializer/__init__.py index 90cd93ab2..168277519 100644 --- a/kafka/serializer/__init__.py +++ b/kafka/serializer/__init__.py @@ -1,3 +1 @@ -from __future__ import absolute_import - from kafka.serializer.abstract import Serializer, Deserializer diff --git a/kafka/serializer/abstract.py b/kafka/serializer/abstract.py index 18ad8d69c..b656b5cef 100644 --- a/kafka/serializer/abstract.py +++ b/kafka/serializer/abstract.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import abc diff --git a/kafka/socks5_wrapper.py b/kafka/socks5_wrapper.py index 6715f2093..e8f3c9682 100755 --- a/kafka/socks5_wrapper.py +++ b/kafka/socks5_wrapper.py @@ -1,13 +1,9 @@ -try: - from urllib.parse import urlparse -except ImportError: - from urlparse import urlparse - import errno import logging import random import socket import struct +from urllib.parse import urlparse log = logging.getLogger(__name__) diff --git a/kafka/structs.py b/kafka/structs.py index 16ba0daac..eb8255e6a 100644 --- a/kafka/structs.py +++ b/kafka/structs.py @@ -1,5 +1,4 @@ """ Other useful structs """ -from __future__ import absolute_import from collections import namedtuple diff --git a/kafka/util.py b/kafka/util.py index 6bc4c7051..29482bce1 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import binascii import functools import re @@ -7,23 +5,18 @@ import weakref from kafka.errors import KafkaTimeoutError -from kafka.vendor import six -if six.PY3: - MAX_INT = 2 ** 31 - TO_SIGNED = 2 ** 32 +MAX_INT = 2 ** 31 +TO_SIGNED = 2 ** 32 - def crc32(data): - crc = binascii.crc32(data) - # py2 and py3 behave a little differently - # CRC is encoded as a signed int in kafka protocol - # so we'll convert the py3 unsigned result to signed - if crc >= MAX_INT: - crc -= TO_SIGNED - return crc -else: - from binascii import crc32 # noqa: F401 +def crc32(data): + crc = binascii.crc32(data) + # CRC is encoded as a signed int in kafka protocol + # so we'll convert the unsigned result to signed + if crc >= MAX_INT: + crc -= TO_SIGNED + return crc class Timer: @@ -76,7 +69,7 @@ def ensure_valid_topic_name(topic): # https://github.com/apache/kafka/blob/39eb31feaeebfb184d98cc5d94da9148c2319d81/clients/src/main/java/org/apache/kafka/common/internals/Topic.java if topic is None: raise TypeError('All topics must not be None') - if not isinstance(topic, six.string_types): + if not isinstance(topic, str): raise TypeError('All topics must be strings') if len(topic) == 0: raise ValueError('All topics must be non-empty strings') diff --git a/kafka/vendor/enum34.py b/kafka/vendor/enum34.py deleted file mode 100644 index 5f64bd2d8..000000000 --- a/kafka/vendor/enum34.py +++ /dev/null @@ -1,841 +0,0 @@ -# pylint: skip-file -# vendored from: -# https://bitbucket.org/stoneleaf/enum34/src/58c4cd7174ca35f164304c8a6f0a4d47b779c2a7/enum/__init__.py?at=1.1.6 - -"""Python Enumerations""" - -import sys as _sys - -__all__ = ['Enum', 'IntEnum', 'unique'] - -version = 1, 1, 6 - -pyver = float('%s.%s' % _sys.version_info[:2]) - -try: - any -except NameError: - def any(iterable): - for element in iterable: - if element: - return True - return False - -try: - from collections import OrderedDict -except ImportError: - OrderedDict = None - -try: - basestring -except NameError: - # In Python 2 basestring is the ancestor of both str and unicode - # in Python 3 it's just str, but was missing in 3.1 - basestring = str - -try: - unicode -except NameError: - # In Python 3 unicode no longer exists (it's just str) - unicode = str - -class _RouteClassAttributeToGetattr(object): - """Route attribute access on a class to __getattr__. - - This is a descriptor, used to define attributes that act differently when - accessed through an instance and through a class. Instance access remains - normal, but access to an attribute through a class will be routed to the - class's __getattr__ method; this is done by raising AttributeError. - - """ - def __init__(self, fget=None): - self.fget = fget - - def __get__(self, instance, ownerclass=None): - if instance is None: - raise AttributeError() - return self.fget(instance) - - def __set__(self, instance, value): - raise AttributeError("can't set attribute") - - def __delete__(self, instance): - raise AttributeError("can't delete attribute") - - -def _is_descriptor(obj): - """Returns True if obj is a descriptor, False otherwise.""" - return ( - hasattr(obj, '__get__') or - hasattr(obj, '__set__') or - hasattr(obj, '__delete__')) - - -def _is_dunder(name): - """Returns True if a __dunder__ name, False otherwise.""" - return (name[:2] == name[-2:] == '__' and - name[2:3] != '_' and - name[-3:-2] != '_' and - len(name) > 4) - - -def _is_sunder(name): - """Returns True if a _sunder_ name, False otherwise.""" - return (name[0] == name[-1] == '_' and - name[1:2] != '_' and - name[-2:-1] != '_' and - len(name) > 2) - - -def _make_class_unpicklable(cls): - """Make the given class un-picklable.""" - def _break_on_call_reduce(self, protocol=None): - raise TypeError('%r cannot be pickled' % self) - cls.__reduce_ex__ = _break_on_call_reduce - cls.__module__ = '' - - -class _EnumDict(dict): - """Track enum member order and ensure member names are not reused. - - EnumMeta will use the names found in self._member_names as the - enumeration member names. - - """ - def __init__(self): - super(_EnumDict, self).__init__() - self._member_names = [] - - def __setitem__(self, key, value): - """Changes anything not dundered or not a descriptor. - - If a descriptor is added with the same name as an enum member, the name - is removed from _member_names (this may leave a hole in the numerical - sequence of values). - - If an enum member name is used twice, an error is raised; duplicate - values are not checked for. - - Single underscore (sunder) names are reserved. - - Note: in 3.x __order__ is simply discarded as a not necessary piece - leftover from 2.x - - """ - if pyver >= 3.0 and key in ('_order_', '__order__'): - return - elif key == '__order__': - key = '_order_' - if _is_sunder(key): - if key != '_order_': - raise ValueError('_names_ are reserved for future Enum use') - elif _is_dunder(key): - pass - elif key in self._member_names: - # descriptor overwriting an enum? - raise TypeError('Attempted to reuse key: %r' % key) - elif not _is_descriptor(value): - if key in self: - # enum overwriting a descriptor? - raise TypeError('Key already defined as: %r' % self[key]) - self._member_names.append(key) - super(_EnumDict, self).__setitem__(key, value) - - -# Dummy value for Enum as EnumMeta explicity checks for it, but of course until -# EnumMeta finishes running the first time the Enum class doesn't exist. This -# is also why there are checks in EnumMeta like `if Enum is not None` -Enum = None - - -class EnumMeta(type): - """Metaclass for Enum""" - @classmethod - def __prepare__(metacls, cls, bases): - return _EnumDict() - - def __new__(metacls, cls, bases, classdict): - # an Enum class is final once enumeration items have been defined; it - # cannot be mixed with other types (int, float, etc.) if it has an - # inherited __new__ unless a new __new__ is defined (or the resulting - # class will fail). - if type(classdict) is dict: - original_dict = classdict - classdict = _EnumDict() - for k, v in original_dict.items(): - classdict[k] = v - - member_type, first_enum = metacls._get_mixins_(bases) - __new__, save_new, use_args = metacls._find_new_(classdict, member_type, - first_enum) - # save enum items into separate mapping so they don't get baked into - # the new class - members = dict((k, classdict[k]) for k in classdict._member_names) - for name in classdict._member_names: - del classdict[name] - - # py2 support for definition order - _order_ = classdict.get('_order_') - if _order_ is None: - if pyver < 3.0: - try: - _order_ = [name for (name, value) in sorted(members.items(), key=lambda item: item[1])] - except TypeError: - _order_ = [name for name in sorted(members.keys())] - else: - _order_ = classdict._member_names - else: - del classdict['_order_'] - if pyver < 3.0: - _order_ = _order_.replace(',', ' ').split() - aliases = [name for name in members if name not in _order_] - _order_ += aliases - - # check for illegal enum names (any others?) - invalid_names = set(members) & set(['mro']) - if invalid_names: - raise ValueError('Invalid enum member name(s): %s' % ( - ', '.join(invalid_names), )) - - # save attributes from super classes so we know if we can take - # the shortcut of storing members in the class dict - base_attributes = set([a for b in bases for a in b.__dict__]) - # create our new Enum type - enum_class = super(EnumMeta, metacls).__new__(metacls, cls, bases, classdict) - enum_class._member_names_ = [] # names in random order - if OrderedDict is not None: - enum_class._member_map_ = OrderedDict() - else: - enum_class._member_map_ = {} # name->value map - enum_class._member_type_ = member_type - - # Reverse value->name map for hashable values. - enum_class._value2member_map_ = {} - - # instantiate them, checking for duplicates as we go - # we instantiate first instead of checking for duplicates first in case - # a custom __new__ is doing something funky with the values -- such as - # auto-numbering ;) - if __new__ is None: - __new__ = enum_class.__new__ - for member_name in _order_: - value = members[member_name] - if not isinstance(value, tuple): - args = (value, ) - else: - args = value - if member_type is tuple: # special case for tuple enums - args = (args, ) # wrap it one more time - if not use_args or not args: - enum_member = __new__(enum_class) - if not hasattr(enum_member, '_value_'): - enum_member._value_ = value - else: - enum_member = __new__(enum_class, *args) - if not hasattr(enum_member, '_value_'): - enum_member._value_ = member_type(*args) - value = enum_member._value_ - enum_member._name_ = member_name - enum_member.__objclass__ = enum_class - enum_member.__init__(*args) - # If another member with the same value was already defined, the - # new member becomes an alias to the existing one. - for name, canonical_member in enum_class._member_map_.items(): - if canonical_member.value == enum_member._value_: - enum_member = canonical_member - break - else: - # Aliases don't appear in member names (only in __members__). - enum_class._member_names_.append(member_name) - # performance boost for any member that would not shadow - # a DynamicClassAttribute (aka _RouteClassAttributeToGetattr) - if member_name not in base_attributes: - setattr(enum_class, member_name, enum_member) - # now add to _member_map_ - enum_class._member_map_[member_name] = enum_member - try: - # This may fail if value is not hashable. We can't add the value - # to the map, and by-value lookups for this value will be - # linear. - enum_class._value2member_map_[value] = enum_member - except TypeError: - pass - - - # If a custom type is mixed into the Enum, and it does not know how - # to pickle itself, pickle.dumps will succeed but pickle.loads will - # fail. Rather than have the error show up later and possibly far - # from the source, sabotage the pickle protocol for this class so - # that pickle.dumps also fails. - # - # However, if the new class implements its own __reduce_ex__, do not - # sabotage -- it's on them to make sure it works correctly. We use - # __reduce_ex__ instead of any of the others as it is preferred by - # pickle over __reduce__, and it handles all pickle protocols. - unpicklable = False - if '__reduce_ex__' not in classdict: - if member_type is not object: - methods = ('__getnewargs_ex__', '__getnewargs__', - '__reduce_ex__', '__reduce__') - if not any(m in member_type.__dict__ for m in methods): - _make_class_unpicklable(enum_class) - unpicklable = True - - - # double check that repr and friends are not the mixin's or various - # things break (such as pickle) - for name in ('__repr__', '__str__', '__format__', '__reduce_ex__'): - class_method = getattr(enum_class, name) - obj_method = getattr(member_type, name, None) - enum_method = getattr(first_enum, name, None) - if name not in classdict and class_method is not enum_method: - if name == '__reduce_ex__' and unpicklable: - continue - setattr(enum_class, name, enum_method) - - # method resolution and int's are not playing nice - # Python's less than 2.6 use __cmp__ - - if pyver < 2.6: - - if issubclass(enum_class, int): - setattr(enum_class, '__cmp__', getattr(int, '__cmp__')) - - elif pyver < 3.0: - - if issubclass(enum_class, int): - for method in ( - '__le__', - '__lt__', - '__gt__', - '__ge__', - '__eq__', - '__ne__', - '__hash__', - ): - setattr(enum_class, method, getattr(int, method)) - - # replace any other __new__ with our own (as long as Enum is not None, - # anyway) -- again, this is to support pickle - if Enum is not None: - # if the user defined their own __new__, save it before it gets - # clobbered in case they subclass later - if save_new: - setattr(enum_class, '__member_new__', enum_class.__dict__['__new__']) - setattr(enum_class, '__new__', Enum.__dict__['__new__']) - return enum_class - - def __bool__(cls): - """ - classes/types should always be True. - """ - return True - - def __call__(cls, value, names=None, module=None, type=None, start=1): - """Either returns an existing member, or creates a new enum class. - - This method is used both when an enum class is given a value to match - to an enumeration member (i.e. Color(3)) and for the functional API - (i.e. Color = Enum('Color', names='red green blue')). - - When used for the functional API: `module`, if set, will be stored in - the new class' __module__ attribute; `type`, if set, will be mixed in - as the first base class. - - Note: if `module` is not set this routine will attempt to discover the - calling module by walking the frame stack; if this is unsuccessful - the resulting class will not be pickleable. - - """ - if names is None: # simple value lookup - return cls.__new__(cls, value) - # otherwise, functional API: we're creating a new Enum type - return cls._create_(value, names, module=module, type=type, start=start) - - def __contains__(cls, member): - return isinstance(member, cls) and member.name in cls._member_map_ - - def __delattr__(cls, attr): - # nicer error message when someone tries to delete an attribute - # (see issue19025). - if attr in cls._member_map_: - raise AttributeError( - "%s: cannot delete Enum member." % cls.__name__) - super(EnumMeta, cls).__delattr__(attr) - - def __dir__(self): - return (['__class__', '__doc__', '__members__', '__module__'] + - self._member_names_) - - @property - def __members__(cls): - """Returns a mapping of member name->value. - - This mapping lists all enum members, including aliases. Note that this - is a copy of the internal mapping. - - """ - return cls._member_map_.copy() - - def __getattr__(cls, name): - """Return the enum member matching `name` - - We use __getattr__ instead of descriptors or inserting into the enum - class' __dict__ in order to support `name` and `value` being both - properties for enum members (which live in the class' __dict__) and - enum members themselves. - - """ - if _is_dunder(name): - raise AttributeError(name) - try: - return cls._member_map_[name] - except KeyError: - raise AttributeError(name) - - def __getitem__(cls, name): - return cls._member_map_[name] - - def __iter__(cls): - return (cls._member_map_[name] for name in cls._member_names_) - - def __reversed__(cls): - return (cls._member_map_[name] for name in reversed(cls._member_names_)) - - def __len__(cls): - return len(cls._member_names_) - - __nonzero__ = __bool__ - - def __repr__(cls): - return "" % cls.__name__ - - def __setattr__(cls, name, value): - """Block attempts to reassign Enum members. - - A simple assignment to the class namespace only changes one of the - several possible ways to get an Enum member from the Enum class, - resulting in an inconsistent Enumeration. - - """ - member_map = cls.__dict__.get('_member_map_', {}) - if name in member_map: - raise AttributeError('Cannot reassign members.') - super(EnumMeta, cls).__setattr__(name, value) - - def _create_(cls, class_name, names=None, module=None, type=None, start=1): - """Convenience method to create a new Enum class. - - `names` can be: - - * A string containing member names, separated either with spaces or - commas. Values are auto-numbered from 1. - * An iterable of member names. Values are auto-numbered from 1. - * An iterable of (member name, value) pairs. - * A mapping of member name -> value. - - """ - if pyver < 3.0: - # if class_name is unicode, attempt a conversion to ASCII - if isinstance(class_name, unicode): - try: - class_name = class_name.encode('ascii') - except UnicodeEncodeError: - raise TypeError('%r is not representable in ASCII' % class_name) - metacls = cls.__class__ - if type is None: - bases = (cls, ) - else: - bases = (type, cls) - classdict = metacls.__prepare__(class_name, bases) - _order_ = [] - - # special processing needed for names? - if isinstance(names, basestring): - names = names.replace(',', ' ').split() - if isinstance(names, (tuple, list)) and isinstance(names[0], basestring): - names = [(e, i+start) for (i, e) in enumerate(names)] - - # Here, names is either an iterable of (name, value) or a mapping. - item = None # in case names is empty - for item in names: - if isinstance(item, basestring): - member_name, member_value = item, names[item] - else: - member_name, member_value = item - classdict[member_name] = member_value - _order_.append(member_name) - # only set _order_ in classdict if name/value was not from a mapping - if not isinstance(item, basestring): - classdict['_order_'] = ' '.join(_order_) - enum_class = metacls.__new__(metacls, class_name, bases, classdict) - - # TODO: replace the frame hack if a blessed way to know the calling - # module is ever developed - if module is None: - try: - module = _sys._getframe(2).f_globals['__name__'] - except (AttributeError, ValueError): - pass - if module is None: - _make_class_unpicklable(enum_class) - else: - enum_class.__module__ = module - - return enum_class - - @staticmethod - def _get_mixins_(bases): - """Returns the type for creating enum members, and the first inherited - enum class. - - bases: the tuple of bases that was given to __new__ - - """ - if not bases or Enum is None: - return object, Enum - - - # double check that we are not subclassing a class with existing - # enumeration members; while we're at it, see if any other data - # type has been mixed in so we can use the correct __new__ - member_type = first_enum = None - for base in bases: - if (base is not Enum and - issubclass(base, Enum) and - base._member_names_): - raise TypeError("Cannot extend enumerations") - # base is now the last base in bases - if not issubclass(base, Enum): - raise TypeError("new enumerations must be created as " - "`ClassName([mixin_type,] enum_type)`") - - # get correct mix-in type (either mix-in type of Enum subclass, or - # first base if last base is Enum) - if not issubclass(bases[0], Enum): - member_type = bases[0] # first data type - first_enum = bases[-1] # enum type - else: - for base in bases[0].__mro__: - # most common: (IntEnum, int, Enum, object) - # possible: (, , - # , , - # ) - if issubclass(base, Enum): - if first_enum is None: - first_enum = base - else: - if member_type is None: - member_type = base - - return member_type, first_enum - - if pyver < 3.0: - @staticmethod - def _find_new_(classdict, member_type, first_enum): - """Returns the __new__ to be used for creating the enum members. - - classdict: the class dictionary given to __new__ - member_type: the data type whose __new__ will be used by default - first_enum: enumeration to check for an overriding __new__ - - """ - # now find the correct __new__, checking to see of one was defined - # by the user; also check earlier enum classes in case a __new__ was - # saved as __member_new__ - __new__ = classdict.get('__new__', None) - if __new__: - return None, True, True # __new__, save_new, use_args - - N__new__ = getattr(None, '__new__') - O__new__ = getattr(object, '__new__') - if Enum is None: - E__new__ = N__new__ - else: - E__new__ = Enum.__dict__['__new__'] - # check all possibles for __member_new__ before falling back to - # __new__ - for method in ('__member_new__', '__new__'): - for possible in (member_type, first_enum): - try: - target = possible.__dict__[method] - except (AttributeError, KeyError): - target = getattr(possible, method, None) - if target not in [ - None, - N__new__, - O__new__, - E__new__, - ]: - if method == '__member_new__': - classdict['__new__'] = target - return None, False, True - if isinstance(target, staticmethod): - target = target.__get__(member_type) - __new__ = target - break - if __new__ is not None: - break - else: - __new__ = object.__new__ - - # if a non-object.__new__ is used then whatever value/tuple was - # assigned to the enum member name will be passed to __new__ and to the - # new enum member's __init__ - if __new__ is object.__new__: - use_args = False - else: - use_args = True - - return __new__, False, use_args - else: - @staticmethod - def _find_new_(classdict, member_type, first_enum): - """Returns the __new__ to be used for creating the enum members. - - classdict: the class dictionary given to __new__ - member_type: the data type whose __new__ will be used by default - first_enum: enumeration to check for an overriding __new__ - - """ - # now find the correct __new__, checking to see of one was defined - # by the user; also check earlier enum classes in case a __new__ was - # saved as __member_new__ - __new__ = classdict.get('__new__', None) - - # should __new__ be saved as __member_new__ later? - save_new = __new__ is not None - - if __new__ is None: - # check all possibles for __member_new__ before falling back to - # __new__ - for method in ('__member_new__', '__new__'): - for possible in (member_type, first_enum): - target = getattr(possible, method, None) - if target not in ( - None, - None.__new__, - object.__new__, - Enum.__new__, - ): - __new__ = target - break - if __new__ is not None: - break - else: - __new__ = object.__new__ - - # if a non-object.__new__ is used then whatever value/tuple was - # assigned to the enum member name will be passed to __new__ and to the - # new enum member's __init__ - if __new__ is object.__new__: - use_args = False - else: - use_args = True - - return __new__, save_new, use_args - - -######################################################## -# In order to support Python 2 and 3 with a single -# codebase we have to create the Enum methods separately -# and then use the `type(name, bases, dict)` method to -# create the class. -######################################################## -temp_enum_dict = {} -temp_enum_dict['__doc__'] = "Generic enumeration.\n\n Derive from this class to define new enumerations.\n\n" - -def __new__(cls, value): - # all enum instances are actually created during class construction - # without calling this method; this method is called by the metaclass' - # __call__ (i.e. Color(3) ), and by pickle - if type(value) is cls: - # For lookups like Color(Color.red) - value = value.value - #return value - # by-value search for a matching enum member - # see if it's in the reverse mapping (for hashable values) - try: - if value in cls._value2member_map_: - return cls._value2member_map_[value] - except TypeError: - # not there, now do long search -- O(n) behavior - for member in cls._member_map_.values(): - if member.value == value: - return member - raise ValueError("%s is not a valid %s" % (value, cls.__name__)) -temp_enum_dict['__new__'] = __new__ -del __new__ - -def __repr__(self): - return "<%s.%s: %r>" % ( - self.__class__.__name__, self._name_, self._value_) -temp_enum_dict['__repr__'] = __repr__ -del __repr__ - -def __str__(self): - return "%s.%s" % (self.__class__.__name__, self._name_) -temp_enum_dict['__str__'] = __str__ -del __str__ - -if pyver >= 3.0: - def __dir__(self): - added_behavior = [ - m - for cls in self.__class__.mro() - for m in cls.__dict__ - if m[0] != '_' and m not in self._member_map_ - ] - return (['__class__', '__doc__', '__module__', ] + added_behavior) - temp_enum_dict['__dir__'] = __dir__ - del __dir__ - -def __format__(self, format_spec): - # mixed-in Enums should use the mixed-in type's __format__, otherwise - # we can get strange results with the Enum name showing up instead of - # the value - - # pure Enum branch - if self._member_type_ is object: - cls = str - val = str(self) - # mix-in branch - else: - cls = self._member_type_ - val = self.value - return cls.__format__(val, format_spec) -temp_enum_dict['__format__'] = __format__ -del __format__ - - -#################################### -# Python's less than 2.6 use __cmp__ - -if pyver < 2.6: - - def __cmp__(self, other): - if type(other) is self.__class__: - if self is other: - return 0 - return -1 - return NotImplemented - raise TypeError("unorderable types: %s() and %s()" % (self.__class__.__name__, other.__class__.__name__)) - temp_enum_dict['__cmp__'] = __cmp__ - del __cmp__ - -else: - - def __le__(self, other): - raise TypeError("unorderable types: %s() <= %s()" % (self.__class__.__name__, other.__class__.__name__)) - temp_enum_dict['__le__'] = __le__ - del __le__ - - def __lt__(self, other): - raise TypeError("unorderable types: %s() < %s()" % (self.__class__.__name__, other.__class__.__name__)) - temp_enum_dict['__lt__'] = __lt__ - del __lt__ - - def __ge__(self, other): - raise TypeError("unorderable types: %s() >= %s()" % (self.__class__.__name__, other.__class__.__name__)) - temp_enum_dict['__ge__'] = __ge__ - del __ge__ - - def __gt__(self, other): - raise TypeError("unorderable types: %s() > %s()" % (self.__class__.__name__, other.__class__.__name__)) - temp_enum_dict['__gt__'] = __gt__ - del __gt__ - - -def __eq__(self, other): - if type(other) is self.__class__: - return self is other - return NotImplemented -temp_enum_dict['__eq__'] = __eq__ -del __eq__ - -def __ne__(self, other): - if type(other) is self.__class__: - return self is not other - return NotImplemented -temp_enum_dict['__ne__'] = __ne__ -del __ne__ - -def __hash__(self): - return hash(self._name_) -temp_enum_dict['__hash__'] = __hash__ -del __hash__ - -def __reduce_ex__(self, proto): - return self.__class__, (self._value_, ) -temp_enum_dict['__reduce_ex__'] = __reduce_ex__ -del __reduce_ex__ - -# _RouteClassAttributeToGetattr is used to provide access to the `name` -# and `value` properties of enum members while keeping some measure of -# protection from modification, while still allowing for an enumeration -# to have members named `name` and `value`. This works because enumeration -# members are not set directly on the enum class -- __getattr__ is -# used to look them up. - -@_RouteClassAttributeToGetattr -def name(self): - return self._name_ -temp_enum_dict['name'] = name -del name - -@_RouteClassAttributeToGetattr -def value(self): - return self._value_ -temp_enum_dict['value'] = value -del value - -@classmethod -def _convert(cls, name, module, filter, source=None): - """ - Create a new Enum subclass that replaces a collection of global constants - """ - # convert all constants from source (or module) that pass filter() to - # a new Enum called name, and export the enum and its members back to - # module; - # also, replace the __reduce_ex__ method so unpickling works in - # previous Python versions - module_globals = vars(_sys.modules[module]) - if source: - source = vars(source) - else: - source = module_globals - members = dict((name, value) for name, value in source.items() if filter(name)) - cls = cls(name, members, module=module) - cls.__reduce_ex__ = _reduce_ex_by_name - module_globals.update(cls.__members__) - module_globals[name] = cls - return cls -temp_enum_dict['_convert'] = _convert -del _convert - -Enum = EnumMeta('Enum', (object, ), temp_enum_dict) -del temp_enum_dict - -# Enum has now been created -########################### - -class IntEnum(int, Enum): - """Enum where members are also (and must be) ints""" - -def _reduce_ex_by_name(self, proto): - return self.name - -def unique(enumeration): - """Class decorator that ensures only unique members exist in an enumeration.""" - duplicates = [] - for name, member in enumeration.__members__.items(): - if name != member.name: - duplicates.append((name, member.name)) - if duplicates: - duplicate_names = ', '.join( - ["%s -> %s" % (alias, name) for (alias, name) in duplicates] - ) - raise ValueError('duplicate names found in %r: %s' % - (enumeration, duplicate_names) - ) - return enumeration diff --git a/kafka/vendor/selectors34.py b/kafka/vendor/selectors34.py deleted file mode 100644 index 787490340..000000000 --- a/kafka/vendor/selectors34.py +++ /dev/null @@ -1,641 +0,0 @@ -# pylint: skip-file -# vendored from https://github.com/berkerpeksag/selectors34 -# at commit ff61b82168d2cc9c4922ae08e2a8bf94aab61ea2 (unreleased, ~1.2) -# -# Original author: Charles-Francois Natali (c.f.natali[at]gmail.com) -# Maintainer: Berker Peksag (berker.peksag[at]gmail.com) -# Also see https://pypi.python.org/pypi/selectors34 -"""Selectors module. - -This module allows high-level and efficient I/O multiplexing, built upon the -`select` module primitives. - -The following code adapted from trollius.selectors. -""" -from __future__ import absolute_import - -from abc import ABCMeta, abstractmethod -from collections import namedtuple -try: - from collections.abc import Mapping -except ImportError: - from collections import Mapping -from errno import EINTR -import math -import select -import sys - -from kafka.vendor import six - - -def _wrap_error(exc, mapping, key): - if key not in mapping: - return - new_err_cls = mapping[key] - new_err = new_err_cls(*exc.args) - - # raise a new exception with the original traceback - if hasattr(exc, '__traceback__'): - traceback = exc.__traceback__ - else: - traceback = sys.exc_info()[2] - six.reraise(new_err_cls, new_err, traceback) - - -# generic events, that must be mapped to implementation-specific ones -EVENT_READ = (1 << 0) -EVENT_WRITE = (1 << 1) - - -def _fileobj_to_fd(fileobj): - """Return a file descriptor from a file object. - - Parameters: - fileobj -- file object or file descriptor - - Returns: - corresponding file descriptor - - Raises: - ValueError if the object is invalid - """ - if isinstance(fileobj, six.integer_types): - fd = fileobj - else: - try: - fd = int(fileobj.fileno()) - except (AttributeError, TypeError, ValueError): - raise ValueError("Invalid file object: " - "{0!r}".format(fileobj)) - if fd < 0: - raise ValueError("Invalid file descriptor: {0}".format(fd)) - return fd - - -SelectorKey = namedtuple('SelectorKey', ['fileobj', 'fd', 'events', 'data']) -"""Object used to associate a file object to its backing file descriptor, -selected event mask and attached data.""" - - -class _SelectorMapping(Mapping): - """Mapping of file objects to selector keys.""" - - def __init__(self, selector): - self._selector = selector - - def __len__(self): - return len(self._selector._fd_to_key) - - def __getitem__(self, fileobj): - try: - fd = self._selector._fileobj_lookup(fileobj) - return self._selector._fd_to_key[fd] - except KeyError: - raise KeyError("{0!r} is not registered".format(fileobj)) - - def __iter__(self): - return iter(self._selector._fd_to_key) - -# Using six.add_metaclass() decorator instead of six.with_metaclass() because -# the latter leaks temporary_class to garbage with gc disabled -@six.add_metaclass(ABCMeta) -class BaseSelector(object): - """Selector abstract base class. - - A selector supports registering file objects to be monitored for specific - I/O events. - - A file object is a file descriptor or any object with a `fileno()` method. - An arbitrary object can be attached to the file object, which can be used - for example to store context information, a callback, etc. - - A selector can use various implementations (select(), poll(), epoll()...) - depending on the platform. The default `Selector` class uses the most - efficient implementation on the current platform. - """ - - @abstractmethod - def register(self, fileobj, events, data=None): - """Register a file object. - - Parameters: - fileobj -- file object or file descriptor - events -- events to monitor (bitwise mask of EVENT_READ|EVENT_WRITE) - data -- attached data - - Returns: - SelectorKey instance - - Raises: - ValueError if events is invalid - KeyError if fileobj is already registered - OSError if fileobj is closed or otherwise is unacceptable to - the underlying system call (if a system call is made) - - Note: - OSError may or may not be raised - """ - raise NotImplementedError - - @abstractmethod - def unregister(self, fileobj): - """Unregister a file object. - - Parameters: - fileobj -- file object or file descriptor - - Returns: - SelectorKey instance - - Raises: - KeyError if fileobj is not registered - - Note: - If fileobj is registered but has since been closed this does - *not* raise OSError (even if the wrapped syscall does) - """ - raise NotImplementedError - - def modify(self, fileobj, events, data=None): - """Change a registered file object monitored events or attached data. - - Parameters: - fileobj -- file object or file descriptor - events -- events to monitor (bitwise mask of EVENT_READ|EVENT_WRITE) - data -- attached data - - Returns: - SelectorKey instance - - Raises: - Anything that unregister() or register() raises - """ - self.unregister(fileobj) - return self.register(fileobj, events, data) - - @abstractmethod - def select(self, timeout=None): - """Perform the actual selection, until some monitored file objects are - ready or a timeout expires. - - Parameters: - timeout -- if timeout > 0, this specifies the maximum wait time, in - seconds - if timeout <= 0, the select() call won't block, and will - report the currently ready file objects - if timeout is None, select() will block until a monitored - file object becomes ready - - Returns: - list of (key, events) for ready file objects - `events` is a bitwise mask of EVENT_READ|EVENT_WRITE - """ - raise NotImplementedError - - def close(self): - """Close the selector. - - This must be called to make sure that any underlying resource is freed. - """ - pass - - def get_key(self, fileobj): - """Return the key associated to a registered file object. - - Returns: - SelectorKey for this file object - """ - mapping = self.get_map() - if mapping is None: - raise RuntimeError('Selector is closed') - try: - return mapping[fileobj] - except KeyError: - raise KeyError("{0!r} is not registered".format(fileobj)) - - @abstractmethod - def get_map(self): - """Return a mapping of file objects to selector keys.""" - raise NotImplementedError - - def __enter__(self): - return self - - def __exit__(self, *args): - self.close() - - -class _BaseSelectorImpl(BaseSelector): - """Base selector implementation.""" - - def __init__(self): - # this maps file descriptors to keys - self._fd_to_key = {} - # read-only mapping returned by get_map() - self._map = _SelectorMapping(self) - - def _fileobj_lookup(self, fileobj): - """Return a file descriptor from a file object. - - This wraps _fileobj_to_fd() to do an exhaustive search in case - the object is invalid but we still have it in our map. This - is used by unregister() so we can unregister an object that - was previously registered even if it is closed. It is also - used by _SelectorMapping. - """ - try: - return _fileobj_to_fd(fileobj) - except ValueError: - # Do an exhaustive search. - for key in self._fd_to_key.values(): - if key.fileobj is fileobj: - return key.fd - # Raise ValueError after all. - raise - - def register(self, fileobj, events, data=None): - if (not events) or (events & ~(EVENT_READ | EVENT_WRITE)): - raise ValueError("Invalid events: {0!r}".format(events)) - - key = SelectorKey(fileobj, self._fileobj_lookup(fileobj), events, data) - - if key.fd in self._fd_to_key: - raise KeyError("{0!r} (FD {1}) is already registered" - .format(fileobj, key.fd)) - - self._fd_to_key[key.fd] = key - return key - - def unregister(self, fileobj): - try: - key = self._fd_to_key.pop(self._fileobj_lookup(fileobj)) - except KeyError: - raise KeyError("{0!r} is not registered".format(fileobj)) - return key - - def modify(self, fileobj, events, data=None): - # TODO: Subclasses can probably optimize this even further. - try: - key = self._fd_to_key[self._fileobj_lookup(fileobj)] - except KeyError: - raise KeyError("{0!r} is not registered".format(fileobj)) - if events != key.events: - self.unregister(fileobj) - key = self.register(fileobj, events, data) - elif data != key.data: - # Use a shortcut to update the data. - key = key._replace(data=data) - self._fd_to_key[key.fd] = key - return key - - def close(self): - self._fd_to_key.clear() - self._map = None - - def get_map(self): - return self._map - - def _key_from_fd(self, fd): - """Return the key associated to a given file descriptor. - - Parameters: - fd -- file descriptor - - Returns: - corresponding key, or None if not found - """ - try: - return self._fd_to_key[fd] - except KeyError: - return None - - -class SelectSelector(_BaseSelectorImpl): - """Select-based selector.""" - - def __init__(self): - super(SelectSelector, self).__init__() - self._readers = set() - self._writers = set() - - def register(self, fileobj, events, data=None): - key = super(SelectSelector, self).register(fileobj, events, data) - if events & EVENT_READ: - self._readers.add(key.fd) - if events & EVENT_WRITE: - self._writers.add(key.fd) - return key - - def unregister(self, fileobj): - key = super(SelectSelector, self).unregister(fileobj) - self._readers.discard(key.fd) - self._writers.discard(key.fd) - return key - - if sys.platform == 'win32': - def _select(self, r, w, _, timeout=None): - r, w, x = select.select(r, w, w, timeout) - return r, w + x, [] - else: - _select = staticmethod(select.select) - - def select(self, timeout=None): - timeout = None if timeout is None else max(timeout, 0) - ready = [] - try: - r, w, _ = self._select(self._readers, self._writers, [], timeout) - except select.error as exc: - if exc.args[0] == EINTR: - return ready - else: - raise - r = set(r) - w = set(w) - for fd in r | w: - events = 0 - if fd in r: - events |= EVENT_READ - if fd in w: - events |= EVENT_WRITE - - key = self._key_from_fd(fd) - if key: - ready.append((key, events & key.events)) - return ready - - -if hasattr(select, 'poll'): - - class PollSelector(_BaseSelectorImpl): - """Poll-based selector.""" - - def __init__(self): - super(PollSelector, self).__init__() - self._poll = select.poll() - - def register(self, fileobj, events, data=None): - key = super(PollSelector, self).register(fileobj, events, data) - poll_events = 0 - if events & EVENT_READ: - poll_events |= select.POLLIN - if events & EVENT_WRITE: - poll_events |= select.POLLOUT - self._poll.register(key.fd, poll_events) - return key - - def unregister(self, fileobj): - key = super(PollSelector, self).unregister(fileobj) - self._poll.unregister(key.fd) - return key - - def select(self, timeout=None): - if timeout is None: - timeout = None - elif timeout <= 0: - timeout = 0 - else: - # poll() has a resolution of 1 millisecond, round away from - # zero to wait *at least* timeout seconds. - timeout = int(math.ceil(timeout * 1e3)) - ready = [] - try: - fd_event_list = self._poll.poll(timeout) - except select.error as exc: - if exc.args[0] == EINTR: - return ready - else: - raise - for fd, event in fd_event_list: - events = 0 - if event & ~select.POLLIN: - events |= EVENT_WRITE - if event & ~select.POLLOUT: - events |= EVENT_READ - - key = self._key_from_fd(fd) - if key: - ready.append((key, events & key.events)) - return ready - - -if hasattr(select, 'epoll'): - - class EpollSelector(_BaseSelectorImpl): - """Epoll-based selector.""" - - def __init__(self): - super(EpollSelector, self).__init__() - self._epoll = select.epoll() - - def fileno(self): - return self._epoll.fileno() - - def register(self, fileobj, events, data=None): - key = super(EpollSelector, self).register(fileobj, events, data) - epoll_events = 0 - if events & EVENT_READ: - epoll_events |= select.EPOLLIN - if events & EVENT_WRITE: - epoll_events |= select.EPOLLOUT - self._epoll.register(key.fd, epoll_events) - return key - - def unregister(self, fileobj): - key = super(EpollSelector, self).unregister(fileobj) - try: - self._epoll.unregister(key.fd) - except IOError: - # This can happen if the FD was closed since it - # was registered. - pass - return key - - def select(self, timeout=None): - if timeout is None: - timeout = -1 - elif timeout <= 0: - timeout = 0 - else: - # epoll_wait() has a resolution of 1 millisecond, round away - # from zero to wait *at least* timeout seconds. - timeout = math.ceil(timeout * 1e3) * 1e-3 - - # epoll_wait() expects `maxevents` to be greater than zero; - # we want to make sure that `select()` can be called when no - # FD is registered. - max_ev = max(len(self._fd_to_key), 1) - - ready = [] - try: - fd_event_list = self._epoll.poll(timeout, max_ev) - except IOError as exc: - if exc.errno == EINTR: - return ready - else: - raise - for fd, event in fd_event_list: - events = 0 - if event & ~select.EPOLLIN: - events |= EVENT_WRITE - if event & ~select.EPOLLOUT: - events |= EVENT_READ - - key = self._key_from_fd(fd) - if key: - ready.append((key, events & key.events)) - return ready - - def close(self): - self._epoll.close() - super(EpollSelector, self).close() - - -if hasattr(select, 'devpoll'): - - class DevpollSelector(_BaseSelectorImpl): - """Solaris /dev/poll selector.""" - - def __init__(self): - super(DevpollSelector, self).__init__() - self._devpoll = select.devpoll() - - def fileno(self): - return self._devpoll.fileno() - - def register(self, fileobj, events, data=None): - key = super(DevpollSelector, self).register(fileobj, events, data) - poll_events = 0 - if events & EVENT_READ: - poll_events |= select.POLLIN - if events & EVENT_WRITE: - poll_events |= select.POLLOUT - self._devpoll.register(key.fd, poll_events) - return key - - def unregister(self, fileobj): - key = super(DevpollSelector, self).unregister(fileobj) - self._devpoll.unregister(key.fd) - return key - - def select(self, timeout=None): - if timeout is None: - timeout = None - elif timeout <= 0: - timeout = 0 - else: - # devpoll() has a resolution of 1 millisecond, round away from - # zero to wait *at least* timeout seconds. - timeout = math.ceil(timeout * 1e3) - ready = [] - try: - fd_event_list = self._devpoll.poll(timeout) - except OSError as exc: - if exc.errno == EINTR: - return ready - else: - raise - for fd, event in fd_event_list: - events = 0 - if event & ~select.POLLIN: - events |= EVENT_WRITE - if event & ~select.POLLOUT: - events |= EVENT_READ - - key = self._key_from_fd(fd) - if key: - ready.append((key, events & key.events)) - return ready - - def close(self): - self._devpoll.close() - super(DevpollSelector, self).close() - - -if hasattr(select, 'kqueue'): - - class KqueueSelector(_BaseSelectorImpl): - """Kqueue-based selector.""" - - def __init__(self): - super(KqueueSelector, self).__init__() - self._kqueue = select.kqueue() - - def fileno(self): - return self._kqueue.fileno() - - def register(self, fileobj, events, data=None): - key = super(KqueueSelector, self).register(fileobj, events, data) - if events & EVENT_READ: - kev = select.kevent(key.fd, select.KQ_FILTER_READ, - select.KQ_EV_ADD) - self._kqueue.control([kev], 0, 0) - if events & EVENT_WRITE: - kev = select.kevent(key.fd, select.KQ_FILTER_WRITE, - select.KQ_EV_ADD) - self._kqueue.control([kev], 0, 0) - return key - - def unregister(self, fileobj): - key = super(KqueueSelector, self).unregister(fileobj) - if key.events & EVENT_READ: - kev = select.kevent(key.fd, select.KQ_FILTER_READ, - select.KQ_EV_DELETE) - try: - self._kqueue.control([kev], 0, 0) - except OSError: - # This can happen if the FD was closed since it - # was registered. - pass - if key.events & EVENT_WRITE: - kev = select.kevent(key.fd, select.KQ_FILTER_WRITE, - select.KQ_EV_DELETE) - try: - self._kqueue.control([kev], 0, 0) - except OSError: - # See comment above. - pass - return key - - def select(self, timeout=None): - timeout = None if timeout is None else max(timeout, 0) - max_ev = len(self._fd_to_key) - ready = [] - try: - kev_list = self._kqueue.control(None, max_ev, timeout) - except OSError as exc: - if exc.errno == EINTR: - return ready - else: - raise - for kev in kev_list: - fd = kev.ident - flag = kev.filter - events = 0 - if flag == select.KQ_FILTER_READ: - events |= EVENT_READ - if flag == select.KQ_FILTER_WRITE: - events |= EVENT_WRITE - - key = self._key_from_fd(fd) - if key: - ready.append((key, events & key.events)) - return ready - - def close(self): - self._kqueue.close() - super(KqueueSelector, self).close() - - -# Choose the best implementation, roughly: -# epoll|kqueue|devpoll > poll > select. -# select() also can't accept a FD > FD_SETSIZE (usually around 1024) -if 'KqueueSelector' in globals(): - DefaultSelector = KqueueSelector -elif 'EpollSelector' in globals(): - DefaultSelector = EpollSelector -elif 'DevpollSelector' in globals(): - DefaultSelector = DevpollSelector -elif 'PollSelector' in globals(): - DefaultSelector = PollSelector -else: - DefaultSelector = SelectSelector diff --git a/kafka/vendor/six.py b/kafka/vendor/six.py deleted file mode 100644 index 319821353..000000000 --- a/kafka/vendor/six.py +++ /dev/null @@ -1,1004 +0,0 @@ -# pylint: skip-file - -# Copyright (c) 2010-2020 Benjamin Peterson -# -# Permission is hereby granted, free of charge, to any person obtaining a copy -# of this software and associated documentation files (the "Software"), to deal -# in the Software without restriction, including without limitation the rights -# to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -# copies of the Software, and to permit persons to whom the Software is -# furnished to do so, subject to the following conditions: -# -# The above copyright notice and this permission notice shall be included in all -# copies or substantial portions of the Software. -# -# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -# OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -# SOFTWARE. - -"""Utilities for writing code that runs on Python 2 and 3""" - -from __future__ import absolute_import - -import functools -import itertools -import operator -import sys -import types - -__author__ = "Benjamin Peterson " -__version__ = "1.16.0" - - -# Useful for very coarse version differentiation. -PY2 = sys.version_info[0] == 2 -PY3 = sys.version_info[0] == 3 -PY34 = sys.version_info[0:2] >= (3, 4) - -if PY3: - string_types = str, - integer_types = int, - class_types = type, - text_type = str - binary_type = bytes - - MAXSIZE = sys.maxsize -else: - string_types = basestring, - integer_types = (int, long) - class_types = (type, types.ClassType) - text_type = unicode - binary_type = str - - if sys.platform.startswith("java"): - # Jython always uses 32 bits. - MAXSIZE = int((1 << 31) - 1) - else: - # It's possible to have sizeof(long) != sizeof(Py_ssize_t). - class X(object): - - def __len__(self): - return 1 << 31 - try: - len(X()) - except OverflowError: - # 32-bit - MAXSIZE = int((1 << 31) - 1) - else: - # 64-bit - MAXSIZE = int((1 << 63) - 1) - - # Don't del it here, cause with gc disabled this "leaks" to garbage. - # Note: This is a kafka-python customization, details at: - # https://github.com/dpkp/kafka-python/pull/979#discussion_r100403389 - # del X - -if PY34: - from importlib.util import spec_from_loader -else: - spec_from_loader = None - - -def _add_doc(func, doc): - """Add documentation to a function.""" - func.__doc__ = doc - - -def _import_module(name): - """Import module, returning the module after the last dot.""" - __import__(name) - return sys.modules[name] - - -class _LazyDescr(object): - - def __init__(self, name): - self.name = name - - def __get__(self, obj, tp): - result = self._resolve() - setattr(obj, self.name, result) # Invokes __set__. - try: - # This is a bit ugly, but it avoids running this again by - # removing this descriptor. - delattr(obj.__class__, self.name) - except AttributeError: - pass - return result - - -class MovedModule(_LazyDescr): - - def __init__(self, name, old, new=None): - super(MovedModule, self).__init__(name) - if PY3: - if new is None: - new = name - self.mod = new - else: - self.mod = old - - def _resolve(self): - return _import_module(self.mod) - - def __getattr__(self, attr): - _module = self._resolve() - value = getattr(_module, attr) - setattr(self, attr, value) - return value - - -class _LazyModule(types.ModuleType): - - def __init__(self, name): - super(_LazyModule, self).__init__(name) - self.__doc__ = self.__class__.__doc__ - - def __dir__(self): - attrs = ["__doc__", "__name__"] - attrs += [attr.name for attr in self._moved_attributes] - return attrs - - # Subclasses should override this - _moved_attributes = [] - - -class MovedAttribute(_LazyDescr): - - def __init__(self, name, old_mod, new_mod, old_attr=None, new_attr=None): - super(MovedAttribute, self).__init__(name) - if PY3: - if new_mod is None: - new_mod = name - self.mod = new_mod - if new_attr is None: - if old_attr is None: - new_attr = name - else: - new_attr = old_attr - self.attr = new_attr - else: - self.mod = old_mod - if old_attr is None: - old_attr = name - self.attr = old_attr - - def _resolve(self): - module = _import_module(self.mod) - return getattr(module, self.attr) - - -class _SixMetaPathImporter(object): - - """ - A meta path importer to import six.moves and its submodules. - - This class implements a PEP302 finder and loader. It should be compatible - with Python 2.5 and all existing versions of Python3 - """ - - def __init__(self, six_module_name): - self.name = six_module_name - self.known_modules = {} - - def _add_module(self, mod, *fullnames): - for fullname in fullnames: - self.known_modules[self.name + "." + fullname] = mod - - def _get_module(self, fullname): - return self.known_modules[self.name + "." + fullname] - - def find_module(self, fullname, path=None): - if fullname in self.known_modules: - return self - return None - - def find_spec(self, fullname, path, target=None): - if fullname in self.known_modules: - return spec_from_loader(fullname, self) - return None - - def __get_module(self, fullname): - try: - return self.known_modules[fullname] - except KeyError: - raise ImportError("This loader does not know module " + fullname) - - def load_module(self, fullname): - try: - # in case of a reload - return sys.modules[fullname] - except KeyError: - pass - mod = self.__get_module(fullname) - if isinstance(mod, MovedModule): - mod = mod._resolve() - else: - mod.__loader__ = self - sys.modules[fullname] = mod - return mod - - def is_package(self, fullname): - """ - Return true, if the named module is a package. - - We need this method to get correct spec objects with - Python 3.4 (see PEP451) - """ - return hasattr(self.__get_module(fullname), "__path__") - - def get_code(self, fullname): - """Return None - - Required, if is_package is implemented""" - self.__get_module(fullname) # eventually raises ImportError - return None - get_source = get_code # same as get_code - - def create_module(self, spec): - return self.load_module(spec.name) - - def exec_module(self, module): - pass - -_importer = _SixMetaPathImporter(__name__) - - -class _MovedItems(_LazyModule): - - """Lazy loading of moved objects""" - __path__ = [] # mark as package - - -_moved_attributes = [ - MovedAttribute("cStringIO", "cStringIO", "io", "StringIO"), - MovedAttribute("filter", "itertools", "builtins", "ifilter", "filter"), - MovedAttribute("filterfalse", "itertools", "itertools", "ifilterfalse", "filterfalse"), - MovedAttribute("input", "__builtin__", "builtins", "raw_input", "input"), - MovedAttribute("intern", "__builtin__", "sys"), - MovedAttribute("map", "itertools", "builtins", "imap", "map"), - MovedAttribute("getcwd", "os", "os", "getcwdu", "getcwd"), - MovedAttribute("getcwdb", "os", "os", "getcwd", "getcwdb"), - MovedAttribute("getoutput", "commands", "subprocess"), - MovedAttribute("range", "__builtin__", "builtins", "xrange", "range"), - MovedAttribute("reload_module", "__builtin__", "importlib" if PY34 else "imp", "reload"), - MovedAttribute("reduce", "__builtin__", "functools"), - MovedAttribute("shlex_quote", "pipes", "shlex", "quote"), - MovedAttribute("StringIO", "StringIO", "io"), - MovedAttribute("UserDict", "UserDict", "collections", "IterableUserDict", "UserDict"), - MovedAttribute("UserList", "UserList", "collections"), - MovedAttribute("UserString", "UserString", "collections"), - MovedAttribute("xrange", "__builtin__", "builtins", "xrange", "range"), - MovedAttribute("zip", "itertools", "builtins", "izip", "zip"), - MovedAttribute("zip_longest", "itertools", "itertools", "izip_longest", "zip_longest"), - MovedModule("builtins", "__builtin__"), - MovedModule("configparser", "ConfigParser"), - MovedModule("collections_abc", "collections", "collections.abc" if sys.version_info >= (3, 3) else "collections"), - MovedModule("copyreg", "copy_reg"), - MovedModule("dbm_gnu", "gdbm", "dbm.gnu"), - MovedModule("dbm_ndbm", "dbm", "dbm.ndbm"), - MovedModule("_dummy_thread", "dummy_thread", "_dummy_thread" if sys.version_info < (3, 9) else "_thread"), - MovedModule("http_cookiejar", "cookielib", "http.cookiejar"), - MovedModule("http_cookies", "Cookie", "http.cookies"), - MovedModule("html_entities", "htmlentitydefs", "html.entities"), - MovedModule("html_parser", "HTMLParser", "html.parser"), - MovedModule("http_client", "httplib", "http.client"), - MovedModule("email_mime_base", "email.MIMEBase", "email.mime.base"), - MovedModule("email_mime_image", "email.MIMEImage", "email.mime.image"), - MovedModule("email_mime_multipart", "email.MIMEMultipart", "email.mime.multipart"), - MovedModule("email_mime_nonmultipart", "email.MIMENonMultipart", "email.mime.nonmultipart"), - MovedModule("email_mime_text", "email.MIMEText", "email.mime.text"), - MovedModule("BaseHTTPServer", "BaseHTTPServer", "http.server"), - MovedModule("CGIHTTPServer", "CGIHTTPServer", "http.server"), - MovedModule("SimpleHTTPServer", "SimpleHTTPServer", "http.server"), - MovedModule("cPickle", "cPickle", "pickle"), - MovedModule("queue", "Queue"), - MovedModule("reprlib", "repr"), - MovedModule("socketserver", "SocketServer"), - MovedModule("_thread", "thread", "_thread"), - MovedModule("tkinter", "Tkinter"), - MovedModule("tkinter_dialog", "Dialog", "tkinter.dialog"), - MovedModule("tkinter_filedialog", "FileDialog", "tkinter.filedialog"), - MovedModule("tkinter_scrolledtext", "ScrolledText", "tkinter.scrolledtext"), - MovedModule("tkinter_simpledialog", "SimpleDialog", "tkinter.simpledialog"), - MovedModule("tkinter_tix", "Tix", "tkinter.tix"), - MovedModule("tkinter_ttk", "ttk", "tkinter.ttk"), - MovedModule("tkinter_constants", "Tkconstants", "tkinter.constants"), - MovedModule("tkinter_dnd", "Tkdnd", "tkinter.dnd"), - MovedModule("tkinter_colorchooser", "tkColorChooser", - "tkinter.colorchooser"), - MovedModule("tkinter_commondialog", "tkCommonDialog", - "tkinter.commondialog"), - MovedModule("tkinter_tkfiledialog", "tkFileDialog", "tkinter.filedialog"), - MovedModule("tkinter_font", "tkFont", "tkinter.font"), - MovedModule("tkinter_messagebox", "tkMessageBox", "tkinter.messagebox"), - MovedModule("tkinter_tksimpledialog", "tkSimpleDialog", - "tkinter.simpledialog"), - MovedModule("urllib_parse", __name__ + ".moves.urllib_parse", "urllib.parse"), - MovedModule("urllib_error", __name__ + ".moves.urllib_error", "urllib.error"), - MovedModule("urllib", __name__ + ".moves.urllib", __name__ + ".moves.urllib"), - MovedModule("urllib_robotparser", "robotparser", "urllib.robotparser"), - MovedModule("xmlrpc_client", "xmlrpclib", "xmlrpc.client"), - MovedModule("xmlrpc_server", "SimpleXMLRPCServer", "xmlrpc.server"), -] -# Add windows specific modules. -if sys.platform == "win32": - _moved_attributes += [ - MovedModule("winreg", "_winreg"), - ] - -for attr in _moved_attributes: - setattr(_MovedItems, attr.name, attr) - if isinstance(attr, MovedModule): - _importer._add_module(attr, "moves." + attr.name) -del attr - -_MovedItems._moved_attributes = _moved_attributes - -moves = _MovedItems(__name__ + ".moves") -_importer._add_module(moves, "moves") - - -class Module_six_moves_urllib_parse(_LazyModule): - - """Lazy loading of moved objects in six.moves.urllib_parse""" - - -_urllib_parse_moved_attributes = [ - MovedAttribute("ParseResult", "urlparse", "urllib.parse"), - MovedAttribute("SplitResult", "urlparse", "urllib.parse"), - MovedAttribute("parse_qs", "urlparse", "urllib.parse"), - MovedAttribute("parse_qsl", "urlparse", "urllib.parse"), - MovedAttribute("urldefrag", "urlparse", "urllib.parse"), - MovedAttribute("urljoin", "urlparse", "urllib.parse"), - MovedAttribute("urlparse", "urlparse", "urllib.parse"), - MovedAttribute("urlsplit", "urlparse", "urllib.parse"), - MovedAttribute("urlunparse", "urlparse", "urllib.parse"), - MovedAttribute("urlunsplit", "urlparse", "urllib.parse"), - MovedAttribute("quote", "urllib", "urllib.parse"), - MovedAttribute("quote_plus", "urllib", "urllib.parse"), - MovedAttribute("unquote", "urllib", "urllib.parse"), - MovedAttribute("unquote_plus", "urllib", "urllib.parse"), - MovedAttribute("unquote_to_bytes", "urllib", "urllib.parse", "unquote", "unquote_to_bytes"), - MovedAttribute("urlencode", "urllib", "urllib.parse"), - MovedAttribute("splitquery", "urllib", "urllib.parse"), - MovedAttribute("splittag", "urllib", "urllib.parse"), - MovedAttribute("splituser", "urllib", "urllib.parse"), - MovedAttribute("splitvalue", "urllib", "urllib.parse"), - MovedAttribute("uses_fragment", "urlparse", "urllib.parse"), - MovedAttribute("uses_netloc", "urlparse", "urllib.parse"), - MovedAttribute("uses_params", "urlparse", "urllib.parse"), - MovedAttribute("uses_query", "urlparse", "urllib.parse"), - MovedAttribute("uses_relative", "urlparse", "urllib.parse"), -] -for attr in _urllib_parse_moved_attributes: - setattr(Module_six_moves_urllib_parse, attr.name, attr) -del attr - -Module_six_moves_urllib_parse._moved_attributes = _urllib_parse_moved_attributes - -_importer._add_module(Module_six_moves_urllib_parse(__name__ + ".moves.urllib_parse"), - "moves.urllib_parse", "moves.urllib.parse") - - -class Module_six_moves_urllib_error(_LazyModule): - - """Lazy loading of moved objects in six.moves.urllib_error""" - - -_urllib_error_moved_attributes = [ - MovedAttribute("URLError", "urllib2", "urllib.error"), - MovedAttribute("HTTPError", "urllib2", "urllib.error"), - MovedAttribute("ContentTooShortError", "urllib", "urllib.error"), -] -for attr in _urllib_error_moved_attributes: - setattr(Module_six_moves_urllib_error, attr.name, attr) -del attr - -Module_six_moves_urllib_error._moved_attributes = _urllib_error_moved_attributes - -_importer._add_module(Module_six_moves_urllib_error(__name__ + ".moves.urllib.error"), - "moves.urllib_error", "moves.urllib.error") - - -class Module_six_moves_urllib_request(_LazyModule): - - """Lazy loading of moved objects in six.moves.urllib_request""" - - -_urllib_request_moved_attributes = [ - MovedAttribute("urlopen", "urllib2", "urllib.request"), - MovedAttribute("install_opener", "urllib2", "urllib.request"), - MovedAttribute("build_opener", "urllib2", "urllib.request"), - MovedAttribute("pathname2url", "urllib", "urllib.request"), - MovedAttribute("url2pathname", "urllib", "urllib.request"), - MovedAttribute("getproxies", "urllib", "urllib.request"), - MovedAttribute("Request", "urllib2", "urllib.request"), - MovedAttribute("OpenerDirector", "urllib2", "urllib.request"), - MovedAttribute("HTTPDefaultErrorHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPRedirectHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPCookieProcessor", "urllib2", "urllib.request"), - MovedAttribute("ProxyHandler", "urllib2", "urllib.request"), - MovedAttribute("BaseHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPPasswordMgr", "urllib2", "urllib.request"), - MovedAttribute("HTTPPasswordMgrWithDefaultRealm", "urllib2", "urllib.request"), - MovedAttribute("AbstractBasicAuthHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPBasicAuthHandler", "urllib2", "urllib.request"), - MovedAttribute("ProxyBasicAuthHandler", "urllib2", "urllib.request"), - MovedAttribute("AbstractDigestAuthHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPDigestAuthHandler", "urllib2", "urllib.request"), - MovedAttribute("ProxyDigestAuthHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPSHandler", "urllib2", "urllib.request"), - MovedAttribute("FileHandler", "urllib2", "urllib.request"), - MovedAttribute("FTPHandler", "urllib2", "urllib.request"), - MovedAttribute("CacheFTPHandler", "urllib2", "urllib.request"), - MovedAttribute("UnknownHandler", "urllib2", "urllib.request"), - MovedAttribute("HTTPErrorProcessor", "urllib2", "urllib.request"), - MovedAttribute("urlretrieve", "urllib", "urllib.request"), - MovedAttribute("urlcleanup", "urllib", "urllib.request"), - MovedAttribute("URLopener", "urllib", "urllib.request"), - MovedAttribute("FancyURLopener", "urllib", "urllib.request"), - MovedAttribute("proxy_bypass", "urllib", "urllib.request"), - MovedAttribute("parse_http_list", "urllib2", "urllib.request"), - MovedAttribute("parse_keqv_list", "urllib2", "urllib.request"), -] -for attr in _urllib_request_moved_attributes: - setattr(Module_six_moves_urllib_request, attr.name, attr) -del attr - -Module_six_moves_urllib_request._moved_attributes = _urllib_request_moved_attributes - -_importer._add_module(Module_six_moves_urllib_request(__name__ + ".moves.urllib.request"), - "moves.urllib_request", "moves.urllib.request") - - -class Module_six_moves_urllib_response(_LazyModule): - - """Lazy loading of moved objects in six.moves.urllib_response""" - - -_urllib_response_moved_attributes = [ - MovedAttribute("addbase", "urllib", "urllib.response"), - MovedAttribute("addclosehook", "urllib", "urllib.response"), - MovedAttribute("addinfo", "urllib", "urllib.response"), - MovedAttribute("addinfourl", "urllib", "urllib.response"), -] -for attr in _urllib_response_moved_attributes: - setattr(Module_six_moves_urllib_response, attr.name, attr) -del attr - -Module_six_moves_urllib_response._moved_attributes = _urllib_response_moved_attributes - -_importer._add_module(Module_six_moves_urllib_response(__name__ + ".moves.urllib.response"), - "moves.urllib_response", "moves.urllib.response") - - -class Module_six_moves_urllib_robotparser(_LazyModule): - - """Lazy loading of moved objects in six.moves.urllib_robotparser""" - - -_urllib_robotparser_moved_attributes = [ - MovedAttribute("RobotFileParser", "robotparser", "urllib.robotparser"), -] -for attr in _urllib_robotparser_moved_attributes: - setattr(Module_six_moves_urllib_robotparser, attr.name, attr) -del attr - -Module_six_moves_urllib_robotparser._moved_attributes = _urllib_robotparser_moved_attributes - -_importer._add_module(Module_six_moves_urllib_robotparser(__name__ + ".moves.urllib.robotparser"), - "moves.urllib_robotparser", "moves.urllib.robotparser") - - -class Module_six_moves_urllib(types.ModuleType): - - """Create a six.moves.urllib namespace that resembles the Python 3 namespace""" - __path__ = [] # mark as package - parse = _importer._get_module("moves.urllib_parse") - error = _importer._get_module("moves.urllib_error") - request = _importer._get_module("moves.urllib_request") - response = _importer._get_module("moves.urllib_response") - robotparser = _importer._get_module("moves.urllib_robotparser") - - def __dir__(self): - return ['parse', 'error', 'request', 'response', 'robotparser'] - -_importer._add_module(Module_six_moves_urllib(__name__ + ".moves.urllib"), - "moves.urllib") - - -def add_move(move): - """Add an item to six.moves.""" - setattr(_MovedItems, move.name, move) - - -def remove_move(name): - """Remove item from six.moves.""" - try: - delattr(_MovedItems, name) - except AttributeError: - try: - del moves.__dict__[name] - except KeyError: - raise AttributeError("no such move, %r" % (name,)) - - -if PY3: - _meth_func = "__func__" - _meth_self = "__self__" - - _func_closure = "__closure__" - _func_code = "__code__" - _func_defaults = "__defaults__" - _func_globals = "__globals__" -else: - _meth_func = "im_func" - _meth_self = "im_self" - - _func_closure = "func_closure" - _func_code = "func_code" - _func_defaults = "func_defaults" - _func_globals = "func_globals" - - -try: - advance_iterator = next -except NameError: - def advance_iterator(it): - return it.next() -next = advance_iterator - - -try: - callable = callable -except NameError: - def callable(obj): - return any("__call__" in klass.__dict__ for klass in type(obj).__mro__) - - -if PY3: - def get_unbound_function(unbound): - return unbound - - create_bound_method = types.MethodType - - def create_unbound_method(func, cls): - return func - - Iterator = object -else: - def get_unbound_function(unbound): - return unbound.im_func - - def create_bound_method(func, obj): - return types.MethodType(func, obj, obj.__class__) - - def create_unbound_method(func, cls): - return types.MethodType(func, None, cls) - - class Iterator(object): - - def next(self): - return type(self).__next__(self) - - callable = callable -_add_doc(get_unbound_function, - """Get the function out of a possibly unbound function""") - - -get_method_function = operator.attrgetter(_meth_func) -get_method_self = operator.attrgetter(_meth_self) -get_function_closure = operator.attrgetter(_func_closure) -get_function_code = operator.attrgetter(_func_code) -get_function_defaults = operator.attrgetter(_func_defaults) -get_function_globals = operator.attrgetter(_func_globals) - - -if PY3: - def iterkeys(d, **kw): - return iter(d.keys(**kw)) - - def itervalues(d, **kw): - return iter(d.values(**kw)) - - def iteritems(d, **kw): - return iter(d.items(**kw)) - - def iterlists(d, **kw): - return iter(d.lists(**kw)) - - viewkeys = operator.methodcaller("keys") - - viewvalues = operator.methodcaller("values") - - viewitems = operator.methodcaller("items") -else: - def iterkeys(d, **kw): - return d.iterkeys(**kw) - - def itervalues(d, **kw): - return d.itervalues(**kw) - - def iteritems(d, **kw): - return d.iteritems(**kw) - - def iterlists(d, **kw): - return d.iterlists(**kw) - - viewkeys = operator.methodcaller("viewkeys") - - viewvalues = operator.methodcaller("viewvalues") - - viewitems = operator.methodcaller("viewitems") - -_add_doc(iterkeys, "Return an iterator over the keys of a dictionary.") -_add_doc(itervalues, "Return an iterator over the values of a dictionary.") -_add_doc(iteritems, - "Return an iterator over the (key, value) pairs of a dictionary.") -_add_doc(iterlists, - "Return an iterator over the (key, [values]) pairs of a dictionary.") - - -if PY3: - def b(s): - return s.encode("latin-1") - - def u(s): - return s - unichr = chr - import struct - int2byte = struct.Struct(">B").pack - del struct - byte2int = operator.itemgetter(0) - indexbytes = operator.getitem - iterbytes = iter - import io - StringIO = io.StringIO - BytesIO = io.BytesIO - del io - _assertCountEqual = "assertCountEqual" - if sys.version_info[1] <= 1: - _assertRaisesRegex = "assertRaisesRegexp" - _assertRegex = "assertRegexpMatches" - _assertNotRegex = "assertNotRegexpMatches" - else: - _assertRaisesRegex = "assertRaisesRegex" - _assertRegex = "assertRegex" - _assertNotRegex = "assertNotRegex" -else: - def b(s): - return s - # Workaround for standalone backslash - - def u(s): - return unicode(s.replace(r'\\', r'\\\\'), "unicode_escape") - unichr = unichr - int2byte = chr - - def byte2int(bs): - return ord(bs[0]) - - def indexbytes(buf, i): - return ord(buf[i]) - iterbytes = functools.partial(itertools.imap, ord) - import StringIO - StringIO = BytesIO = StringIO.StringIO - _assertCountEqual = "assertItemsEqual" - _assertRaisesRegex = "assertRaisesRegexp" - _assertRegex = "assertRegexpMatches" - _assertNotRegex = "assertNotRegexpMatches" -_add_doc(b, """Byte literal""") -_add_doc(u, """Text literal""") - - -def assertCountEqual(self, *args, **kwargs): - return getattr(self, _assertCountEqual)(*args, **kwargs) - - -def assertRaisesRegex(self, *args, **kwargs): - return getattr(self, _assertRaisesRegex)(*args, **kwargs) - - -def assertRegex(self, *args, **kwargs): - return getattr(self, _assertRegex)(*args, **kwargs) - - -def assertNotRegex(self, *args, **kwargs): - return getattr(self, _assertNotRegex)(*args, **kwargs) - - -if PY3: - exec_ = getattr(moves.builtins, "exec") - - def reraise(tp, value, tb=None): - try: - if value is None: - value = tp() - if value.__traceback__ is not tb: - raise value.with_traceback(tb) - raise value - finally: - value = None - tb = None - -else: - def exec_(_code_, _globs_=None, _locs_=None): - """Execute code in a namespace.""" - if _globs_ is None: - frame = sys._getframe(1) - _globs_ = frame.f_globals - if _locs_ is None: - _locs_ = frame.f_locals - del frame - elif _locs_ is None: - _locs_ = _globs_ - exec("""exec _code_ in _globs_, _locs_""") - - exec_("""def reraise(tp, value, tb=None): - try: - raise tp, value, tb - finally: - tb = None -""") - - -if sys.version_info[:2] > (3,): - exec_("""def raise_from(value, from_value): - try: - raise value from from_value - finally: - value = None -""") -else: - def raise_from(value, from_value): - raise value - - -print_ = getattr(moves.builtins, "print", None) -if print_ is None: - def print_(*args, **kwargs): - """The new-style print function for Python 2.4 and 2.5.""" - fp = kwargs.pop("file", sys.stdout) - if fp is None: - return - - def write(data): - if not isinstance(data, basestring): - data = str(data) - # If the file has an encoding, encode unicode with it. - if (isinstance(fp, file) and - isinstance(data, unicode) and - fp.encoding is not None): - errors = getattr(fp, "errors", None) - if errors is None: - errors = "strict" - data = data.encode(fp.encoding, errors) - fp.write(data) - want_unicode = False - sep = kwargs.pop("sep", None) - if sep is not None: - if isinstance(sep, unicode): - want_unicode = True - elif not isinstance(sep, str): - raise TypeError("sep must be None or a string") - end = kwargs.pop("end", None) - if end is not None: - if isinstance(end, unicode): - want_unicode = True - elif not isinstance(end, str): - raise TypeError("end must be None or a string") - if kwargs: - raise TypeError("invalid keyword arguments to print()") - if not want_unicode: - for arg in args: - if isinstance(arg, unicode): - want_unicode = True - break - if want_unicode: - newline = unicode("\n") - space = unicode(" ") - else: - newline = "\n" - space = " " - if sep is None: - sep = space - if end is None: - end = newline - for i, arg in enumerate(args): - if i: - write(sep) - write(arg) - write(end) -if sys.version_info[:2] < (3, 3): - _print = print_ - - def print_(*args, **kwargs): - fp = kwargs.get("file", sys.stdout) - flush = kwargs.pop("flush", False) - _print(*args, **kwargs) - if flush and fp is not None: - fp.flush() - -_add_doc(reraise, """Reraise an exception.""") - -if sys.version_info[0:2] < (3, 4): - # This does exactly the same what the :func:`py3:functools.update_wrapper` - # function does on Python versions after 3.2. It sets the ``__wrapped__`` - # attribute on ``wrapper`` object and it doesn't raise an error if any of - # the attributes mentioned in ``assigned`` and ``updated`` are missing on - # ``wrapped`` object. - def _update_wrapper(wrapper, wrapped, - assigned=functools.WRAPPER_ASSIGNMENTS, - updated=functools.WRAPPER_UPDATES): - for attr in assigned: - try: - value = getattr(wrapped, attr) - except AttributeError: - continue - else: - setattr(wrapper, attr, value) - for attr in updated: - getattr(wrapper, attr).update(getattr(wrapped, attr, {})) - wrapper.__wrapped__ = wrapped - return wrapper - _update_wrapper.__doc__ = functools.update_wrapper.__doc__ - - def wraps(wrapped, assigned=functools.WRAPPER_ASSIGNMENTS, - updated=functools.WRAPPER_UPDATES): - return functools.partial(_update_wrapper, wrapped=wrapped, - assigned=assigned, updated=updated) - wraps.__doc__ = functools.wraps.__doc__ - -else: - wraps = functools.wraps - - -def with_metaclass(meta, *bases): - """Create a base class with a metaclass.""" - # This requires a bit of explanation: the basic idea is to make a dummy - # metaclass for one level of class instantiation that replaces itself with - # the actual metaclass. - class metaclass(type): - - def __new__(cls, name, this_bases, d): - if sys.version_info[:2] >= (3, 7): - # This version introduced PEP 560 that requires a bit - # of extra care (we mimic what is done by __build_class__). - resolved_bases = types.resolve_bases(bases) - if resolved_bases is not bases: - d['__orig_bases__'] = bases - else: - resolved_bases = bases - return meta(name, resolved_bases, d) - - @classmethod - def __prepare__(cls, name, this_bases): - return meta.__prepare__(name, bases) - return type.__new__(metaclass, 'temporary_class', (), {}) - - -def add_metaclass(metaclass): - """Class decorator for creating a class with a metaclass.""" - def wrapper(cls): - orig_vars = cls.__dict__.copy() - slots = orig_vars.get('__slots__') - if slots is not None: - if isinstance(slots, str): - slots = [slots] - for slots_var in slots: - orig_vars.pop(slots_var) - orig_vars.pop('__dict__', None) - orig_vars.pop('__weakref__', None) - if hasattr(cls, '__qualname__'): - orig_vars['__qualname__'] = cls.__qualname__ - return metaclass(cls.__name__, cls.__bases__, orig_vars) - return wrapper - - -def ensure_binary(s, encoding='utf-8', errors='strict'): - """Coerce **s** to six.binary_type. - - For Python 2: - - `unicode` -> encoded to `str` - - `str` -> `str` - - For Python 3: - - `str` -> encoded to `bytes` - - `bytes` -> `bytes` - """ - if isinstance(s, binary_type): - return s - if isinstance(s, text_type): - return s.encode(encoding, errors) - raise TypeError("not expecting type '%s'" % type(s)) - - -def ensure_str(s, encoding='utf-8', errors='strict'): - """Coerce *s* to `str`. - - For Python 2: - - `unicode` -> encoded to `str` - - `str` -> `str` - - For Python 3: - - `str` -> `str` - - `bytes` -> decoded to `str` - """ - # Optimization: Fast return for the common case. - if type(s) is str: - return s - if PY2 and isinstance(s, text_type): - return s.encode(encoding, errors) - elif PY3 and isinstance(s, binary_type): - return s.decode(encoding, errors) - elif not isinstance(s, (text_type, binary_type)): - raise TypeError("not expecting type '%s'" % type(s)) - return s - - -def ensure_text(s, encoding='utf-8', errors='strict'): - """Coerce *s* to six.text_type. - - For Python 2: - - `unicode` -> `unicode` - - `str` -> `unicode` - - For Python 3: - - `str` -> `str` - - `bytes` -> decoded to `str` - """ - if isinstance(s, binary_type): - return s.decode(encoding, errors) - elif isinstance(s, text_type): - return s - else: - raise TypeError("not expecting type '%s'" % type(s)) - - -def python_2_unicode_compatible(klass): - """ - A class decorator that defines __unicode__ and __str__ methods under Python 2. - Under Python 3 it does nothing. - - To support Python 2 and 3 with a single code base, define a __str__ method - returning text and apply this decorator to the class. - """ - if PY2: - if '__str__' not in klass.__dict__: - raise ValueError("@python_2_unicode_compatible cannot be applied " - "to %s because it doesn't define __str__()." % - klass.__name__) - klass.__unicode__ = klass.__str__ - klass.__str__ = lambda self: self.__unicode__().encode('utf-8') - return klass - - -# Complete the moves implementation. -# This code is at the end of this module to speed up module loading. -# Turn this module into a package. -__path__ = [] # required for PEP 302 and PEP 451 -__package__ = __name__ # see PEP 366 @ReservedAssignment -if globals().get("__spec__") is not None: - __spec__.submodule_search_locations = [] # PEP 451 @UndefinedVariable -# Remove other six meta path importers, since they cause problems. This can -# happen if six is removed from sys.modules and then reloaded. (Setuptools does -# this for some reason.) -if sys.meta_path: - for i, importer in enumerate(sys.meta_path): - # Here's some real nastiness: Another "instance" of the six module might - # be floating around. Therefore, we can't use isinstance() to check for - # the six meta path importer, since the other six instance will have - # inserted an importer with different class. - if (type(importer).__name__ == "_SixMetaPathImporter" and - importer.name == __name__): - del sys.meta_path[i] - break - del i, importer -# Finally, add the importer to the meta path import hook. -sys.meta_path.append(_importer) diff --git a/kafka/vendor/socketpair.py b/kafka/vendor/socketpair.py deleted file mode 100644 index 54d908767..000000000 --- a/kafka/vendor/socketpair.py +++ /dev/null @@ -1,75 +0,0 @@ -# pylint: skip-file -# vendored from https://github.com/mhils/backports.socketpair -from __future__ import absolute_import - -import sys -import socket -import errno - -_LOCALHOST = '127.0.0.1' -_LOCALHOST_V6 = '::1' - -if not hasattr(socket, "socketpair"): - # Origin: https://gist.github.com/4325783, by Geert Jansen. Public domain. - def socketpair(family=socket.AF_INET, type=socket.SOCK_STREAM, proto=0): - if family == socket.AF_INET: - host = _LOCALHOST - elif family == socket.AF_INET6: - host = _LOCALHOST_V6 - else: - raise ValueError("Only AF_INET and AF_INET6 socket address families " - "are supported") - if type != socket.SOCK_STREAM: - raise ValueError("Only SOCK_STREAM socket type is supported") - if proto != 0: - raise ValueError("Only protocol zero is supported") - - # We create a connected TCP socket. Note the trick with - # setblocking(False) that prevents us from having to create a thread. - lsock = socket.socket(family, type, proto) - try: - lsock.bind((host, 0)) - lsock.listen(min(socket.SOMAXCONN, 128)) - # On IPv6, ignore flow_info and scope_id - addr, port = lsock.getsockname()[:2] - csock = socket.socket(family, type, proto) - try: - csock.setblocking(False) - if sys.version_info >= (3, 0): - try: - csock.connect((addr, port)) - except (BlockingIOError, InterruptedError): - pass - else: - try: - csock.connect((addr, port)) - except socket.error as e: - if e.errno != errno.WSAEWOULDBLOCK: - raise - csock.setblocking(True) - ssock, _ = lsock.accept() - except Exception: - csock.close() - raise - finally: - lsock.close() - - # Authenticating avoids using a connection from something else - # able to connect to {host}:{port} instead of us. - # We expect only AF_INET and AF_INET6 families. - try: - if ( - ssock.getsockname() != csock.getpeername() - or csock.getsockname() != ssock.getpeername() - ): - raise ConnectionError("Unexpected peer connection") - except: - # getsockname() and getpeername() can fail - # if either socket isn't connected. - ssock.close() - csock.close() - raise - - return (ssock, csock) - - socket.socketpair = socketpair diff --git a/pylint.rc b/pylint.rc index 851275bcc..12be07f50 100644 --- a/pylint.rc +++ b/pylint.rc @@ -1,6 +1,6 @@ [TYPECHECK] ignored-classes=SyncManager,_socketobject -ignored-modules=kafka.vendor.six.moves +ignored-modules= generated-members=py.* [MESSAGES CONTROL] diff --git a/pyproject.toml b/pyproject.toml index d575a8959..312c45656 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -14,11 +14,7 @@ classifiers = [ "Intended Audience :: Developers", "License :: OSI Approved :: Apache Software License", "Programming Language :: Python", - "Programming Language :: Python :: 2", - "Programming Language :: Python :: 2.7", "Programming Language :: Python :: 3", - "Programming Language :: Python :: 3.4", - "Programming Language :: Python :: 3.5", "Programming Language :: Python :: 3.6", "Programming Language :: Python :: 3.7", "Programming Language :: Python :: 3.8", @@ -27,6 +23,7 @@ classifiers = [ "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", "Programming Language :: Python :: 3.13", + "Programming Language :: Python :: 3.14", "Programming Language :: Python :: Implementation :: CPython", "Programming Language :: Python :: Implementation :: PyPy", "Topic :: Software Development :: Libraries :: Python Modules", @@ -38,7 +35,7 @@ crc32c = ["crc32c"] lz4 = ["lz4"] snappy = ["python-snappy"] zstd = ["zstandard"] -testing = ["pytest", "mock; python_version < '3.3'", "pytest-mock", "pytest-timeout"] +testing = ["pytest", "pytest-mock", "pytest-timeout"] benchmarks = ["pyperf"] [tool.setuptools] diff --git a/requirements-dev.txt b/requirements-dev.txt index 8de5e28d4..840bdb7f3 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -3,7 +3,6 @@ crc32c docker-py flake8 lz4 -mock; python_version < '3.3' py pylint pyperf diff --git a/test/__init__.py b/test/__init__.py index 329277dc6..7a0230c28 100644 --- a/test/__init__.py +++ b/test/__init__.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - # Set default logging handler to avoid "No handler found" warnings. import logging logging.basicConfig(level=logging.INFO) diff --git a/test/conftest.py b/test/conftest.py index b65593a86..263fc0f3c 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import pytest diff --git a/test/integration/conftest.py b/test/integration/conftest.py index 8af729296..e4307d01c 100644 --- a/test/integration/conftest.py +++ b/test/integration/conftest.py @@ -1,11 +1,9 @@ -from __future__ import absolute_import - import os +from urllib.parse import urlparse import uuid import pytest -from kafka.vendor.six.moves.urllib.parse import urlparse # pylint: disable=E0611,F0401 from test.testutil import env_kafka_version, random_string from test.integration.fixtures import KafkaFixture, ZookeeperFixture diff --git a/test/integration/fixtures.py b/test/integration/fixtures.py index b9baf5223..0b660e624 100644 --- a/test/integration/fixtures.py +++ b/test/integration/fixtures.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import, division - import atexit import base64 import logging @@ -8,11 +6,10 @@ import socket import subprocess import time +from urllib.parse import urlparse import uuid import py -from kafka.vendor.six.moves import range -from kafka.vendor.six.moves.urllib.parse import urlparse # pylint: disable=E0611,F0401 from kafka import errors, KafkaAdminClient, KafkaClient, KafkaConsumer, KafkaProducer from kafka.errors import InvalidReplicationFactorError, KafkaTimeoutError diff --git a/test/integration/test_consumer_group.py b/test/integration/test_consumer_group.py index eed570074..c0b4857a1 100644 --- a/test/integration/test_consumer_group.py +++ b/test/integration/test_consumer_group.py @@ -4,7 +4,6 @@ import time import pytest -from kafka.vendor import six from kafka.conn import ConnectionStates from kafka.consumer.group import KafkaConsumer @@ -60,7 +59,7 @@ def consumer_thread(i): api_version_auto_timeout_ms=5000, heartbeat_interval_ms=500) while not stop[i].is_set(): - for tp, records in six.iteritems(consumers[i].poll(timeout_ms=200)): + for tp, records in consumers[i].poll(timeout_ms=200).items(): messages[i][tp].extend(records) consumers[i].close(timeout_ms=500) consumers[i] = None @@ -84,7 +83,7 @@ def consumer_thread(i): time.sleep(1) continue - unassigned_consumers = {c for c, consumer in six.iteritems(consumers) if not consumer.assignment()} + unassigned_consumers = {c for c, consumer in consumers.items() if not consumer.assignment()} if unassigned_consumers: logging.info('Waiting for consumer assignments: %s', unassigned_consumers) time.sleep(1) @@ -95,14 +94,14 @@ def consumer_thread(i): # Verify all consumers are in the same generation # then log state and break while loop generations = set([consumer._coordinator._generation.generation_id - for consumer in six.itervalues(consumers)]) + for consumer in consumers.values()]) # New generation assignment is not complete until # coordinator.rejoining = False - rejoining = set([c for c, consumer in six.iteritems(consumers) if consumer._coordinator.rejoining]) + rejoining = set([c for c, consumer in consumers.items() if consumer._coordinator.rejoining]) if not rejoining and len(generations) == 1: - for c, consumer in six.iteritems(consumers): + for c, consumer in consumers.items(): logging.info("[%s] %s %s: %s", c, consumer._coordinator._generation.generation_id, consumer._coordinator._generation.member_id, diff --git a/test/integration/test_consumer_integration.py b/test/integration/test_consumer_integration.py index 9f103c237..6c37ab204 100644 --- a/test/integration/test_consumer_integration.py +++ b/test/integration/test_consumer_integration.py @@ -1,12 +1,8 @@ import logging import time +from unittest.mock import patch, ANY -try: - from unittest.mock import patch, ANY -except ImportError: - from mock import patch, ANY import pytest -from kafka.vendor.six.moves import range import kafka.codec from kafka.errors import KafkaTimeoutError, UnsupportedCodecError, UnsupportedVersionError diff --git a/test/integration/test_producer_integration.py b/test/integration/test_producer_integration.py index 037a82834..c7e6ad4c9 100644 --- a/test/integration/test_producer_integration.py +++ b/test/integration/test_producer_integration.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - from contextlib import contextmanager import platform import time diff --git a/test/record/test_default_records.py b/test/record/test_default_records.py index 540705d50..17f0de5ba 100644 --- a/test/record/test_default_records.py +++ b/test/record/test_default_records.py @@ -1,10 +1,9 @@ # -*- coding: utf-8 -*- -from __future__ import unicode_literals + +from unittest.mock import patch + import pytest -try: - from unittest.mock import patch -except ImportError: - from mock import patch + import kafka.codec from kafka.record.default_records import ( DefaultRecordBatch, DefaultRecordBatchBuilder diff --git a/test/record/test_legacy_records.py b/test/record/test_legacy_records.py index c692d35a1..454a3360b 100644 --- a/test/record/test_legacy_records.py +++ b/test/record/test_legacy_records.py @@ -1,9 +1,7 @@ -from __future__ import unicode_literals +from unittest.mock import patch + import pytest -try: - from unittest.mock import patch -except ImportError: - from mock import patch + from kafka.record.legacy_records import ( LegacyRecordBatch, LegacyRecordBatchBuilder ) diff --git a/test/record/test_records.py b/test/record/test_records.py index 65010d88f..81874d145 100644 --- a/test/record/test_records.py +++ b/test/record/test_records.py @@ -1,5 +1,4 @@ # -*- coding: utf-8 -*- -from __future__ import unicode_literals import pytest from kafka.record import MemoryRecords, MemoryRecordsBuilder from kafka.errors import CorruptRecordError diff --git a/test/sasl/test_gssapi.py b/test/sasl/test_gssapi.py index aa1d86b9e..cdebf2812 100644 --- a/test/sasl/test_gssapi.py +++ b/test/sasl/test_gssapi.py @@ -1,9 +1,4 @@ -from __future__ import absolute_import - -try: - from unittest import mock -except ImportError: - import mock +from unittest import mock from kafka.sasl import get_sasl_mechanism import kafka.sasl.gssapi diff --git a/test/sasl/test_msk.py b/test/sasl/test_msk.py index f3cc46ce8..ec51eb4c6 100644 --- a/test/sasl/test_msk.py +++ b/test/sasl/test_msk.py @@ -1,14 +1,10 @@ import datetime import json import sys +from unittest import mock from kafka.sasl.msk import AwsMskIamClient, SaslMechanismAwsMskIam -try: - from unittest import mock -except ImportError: - import mock - def client_factory(token=None): if sys.version_info >= (3, 3): diff --git a/test/service.py b/test/service.py index a53fab8da..408351544 100644 --- a/test/service.py +++ b/test/service.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import logging import os import re diff --git a/test/test_assignors.py b/test/test_assignors.py index f2ecc0325..1056a7c1a 100644 --- a/test/test_assignors.py +++ b/test/test_assignors.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import from collections import defaultdict from random import randint, sample @@ -12,7 +11,6 @@ from kafka.coordinator.assignors.sticky.sticky_assignor import StickyPartitionAssignor from kafka.coordinator.protocol import ConsumerProtocolMemberAssignment_v0 from kafka.coordinator.subscription import Subscription -from kafka.vendor import six @pytest.fixture(autouse=True) @@ -111,7 +109,7 @@ def test_sticky_assignor1(mocker): del subscriptions['C1'] member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, sticky_assignment[member].partitions()) sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -154,7 +152,7 @@ def test_sticky_assignor2(mocker): 'C2': {'t0', 't1', 't2'}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, []) sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -167,7 +165,7 @@ def test_sticky_assignor2(mocker): del subscriptions['C0'] member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, sticky_assignment[member].partitions()) sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -326,7 +324,7 @@ def test_sticky_add_remove_consumer_one_topic(mocker): 'C2': {'t'}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata( topics, assignment[member].partitions() if member in assignment else [] ) @@ -338,7 +336,7 @@ def test_sticky_add_remove_consumer_one_topic(mocker): 'C2': {'t'}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, assignment[member].partitions()) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -367,7 +365,7 @@ def test_sticky_add_remove_topic_two_consumers(mocker): 'C2': {'t1', 't2'}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, sticky_assignment[member].partitions()) sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -382,7 +380,7 @@ def test_sticky_add_remove_topic_two_consumers(mocker): 'C2': {'t2'}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, sticky_assignment[member].partitions()) sticky_assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -413,7 +411,7 @@ def test_sticky_reassignment_after_one_consumer_leaves(mocker): del subscriptions['C10'] member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, assignment[member].partitions()) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -435,7 +433,7 @@ def test_sticky_reassignment_after_one_consumer_added(mocker): subscriptions['C10'] = {'t'} member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata( topics, assignment[member].partitions() if member in assignment else [] ) @@ -452,7 +450,7 @@ def test_sticky_same_subscriptions(mocker): subscriptions = defaultdict(set) for i in range(1, 9): - for j in range(1, len(six.viewkeys(partitions)) + 1): + for j in range(1, len(partitions) + 1): subscriptions['C{}'.format(i)].add('t{}'.format(j)) member_metadata = make_member_metadata(subscriptions) @@ -462,7 +460,7 @@ def test_sticky_same_subscriptions(mocker): del subscriptions['C5'] member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, assignment[member].partitions()) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) verify_validity_and_balance(subscriptions, assignment) @@ -488,7 +486,7 @@ def test_sticky_large_assignment_with_multiple_consumers_leaving(mocker): verify_validity_and_balance(subscriptions, assignment) member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, assignment[member].partitions()) for i in range(50): @@ -517,7 +515,7 @@ def test_new_subscription(mocker): subscriptions['C0'].add('t1') member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, []) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -540,7 +538,7 @@ def test_move_existing_assignments(mocker): } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, member_assignments[member]) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -560,7 +558,7 @@ def test_stickiness(mocker): assignment = StickyPartitionAssignor.assign(cluster, member_metadata) verify_validity_and_balance(subscriptions, assignment) partitions_assigned = {} - for consumer, consumer_assignment in six.iteritems(assignment): + for consumer, consumer_assignment in assignment.items(): assert ( len(consumer_assignment.partitions()) <= 1 ), 'Consumer {} is assigned more topic partitions than expected.'.format(consumer) @@ -570,14 +568,14 @@ def test_stickiness(mocker): # removing the potential group leader del subscriptions['C1'] member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, assignment[member].partitions()) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) verify_validity_and_balance(subscriptions, assignment) assert StickyPartitionAssignor._latest_partition_movements.are_sticky() - for consumer, consumer_assignment in six.iteritems(assignment): + for consumer, consumer_assignment in assignment.items(): assert ( len(consumer_assignment.partitions()) <= 1 ), 'Consumer {} is assigned more topic partitions than expected.'.format(consumer) @@ -625,7 +623,7 @@ def test_no_exceptions_when_only_subscribed_topic_is_deleted(mocker): 'C': {}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, sticky_assignment[member].partitions()) cluster = create_cluster(mocker, topics={}, topics_partitions={}) @@ -644,7 +642,7 @@ def test_conflicting_previous_assignments(mocker): 'C2': {'t'}, } member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): # assume both C1 and C2 have partition 1 assigned to them in generation 1 member_metadata[member] = StickyPartitionAssignor._metadata(topics, [TopicPartition('t', 0), TopicPartition('t', 0)], 1) @@ -676,7 +674,7 @@ def test_reassignment_with_random_subscriptions_and_changes(mocker, execution_nu subscriptions['C{}'.format(i)].update(topics_sample) member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, assignment[member].partitions()) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -778,7 +776,7 @@ def test_assignment_with_conflicting_previous_generations(mocker, execution_numb 'C3': 2, } member_metadata = {} - for member in six.iterkeys(member_assignments): + for member in member_assignments: member_metadata[member] = StickyPartitionAssignor._metadata({'t'}, member_assignments[member], member_generations[member]) assignment = StickyPartitionAssignor.assign(cluster, member_metadata) @@ -788,7 +786,7 @@ def test_assignment_with_conflicting_previous_generations(mocker, execution_numb def make_member_metadata(subscriptions): member_metadata = {} - for member, topics in six.iteritems(subscriptions): + for member, topics in subscriptions.items(): member_metadata[member] = StickyPartitionAssignor._metadata(topics, []) return member_metadata @@ -813,9 +811,9 @@ def verify_validity_and_balance(subscriptions, assignment): :param subscriptions topic subscriptions of each consumer :param assignment: given assignment for balance check """ - assert six.viewkeys(subscriptions) == six.viewkeys(assignment) + assert subscriptions.keys() == assignment.keys() - consumers = sorted(six.viewkeys(assignment)) + consumers = sorted(assignment.keys()) for i in range(len(consumers)): consumer = consumers[i] partitions = assignment[consumer].partitions() @@ -846,7 +844,7 @@ def verify_validity_and_balance(subscriptions, assignment): assignments_by_topic = group_partitions_by_topic(partitions) other_assignments_by_topic = group_partitions_by_topic(other_partitions) if len(partitions) > len(other_partitions): - for topic in six.iterkeys(assignments_by_topic): + for topic in assignments_by_topic: assert topic not in other_assignments_by_topic, ( 'Error: Some partitions can be moved from {} ({} partitions) ' 'to {} ({} partitions) ' @@ -855,7 +853,7 @@ def verify_validity_and_balance(subscriptions, assignment): 'Assignments: {}'.format(consumer, len(partitions), other_consumer, len(other_partitions), subscriptions, assignment) ) if len(other_partitions) > len(partitions): - for topic in six.iterkeys(other_assignments_by_topic): + for topic in other_assignments_by_topic: assert topic not in assignments_by_topic, ( 'Error: Some partitions can be moved from {} ({} partitions) ' 'to {} ({} partitions) ' diff --git a/test/test_client_async.py b/test/test_client_async.py index acc400f9c..b4811d346 100644 --- a/test/test_client_async.py +++ b/test/test_client_async.py @@ -1,12 +1,4 @@ -from __future__ import absolute_import, division - -# selectors in stdlib as of py3.4 -try: - import selectors # pylint: disable=import-error -except ImportError: - # vendored backport module - import kafka.vendor.selectors34 as selectors - +import selectors import socket import time diff --git a/test/test_cluster.py b/test/test_cluster.py index 730b27cb6..d1dfb9353 100644 --- a/test/test_cluster.py +++ b/test/test_cluster.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import import socket diff --git a/test/test_codec.py b/test/test_codec.py index 24159c253..90c53a3fb 100644 --- a/test/test_codec.py +++ b/test/test_codec.py @@ -1,10 +1,7 @@ -from __future__ import absolute_import - import platform import struct import pytest -from kafka.vendor.six.moves import range from kafka.codec import ( has_snappy, has_lz4, has_zstd, diff --git a/test/test_conn.py b/test/test_conn.py index 3fa64066e..d8db22966 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -1,13 +1,9 @@ # pylint: skip-file -from __future__ import absolute_import from errno import EALREADY, EINPROGRESS, EISCONN, ECONNRESET import socket +from unittest import mock -try: - from unittest import mock -except ImportError: - import mock import pytest from kafka.conn import BrokerConnection, ConnectionStates @@ -22,13 +18,6 @@ import kafka.errors as Errors -from kafka.vendor import six - -if six.PY2: - ConnectionError = socket.error - TimeoutError = socket.error - BlockingIOError = Exception - @pytest.fixture def dns_lookup(mocker): diff --git a/test/test_consumer.py b/test/test_consumer.py index 0d9477729..60d101d69 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import pytest from kafka import KafkaConsumer, TopicPartition diff --git a/test/test_coordinator.py b/test/test_coordinator.py index b7db5ad19..3032a4973 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import import time import pytest diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 4794563ed..71c726bc8 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import import logging import pytest diff --git a/test/test_partitioner.py b/test/test_partitioner.py index 853fbf69e..c347fe02c 100644 --- a/test/test_partitioner.py +++ b/test/test_partitioner.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import pytest from kafka.partitioner import DefaultPartitioner, murmur2 diff --git a/test/test_producer.py b/test/test_producer.py index e79c682a7..970932733 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import gc import platform import threading diff --git a/test/test_producer_batch.py b/test/test_producer_batch.py index 7d959cbe9..e47e10a65 100644 --- a/test/test_producer_batch.py +++ b/test/test_producer_batch.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import, division import pytest diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py index 0f61c21cf..aaea6da58 100644 --- a/test/test_record_accumulator.py +++ b/test/test_record_accumulator.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import, division import pytest diff --git a/test/test_sender.py b/test/test_sender.py index 567f1b2ad..72b8f9a56 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -1,17 +1,11 @@ # pylint: skip-file -from __future__ import absolute_import import collections import io import time +from unittest.mock import call import pytest -try: - from unittest.mock import call -except ImportError: - from mock import call - -from kafka.vendor import six from kafka.client_async import KafkaClient from kafka.cluster import ClusterMetadata diff --git a/test/test_subscription_state.py b/test/test_subscription_state.py index 773606525..1e8de7a3b 100644 --- a/test/test_subscription_state.py +++ b/test/test_subscription_state.py @@ -1,10 +1,7 @@ -from __future__ import absolute_import - import pytest from kafka import TopicPartition from kafka.consumer.subscription_state import SubscriptionState, TopicPartitionState -from kafka.vendor import six def test_type_error(): @@ -44,8 +41,8 @@ def test_assign_from_subscribed(): s.assign_from_subscribed([TopicPartition('foo', 0), TopicPartition('foo', 1)]) assert set(s.assignment.keys()) == set([TopicPartition('foo', 0), TopicPartition('foo', 1)]) - assert all([isinstance(tps, TopicPartitionState) for tps in six.itervalues(s.assignment)]) - assert all([not tps.has_valid_position for tps in six.itervalues(s.assignment)]) + assert all([isinstance(tps, TopicPartitionState) for tps in s.assignment.values()]) + assert all([not tps.has_valid_position for tps in s.assignment.values()]) def test_change_subscription_after_assignment(): diff --git a/test/test_util.py b/test/test_util.py index f9e8a2b51..5db5fc8e0 100644 --- a/test/test_util.py +++ b/test/test_util.py @@ -1,5 +1,4 @@ # pylint: skip-file -from __future__ import absolute_import import pytest diff --git a/test/testutil.py b/test/testutil.py index b5dab1c02..1e1f8e198 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import os import random import re From e05ba6fdb812360a87c6ec7bb25c5a6249c92c07 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Dec 2025 06:29:11 -0800 Subject: [PATCH 44/51] Enable pylint workflow; disable/fix all outstanding errors (#2701) --- .github/workflows/python-package.yml | 2 +- kafka/admin/client.py | 27 ++++++++++++++------------- kafka/record/default_records.py | 4 ++-- kafka/record/legacy_records.py | 7 +++++-- test/integration/fixtures.py | 4 ++-- 5 files changed, 24 insertions(+), 20 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 21acc4a32..4d1e8e125 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -68,7 +68,7 @@ jobs: python -m pip install --upgrade pip pip install -r requirements-dev.txt - name: Pylint - run: pylint --recursive=y --errors-only --exit-zero kafka test + run: pylint --recursive=y --errors-only kafka test - name: Setup java uses: actions/setup-java@v5 with: diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 6bdac5613..6b57091d5 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -290,7 +290,7 @@ def _find_coordinator_id_request(self, group_id): request = FindCoordinatorRequest[version](group_id) elif version <= 2: request = FindCoordinatorRequest[version](group_id, 0) - return request + return request # pylint: disable=E0606 def _find_coordinator_id_process_response(self, response): """Process a FindCoordinatorResponse. @@ -506,7 +506,7 @@ def create_topics(self, new_topics, timeout_ms=None, validate_only=False): ) # TODO convert structs to a more pythonic interface # TODO raise exceptions if errors - return self._send_request_to_controller(request) + return self._send_request_to_controller(request) # pylint: disable=E0606 def delete_topics(self, topics, timeout_ms=None): """Delete topics from the cluster. @@ -680,7 +680,7 @@ def describe_acls(self, acl_filter): permission_type=acl_filter.permission_type ) - response = self.send_request(request) + response = self.send_request(request) # pylint: disable=E0606 error_type = Errors.for_code(response.error_code) if error_type is not Errors.NoError: # optionally we could retry if error_type.retriable @@ -793,7 +793,7 @@ def create_acls(self, acls): request = CreateAclsRequest[version]( creations=[self._convert_create_acls_resource_request_v1(acl) for acl in acls] ) - response = self.send_request(request) + response = self.send_request(request) # pylint: disable=E0606 return self._convert_create_acls_response_to_acls(acls, response) @staticmethod @@ -907,7 +907,7 @@ def delete_acls(self, acl_filters): request = DeleteAclsRequest[version]( filters=[self._convert_delete_acls_resource_request_v1(acl) for acl in acl_filters] ) - response = self.send_request(request) + response = self.send_request(request) # pylint: disable=E0606 return self._convert_delete_acls_response_to_matching_acls(acl_filters, response) @staticmethod @@ -1269,14 +1269,15 @@ def _describe_consumer_groups_process_response(self, response): # TODO: Fix GroupInformation defaults 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)) - return group_description + 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)) + return group_description + assert False, "DescribeGroupsResponse parsing failed" def describe_consumer_groups(self, group_ids, group_coordinator_id=None, include_authorized_operations=False): """Describe a set of consumer groups. diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index a3b9cd5d8..35fcba9cb 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -245,7 +245,7 @@ def _maybe_uncompress(self): uncompressed = lz4_decode(data.tobytes()) if compression_type == self.CODEC_ZSTD: uncompressed = zstd_decode(data.tobytes()) - self._buffer = bytearray(uncompressed) + self._buffer = bytearray(uncompressed) # pylint: disable=E0606 self._pos = 0 self._decompressed = True @@ -658,7 +658,7 @@ def _maybe_compress(self): compressed = lz4_encode(data) elif self._compression_type == self.CODEC_ZSTD: compressed = zstd_encode(data) - compressed_size = len(compressed) + compressed_size = len(compressed) # pylint: disable=E0606 if len(data) <= compressed_size: # We did not get any benefit from compression, lets send # uncompressed diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index f085978f0..4c9bf03dd 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -122,6 +122,9 @@ def _assert_has_codec(self, compression_type): checker, name = codecs.has_snappy, "snappy" elif compression_type == self.CODEC_LZ4: checker, name = codecs.has_lz4, "lz4" + else: + raise UnsupportedCodecError( + "Unrecognized compression type") if not checker(): raise UnsupportedCodecError( "Libraries for {} compression codec not found".format(name)) @@ -206,7 +209,7 @@ def _decompress(self, key_offset): uncompressed = lz4_decode_old_kafka(data.tobytes()) else: uncompressed = lz4_decode(data.tobytes()) - return uncompressed + return uncompressed # pylint: disable=E0606 def _read_header(self, pos): if self._magic == 0: @@ -483,7 +486,7 @@ def _maybe_compress(self): else: compressed = lz4_encode(data) size = self.size_in_bytes( - 0, timestamp=0, key=None, value=compressed) + 0, timestamp=0, key=None, value=compressed) # pylint: disable=E0606 # We will try to reuse the same buffer if we have enough space if size > len(self._buffer): self._buffer = bytearray(size) diff --git a/test/integration/fixtures.py b/test/integration/fixtures.py index 0b660e624..1723a3bd5 100644 --- a/test/integration/fixtures.py +++ b/test/integration/fixtures.py @@ -299,7 +299,7 @@ def __init__(self, host, port, broker_id, zookeeper=None, zk_chroot=None, if self.external: self.child = ExternalService(self.host, self.port) - (self._client,) = self.get_clients(1, client_id='_internal_client') + self._client = next(self.get_clients(1, client_id='_internal_client')) self.running = True else: self._client = None @@ -447,7 +447,7 @@ def start(self): else: raise RuntimeError('Failed to start KafkaInstance before max_timeout') - (self._client,) = self.get_clients(1, client_id='_internal_client') + self._client = next(self.get_clients(1, client_id='_internal_client')) self.out("Done!") self.running = True From 8a3a57d1c5b97490b564ddc9ca86ee38863b842d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 6 Dec 2025 07:29:35 -0800 Subject: [PATCH 45/51] Add OffsetAndMetadata + TopicPartition to api docs list --- docs/apidoc/modules.rst | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/apidoc/modules.rst b/docs/apidoc/modules.rst index 357fb5038..cc72f7511 100644 --- a/docs/apidoc/modules.rst +++ b/docs/apidoc/modules.rst @@ -10,3 +10,5 @@ kafka-python API KafkaClient BrokerConnection ClusterMetadata + OffsetAndMetadata + TopicPartition From 23b848c8a02ca23fe85e1765906710b98a6f390f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 13 Dec 2025 10:08:18 -0800 Subject: [PATCH 46/51] Bump actions/cache from 4 to 5 (#2702) --- .github/workflows/python-package.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 4d1e8e125..3c6446e95 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -76,7 +76,7 @@ jobs: java-version: 23 - name: Restore cached kafka releases id: cache-servers-dist-restore - uses: actions/cache/restore@v4 + uses: actions/cache/restore@v5 with: path: servers/dist key: servers-dist-${{ matrix.kafka }} @@ -84,7 +84,7 @@ jobs: run: make servers/${{ matrix.kafka }}/kafka-bin - name: Update kafka release cache id: cache-servers-dist-save - uses: actions/cache/save@v4 + uses: actions/cache/save@v5 with: path: servers/dist key: ${{ steps.cache-servers-dist-restore.outputs.cache-primary-key }} From 46a1cc69e9644b45a7224b4d92f7f87f63d1cff1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 13 Dec 2025 10:14:06 -0800 Subject: [PATCH 47/51] Add UUID protocol type (#2703) --- kafka/protocol/types.py | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 812c5e74d..7889e06d5 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -1,5 +1,6 @@ import struct from struct import error +import uuid from kafka.protocol.abstract import AbstractType @@ -88,6 +89,20 @@ def decode(cls, data): return _unpack(cls._unpack, data.read(8)) +class UUID(AbstractType): + ZERO_UUID = uuid.UUID(int=0) + + @classmethod + def encode(cls, value): + if isinstance(value, uuid.UUID): + return value.bytes + return uuid.UUID(value).bytes + + @classmethod + def decode(cls, data): + return uuid.UUID(bytes=data.read(16)) + + class String(AbstractType): def __init__(self, encoding='utf-8'): self.encoding = encoding @@ -346,7 +361,6 @@ def encode(cls, value): class CompactArray(Array): - def encode(self, items): if items is None: return UnsignedVarInt32.encode(0) From f6fc5ef083759093615da4575488e37918f8a205 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 13 Dec 2025 10:23:13 -0800 Subject: [PATCH 48/51] Struct/Schema cleanups (#2704) --- .../assignors/sticky/sticky_assignor.py | 8 +++--- kafka/protocol/abstract.py | 2 ++ kafka/protocol/api.py | 10 ------- kafka/protocol/group.py | 2 ++ kafka/protocol/struct.py | 27 +++++++------------ 5 files changed, 19 insertions(+), 30 deletions(-) diff --git a/kafka/coordinator/assignors/sticky/sticky_assignor.py b/kafka/coordinator/assignors/sticky/sticky_assignor.py index 243c26709..5e29d5894 100644 --- a/kafka/coordinator/assignors/sticky/sticky_assignor.py +++ b/kafka/coordinator/assignors/sticky/sticky_assignor.py @@ -6,9 +6,8 @@ from kafka.coordinator.assignors.sticky.partition_movements import PartitionMovements from kafka.coordinator.assignors.sticky.sorted_set import SortedSet from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata_v0, ConsumerProtocolMemberAssignment_v0 -from kafka.coordinator.protocol import Schema from kafka.protocol.struct import Struct -from kafka.protocol.types import String, Array, Int32 +from kafka.protocol.types import Array, Int32, Schema, String from kafka.structs import TopicPartition log = logging.getLogger(__name__) @@ -59,7 +58,10 @@ class StickyAssignorUserDataV1(Struct): """ SCHEMA = Schema( - ("previous_assignment", Array(("topic", String("utf-8")), ("partitions", Array(Int32)))), ("generation", Int32) + ("previous_assignment", Array( + ("topic", String("utf-8")), + ("partitions", Array(Int32)))), + ("generation", Int32) ) diff --git a/kafka/protocol/abstract.py b/kafka/protocol/abstract.py index e0d89433b..5817673cb 100644 --- a/kafka/protocol/abstract.py +++ b/kafka/protocol/abstract.py @@ -2,10 +2,12 @@ class AbstractType(object, metaclass=abc.ABCMeta): + @classmethod @abc.abstractmethod def encode(cls, value): # pylint: disable=no-self-argument pass + @classmethod @abc.abstractmethod def decode(cls, data): # pylint: disable=no-self-argument pass diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index c7a477cac..b6b8a2996 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -60,11 +60,6 @@ def API_VERSION(self): """Integer of api request version""" pass - @abc.abstractproperty - def SCHEMA(self): - """An instance of Schema() representing the request structure""" - pass - @abc.abstractproperty def RESPONSE_TYPE(self): """The Response class associated with the api request""" @@ -96,11 +91,6 @@ def API_VERSION(self): """Integer of api request/response version""" pass - @abc.abstractproperty - def SCHEMA(self): - """An instance of Schema() representing the response structure""" - pass - def to_object(self): return _to_object(self.SCHEMA, self) diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index 5d35ab219..a56bd48dc 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -158,6 +158,7 @@ class JoinGroupRequest_v5(Request): ] +# Currently unused -- see kafka.coordinator.protocol class ProtocolMetadata(Struct): SCHEMA = Schema( ('version', Int16), @@ -250,6 +251,7 @@ class SyncGroupRequest_v3(Request): ] +# Currently unused -- see kafka.coordinator.protocol class MemberAssignment(Struct): SCHEMA = Schema( ('version', Int16), diff --git a/kafka/protocol/struct.py b/kafka/protocol/struct.py index b482326fa..f66170c60 100644 --- a/kafka/protocol/struct.py +++ b/kafka/protocol/struct.py @@ -1,3 +1,4 @@ +import abc from io import BytesIO from kafka.protocol.abstract import AbstractType @@ -6,11 +7,15 @@ from kafka.util import WeakMethod -class Struct(AbstractType): - SCHEMA = Schema() +class Struct(metaclass=abc.ABCMeta): + + @abc.abstractproperty + def SCHEMA(self): + """An instance of Schema() representing the structure""" + pass def __init__(self, *args, **kwargs): - if len(args) == len(self.SCHEMA.fields): + if len(args) == len(self.SCHEMA): for i, name in enumerate(self.SCHEMA.names): setattr(self, name, args[i]) elif len(args) > 0: @@ -23,19 +28,7 @@ def __init__(self, *args, **kwargs): % (list(self.SCHEMA.names), ', '.join(kwargs.keys()))) - # overloading encode() to support both class and instance - # Without WeakMethod() this creates circular ref, which - # causes instances to "leak" to garbage - self.encode = WeakMethod(self._encode_self) - - @classmethod - def encode(cls, item): # pylint: disable=E0202 - bits = [] - for i, field in enumerate(cls.SCHEMA.fields): - bits.append(field.encode(item[i])) - return b''.join(bits) - - def _encode_self(self): + def encode(self): return self.SCHEMA.encode( [getattr(self, name) for name in self.SCHEMA.names] ) @@ -44,7 +37,7 @@ def _encode_self(self): def decode(cls, data): if isinstance(data, bytes): data = BytesIO(data) - return cls(*[field.decode(data) for field in cls.SCHEMA.fields]) + return cls(*cls.SCHEMA.decode(data)) def get_item(self, name): if name not in self.SCHEMA.names: From 9016c028c82921c9fd24a0616423951d73e884a6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Dec 2025 05:40:54 -0800 Subject: [PATCH 49/51] Remove unused kafka.protocol.message bits (#2705) --- kafka/protocol/message.py | 214 -------------------------------------- kafka/util.py | 12 --- test/test_protocol.py | 194 +--------------------------------- 3 files changed, 4 insertions(+), 416 deletions(-) delete mode 100644 kafka/protocol/message.py diff --git a/kafka/protocol/message.py b/kafka/protocol/message.py deleted file mode 100644 index 03d1098c0..000000000 --- a/kafka/protocol/message.py +++ /dev/null @@ -1,214 +0,0 @@ -import io -import time - -from kafka.codec import (has_gzip, has_snappy, has_lz4, has_zstd, - gzip_decode, snappy_decode, zstd_decode, - lz4_decode, lz4_decode_old_kafka) -from kafka.protocol.frame import KafkaBytes -from kafka.protocol.struct import Struct -from kafka.protocol.types import ( - Int8, Int32, Int64, Bytes, Schema, AbstractType -) -from kafka.util import crc32, WeakMethod - - -class Message(Struct): - SCHEMAS = [ - Schema( - ('crc', Int32), - ('magic', Int8), - ('attributes', Int8), - ('key', Bytes), - ('value', Bytes)), - Schema( - ('crc', Int32), - ('magic', Int8), - ('attributes', Int8), - ('timestamp', Int64), - ('key', Bytes), - ('value', Bytes)), - ] - SCHEMA = SCHEMAS[1] - CODEC_MASK = 0x07 - CODEC_GZIP = 0x01 - CODEC_SNAPPY = 0x02 - CODEC_LZ4 = 0x03 - CODEC_ZSTD = 0x04 - TIMESTAMP_TYPE_MASK = 0x08 - HEADER_SIZE = 22 # crc(4), magic(1), attributes(1), timestamp(8), key+value size(4*2) - - def __init__(self, value, key=None, magic=0, attributes=0, crc=0, - timestamp=None): - assert value is None or isinstance(value, bytes), 'value must be bytes' - assert key is None or isinstance(key, bytes), 'key must be bytes' - assert magic > 0 or timestamp is None, 'timestamp not supported in v0' - - # Default timestamp to now for v1 messages - if magic > 0 and timestamp is None: - timestamp = int(time.time() * 1000) - self.timestamp = timestamp - self.crc = crc - self._validated_crc = None - self.magic = magic - self.attributes = attributes - self.key = key - self.value = value - self.encode = WeakMethod(self._encode_self) - - @property - def timestamp_type(self): - """0 for CreateTime; 1 for LogAppendTime; None if unsupported. - - Value is determined by broker; produced messages should always set to 0 - Requires Kafka >= 0.10 / message version >= 1 - """ - if self.magic == 0: - return None - elif self.attributes & self.TIMESTAMP_TYPE_MASK: - return 1 - else: - return 0 - - def _encode_self(self, recalc_crc=True): - version = self.magic - if version == 1: - fields = (self.crc, self.magic, self.attributes, self.timestamp, self.key, self.value) - elif version == 0: - fields = (self.crc, self.magic, self.attributes, self.key, self.value) - else: - raise ValueError('Unrecognized message version: %s' % (version,)) - message = Message.SCHEMAS[version].encode(fields) - if not recalc_crc: - return message - self.crc = crc32(message[4:]) - crc_field = self.SCHEMAS[version].fields[0] - return crc_field.encode(self.crc) + message[4:] - - @classmethod - def decode(cls, data): - _validated_crc = None - if isinstance(data, bytes): - _validated_crc = crc32(data[4:]) - data = io.BytesIO(data) - # Partial decode required to determine message version - base_fields = cls.SCHEMAS[0].fields[0:3] - crc, magic, attributes = [field.decode(data) for field in base_fields] - remaining = cls.SCHEMAS[magic].fields[3:] - fields = [field.decode(data) for field in remaining] - if magic == 1: - timestamp = fields[0] - else: - timestamp = None - msg = cls(fields[-1], key=fields[-2], - magic=magic, attributes=attributes, crc=crc, - timestamp=timestamp) - msg._validated_crc = _validated_crc - return msg - - def validate_crc(self): - if self._validated_crc is None: - raw_msg = self._encode_self(recalc_crc=False) - self._validated_crc = crc32(raw_msg[4:]) - if self.crc == self._validated_crc: - return True - return False - - def is_compressed(self): - return self.attributes & self.CODEC_MASK != 0 - - def decompress(self): - codec = self.attributes & self.CODEC_MASK - assert codec in (self.CODEC_GZIP, self.CODEC_SNAPPY, self.CODEC_LZ4, self.CODEC_ZSTD) - if codec == self.CODEC_GZIP: - assert has_gzip(), 'Gzip decompression unsupported' - raw_bytes = gzip_decode(self.value) - elif codec == self.CODEC_SNAPPY: - assert has_snappy(), 'Snappy decompression unsupported' - raw_bytes = snappy_decode(self.value) - elif codec == self.CODEC_LZ4: - assert has_lz4(), 'LZ4 decompression unsupported' - if self.magic == 0: - raw_bytes = lz4_decode_old_kafka(self.value) - else: - raw_bytes = lz4_decode(self.value) - elif codec == self.CODEC_ZSTD: - assert has_zstd(), "ZSTD decompression unsupported" - raw_bytes = zstd_decode(self.value) - else: - raise Exception('This should be impossible') - - return MessageSet.decode(raw_bytes, bytes_to_read=len(raw_bytes)) - - def __hash__(self): - return hash(self._encode_self(recalc_crc=False)) - - -class PartialMessage(bytes): - def __repr__(self): - return 'PartialMessage(%s)' % (self,) - - -class MessageSet(AbstractType): - ITEM = Schema( - ('offset', Int64), - ('message', Bytes) - ) - HEADER_SIZE = 12 # offset + message_size - - @classmethod - def encode(cls, items, prepend_size=True): - # RecordAccumulator encodes messagesets internally - if isinstance(items, (io.BytesIO, KafkaBytes)): - size = Int32.decode(items) - if prepend_size: - # rewind and return all the bytes - items.seek(items.tell() - 4) - size += 4 - return items.read(size) - - encoded_values = [] - for (offset, message) in items: - encoded_values.append(Int64.encode(offset)) - encoded_values.append(Bytes.encode(message)) - encoded = b''.join(encoded_values) - if prepend_size: - return Bytes.encode(encoded) - else: - return encoded - - @classmethod - def decode(cls, data, bytes_to_read=None): - """Compressed messages should pass in bytes_to_read (via message size) - otherwise, we decode from data as Int32 - """ - if isinstance(data, bytes): - data = io.BytesIO(data) - if bytes_to_read is None: - bytes_to_read = Int32.decode(data) - - # if FetchRequest max_bytes is smaller than the available message set - # the server returns partial data for the final message - # So create an internal buffer to avoid over-reading - raw = io.BytesIO(data.read(bytes_to_read)) - - items = [] - while bytes_to_read: - try: - offset = Int64.decode(raw) - msg_bytes = Bytes.decode(raw) - bytes_to_read -= 8 + 4 + len(msg_bytes) - items.append((offset, len(msg_bytes), Message.decode(msg_bytes))) - except ValueError: - # PartialMessage to signal that max_bytes may be too small - items.append((None, None, PartialMessage())) - break - return items - - @classmethod - def repr(cls, messages): - if isinstance(messages, (KafkaBytes, io.BytesIO)): - offset = messages.tell() - decoded = cls.decode(messages) - messages.seek(offset) - messages = decoded - return str([cls.ITEM.repr(m) for m in messages]) diff --git a/kafka/util.py b/kafka/util.py index 29482bce1..5c7dd927c 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -7,18 +7,6 @@ from kafka.errors import KafkaTimeoutError -MAX_INT = 2 ** 31 -TO_SIGNED = 2 ** 32 - -def crc32(data): - crc = binascii.crc32(data) - # CRC is encoded as a signed int in kafka protocol - # so we'll convert the unsigned result to signed - if crc >= MAX_INT: - crc -= TO_SIGNED - return crc - - class Timer: __slots__ = ('_start_at', '_expire_at', '_timeout_ms', '_error_message') diff --git a/test/test_protocol.py b/test/test_protocol.py index 35ca938e1..45755c4c0 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -7,158 +7,10 @@ from kafka.protocol.api import RequestHeader from kafka.protocol.fetch import FetchRequest, FetchResponse from kafka.protocol.find_coordinator import FindCoordinatorRequest -from kafka.protocol.message import Message, MessageSet, PartialMessage from kafka.protocol.metadata import MetadataRequest from kafka.protocol.types import Int16, Int32, Int64, String, UnsignedVarInt32, CompactString, CompactArray, CompactBytes, BitField -def test_create_message(): - payload = b'test' - key = b'key' - msg = Message(payload, key=key) - assert msg.magic == 0 - assert msg.attributes == 0 - assert msg.key == key - assert msg.value == payload - - -def test_encode_message_v0(): - message = Message(b'test', key=b'key') - encoded = message.encode() - expect = b''.join([ - struct.pack('>i', -1427009701), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - assert encoded == expect - - -def test_encode_message_v1(): - message = Message(b'test', key=b'key', magic=1, timestamp=1234) - encoded = message.encode() - expect = b''.join([ - struct.pack('>i', 1331087195), # CRC - struct.pack('>bb', 1, 0), # Magic, flags - struct.pack('>q', 1234), # Timestamp - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - assert encoded == expect - - -def test_decode_message(): - encoded = b''.join([ - struct.pack('>i', -1427009701), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - decoded_message = Message.decode(encoded) - msg = Message(b'test', key=b'key') - msg.encode() # crc is recalculated during encoding - assert decoded_message == msg - - -def test_decode_message_validate_crc(): - encoded = b''.join([ - struct.pack('>i', -1427009701), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - decoded_message = Message.decode(encoded) - assert decoded_message.validate_crc() is True - - encoded = b''.join([ - struct.pack('>i', 1234), # Incorrect CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - decoded_message = Message.decode(encoded) - assert decoded_message.validate_crc() is False - - -def test_encode_message_set(): - messages = [ - Message(b'v1', key=b'k1'), - Message(b'v2', key=b'k2') - ] - encoded = MessageSet.encode([(0, msg.encode()) - for msg in messages]) - expect = b''.join([ - struct.pack('>q', 0), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - struct.pack('>q', 0), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 2), # Length of value - b'v2', # Value - ]) - expect = struct.pack('>i', len(expect)) + expect - assert encoded == expect - - -def test_decode_message_set(): - encoded = b''.join([ - struct.pack('>q', 0), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - struct.pack('>q', 1), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 2), # Length of value - b'v2', # Value - ]) - - msgs = MessageSet.decode(encoded, bytes_to_read=len(encoded)) - assert len(msgs) == 2 - msg1, msg2 = msgs - - returned_offset1, message1_size, decoded_message1 = msg1 - returned_offset2, message2_size, decoded_message2 = msg2 - - assert returned_offset1 == 0 - message1 = Message(b'v1', key=b'k1') - message1.encode() - assert decoded_message1 == message1 - - assert returned_offset2 == 1 - message2 = Message(b'v2', key=b'k2') - message2.encode() - assert decoded_message2 == message2 - - def test_encode_message_header(): expect = b''.join([ struct.pack('>h', 10), # API Key @@ -173,44 +25,6 @@ def test_encode_message_header(): assert header.encode() == expect -def test_decode_message_set_partial(): - encoded = b''.join([ - struct.pack('>q', 0), # Msg Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - struct.pack('>q', 1), # Msg Offset - struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 8), # Length of value - b'ar', # Value (truncated) - ]) - - msgs = MessageSet.decode(encoded, bytes_to_read=len(encoded)) - assert len(msgs) == 2 - msg1, msg2 = msgs - - returned_offset1, message1_size, decoded_message1 = msg1 - returned_offset2, message2_size, decoded_message2 = msg2 - - assert returned_offset1 == 0 - message1 = Message(b'v1', key=b'k1') - message1.encode() - assert decoded_message1 == message1 - - assert returned_offset2 is None - assert message2_size is None - assert decoded_message2 == PartialMessage() - - def test_decode_fetch_response_partial(): encoded = b''.join([ Int32.encode(1), # Num Topics (Array) @@ -265,10 +79,10 @@ def test_decode_fetch_response_partial(): assert topic == 'foobar' assert len(partitions) == 2 - m1 = MessageSet.decode( - partitions[0][3], bytes_to_read=len(partitions[0][3])) - assert len(m1) == 2 - assert m1[1] == (None, None, PartialMessage()) + #m1 = MessageSet.decode( + # partitions[0][3], bytes_to_read=len(partitions[0][3])) + #assert len(m1) == 2 + #assert m1[1] == (None, None, PartialMessage()) def test_struct_unrecognized_kwargs(): From c22bf58b137df11478eef4fcc99e92632c2ccfc9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Dec 2025 06:15:03 -0800 Subject: [PATCH 50/51] Fix VarInt/VarLong encoding; move tests to test/protocol/ (#2706) --- kafka/protocol/types.py | 38 ++++---- test/protocol/test_api.py | 35 +++++++ test/protocol/test_bit_field.py | 13 +++ test/protocol/test_compact.py | 38 ++++++++ test/protocol/test_fetch.py | 68 ++++++++++++++ test/protocol/test_varint.py | 109 ++++++++++++++++++++++ test/test_protocol.py | 158 -------------------------------- 7 files changed, 282 insertions(+), 177 deletions(-) create mode 100644 test/protocol/test_api.py create mode 100644 test/protocol/test_bit_field.py create mode 100644 test/protocol/test_compact.py create mode 100644 test/protocol/test_fetch.py create mode 100644 test/protocol/test_varint.py delete mode 100644 test/test_protocol.py diff --git a/kafka/protocol/types.py b/kafka/protocol/types.py index 7889e06d5..b0811c59b 100644 --- a/kafka/protocol/types.py +++ b/kafka/protocol/types.py @@ -226,6 +226,17 @@ def repr(self, list_of_items): class UnsignedVarInt32(AbstractType): + @classmethod + def decode(cls, data): + value = VarInt32.decode(data) + return (value << 1) ^ (value >> 31) + + @classmethod + def encode(cls, value): + return VarInt32.encode((value >> 1) ^ -(value & 1)) + + +class VarInt32(AbstractType): @classmethod def decode(cls, data): value, i = 0, 0 @@ -238,10 +249,12 @@ def decode(cls, data): if i > 28: raise ValueError('Invalid value {}'.format(value)) value |= b << i - return value + return (value >> 1) ^ -(value & 1) @classmethod def encode(cls, value): + # bring it in line with the java binary repr + value = (value << 1) ^ (value >> 31) value &= 0xffffffff ret = b'' while (value & 0xffffff80) != 0: @@ -252,25 +265,12 @@ def encode(cls, value): return ret -class VarInt32(AbstractType): - @classmethod - def decode(cls, data): - value = UnsignedVarInt32.decode(data) - return (value >> 1) ^ -(value & 1) - - @classmethod - def encode(cls, value): - # bring it in line with the java binary repr - value &= 0xffffffff - return UnsignedVarInt32.encode((value << 1) ^ (value >> 31)) - - class VarInt64(AbstractType): @classmethod def decode(cls, data): value, i = 0, 0 while True: - b = data.read(1) + b, = struct.unpack('B', data.read(1)) if not (b & 0x80): break value |= (b & 0x7f) << i @@ -283,14 +283,14 @@ def decode(cls, data): @classmethod def encode(cls, value): # bring it in line with the java binary repr + value = (value << 1) ^ (value >> 63) value &= 0xffffffffffffffff - v = (value << 1) ^ (value >> 63) ret = b'' - while (v & 0xffffffffffffff80) != 0: + while (value & 0xffffffffffffff80) != 0: b = (value & 0x7f) | 0x80 ret += struct.pack('B', b) - v >>= 7 - ret += struct.pack('B', v) + value >>= 7 + ret += struct.pack('B', value) return ret diff --git a/test/protocol/test_api.py b/test/protocol/test_api.py new file mode 100644 index 000000000..4bb7273bd --- /dev/null +++ b/test/protocol/test_api.py @@ -0,0 +1,35 @@ +import struct + +import pytest + +from kafka.protocol.api import RequestHeader +from kafka.protocol.fetch import FetchRequest +from kafka.protocol.find_coordinator import FindCoordinatorRequest +from kafka.protocol.metadata import MetadataRequest + + +def test_encode_message_header(): + expect = b''.join([ + struct.pack('>h', 10), # API Key + struct.pack('>h', 0), # API Version + struct.pack('>i', 4), # Correlation Id + struct.pack('>h', len('client3')), # Length of clientId + b'client3', # ClientId + ]) + + req = FindCoordinatorRequest[0]('foo') + header = RequestHeader(req, correlation_id=4, client_id='client3') + assert header.encode() == expect + + +def test_struct_unrecognized_kwargs(): + try: + _mr = MetadataRequest[0](topicz='foo') + assert False, 'Structs should not allow unrecognized kwargs' + except ValueError: + pass + + +def test_struct_missing_kwargs(): + fr = FetchRequest[0](max_wait_time=100) + assert fr.min_bytes is None diff --git a/test/protocol/test_bit_field.py b/test/protocol/test_bit_field.py new file mode 100644 index 000000000..5db155241 --- /dev/null +++ b/test/protocol/test_bit_field.py @@ -0,0 +1,13 @@ +import io + +import pytest + +from kafka.protocol.types import BitField + + +@pytest.mark.parametrize(('test_set',), [ + (set([0, 1, 5, 10, 31]),), + (set(range(32)),), +]) +def test_bit_field(test_set): + assert BitField.decode(io.BytesIO(BitField.encode(test_set))) == test_set diff --git a/test/protocol/test_compact.py b/test/protocol/test_compact.py new file mode 100644 index 000000000..c5940aa70 --- /dev/null +++ b/test/protocol/test_compact.py @@ -0,0 +1,38 @@ +import io +import struct + +import pytest + +from kafka.protocol.types import CompactString, CompactArray, CompactBytes + + +def test_compact_data_structs(): + cs = CompactString() + encoded = cs.encode(None) + assert encoded == struct.pack('B', 0) + decoded = cs.decode(io.BytesIO(encoded)) + assert decoded is None + assert b'\x01' == cs.encode('') + assert '' == cs.decode(io.BytesIO(b'\x01')) + encoded = cs.encode("foobarbaz") + assert cs.decode(io.BytesIO(encoded)) == "foobarbaz" + + arr = CompactArray(CompactString()) + assert arr.encode(None) == b'\x00' + assert arr.decode(io.BytesIO(b'\x00')) is None + enc = arr.encode([]) + assert enc == b'\x01' + assert [] == arr.decode(io.BytesIO(enc)) + encoded = arr.encode(["foo", "bar", "baz", "quux"]) + assert arr.decode(io.BytesIO(encoded)) == ["foo", "bar", "baz", "quux"] + + enc = CompactBytes.encode(None) + assert enc == b'\x00' + assert CompactBytes.decode(io.BytesIO(b'\x00')) is None + enc = CompactBytes.encode(b'') + assert enc == b'\x01' + assert CompactBytes.decode(io.BytesIO(b'\x01')) == b'' + enc = CompactBytes.encode(b'foo') + assert CompactBytes.decode(io.BytesIO(enc)) == b'foo' + + diff --git a/test/protocol/test_fetch.py b/test/protocol/test_fetch.py new file mode 100644 index 000000000..993df9c89 --- /dev/null +++ b/test/protocol/test_fetch.py @@ -0,0 +1,68 @@ +#pylint: skip-file +import io +import struct + +import pytest + +from kafka.protocol.fetch import FetchResponse +from kafka.protocol.types import Int16, Int32, Int64, String + + +def test_decode_fetch_response_partial(): + encoded = b''.join([ + Int32.encode(1), # Num Topics (Array) + String('utf-8').encode('foobar'), + Int32.encode(2), # Num Partitions (Array) + Int32.encode(0), # Partition id + Int16.encode(0), # Error Code + Int64.encode(1234), # Highwater offset + Int32.encode(52), # MessageSet size + Int64.encode(0), # Msg Offset + Int32.encode(18), # Msg Size + struct.pack('>i', 1474775406), # CRC + struct.pack('>bb', 0, 0), # Magic, flags + struct.pack('>i', 2), # Length of key + b'k1', # Key + struct.pack('>i', 2), # Length of value + b'v1', # Value + + Int64.encode(1), # Msg Offset + struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) + struct.pack('>i', -16383415), # CRC + struct.pack('>bb', 0, 0), # Magic, flags + struct.pack('>i', 2), # Length of key + b'k2', # Key + struct.pack('>i', 8), # Length of value + b'ar', # Value (truncated) + Int32.encode(1), + Int16.encode(0), + Int64.encode(2345), + Int32.encode(52), # MessageSet size + Int64.encode(0), # Msg Offset + Int32.encode(18), # Msg Size + struct.pack('>i', 1474775406), # CRC + struct.pack('>bb', 0, 0), # Magic, flags + struct.pack('>i', 2), # Length of key + b'k1', # Key + struct.pack('>i', 2), # Length of value + b'v1', # Value + + Int64.encode(1), # Msg Offset + struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) + struct.pack('>i', -16383415), # CRC + struct.pack('>bb', 0, 0), # Magic, flags + struct.pack('>i', 2), # Length of key + b'k2', # Key + struct.pack('>i', 8), # Length of value + b'ar', # Value (truncated) + ]) + resp = FetchResponse[0].decode(io.BytesIO(encoded)) + assert len(resp.topics) == 1 + topic, partitions = resp.topics[0] + assert topic == 'foobar' + assert len(partitions) == 2 + + #m1 = MessageSet.decode( + # partitions[0][3], bytes_to_read=len(partitions[0][3])) + #assert len(m1) == 2 + #assert m1[1] == (None, None, PartialMessage()) diff --git a/test/protocol/test_varint.py b/test/protocol/test_varint.py new file mode 100644 index 000000000..826ad949c --- /dev/null +++ b/test/protocol/test_varint.py @@ -0,0 +1,109 @@ +import io +import struct + +import pytest + +from kafka.protocol.types import UnsignedVarInt32, VarInt32, VarInt64 + + +@pytest.mark.parametrize(('value','expected_encoded'), [ + (0, [0x00]), + (-1, [0xFF, 0xFF, 0xFF, 0xFF, 0x0F]), + (1, [0x01]), + (63, [0x3F]), + (-64, [0xC0, 0xFF, 0xFF, 0xFF, 0x0F]), + (64, [0x40]), + (8191, [0xFF, 0x3F]), + (-8192, [0x80, 0xC0, 0xFF, 0xFF, 0x0F]), + (8192, [0x80, 0x40]), + (-8193, [0xFF, 0xBF, 0xFF, 0xFF, 0x0F]), + (1048575, [0xFF, 0xFF, 0x3F]), + (1048576, [0x80, 0x80, 0x40]), + (2147483647, [0xFF, 0xFF, 0xFF, 0xFF, 0x07]), + (-2147483648, [0x80, 0x80, 0x80, 0x80, 0x08]), +]) +def test_unsigned_varint_serde(value, expected_encoded): + value &= 0xffffffff + encoded = UnsignedVarInt32.encode(value) + assert encoded == b''.join(struct.pack('>B', x) for x in expected_encoded) + assert value == UnsignedVarInt32.decode(io.BytesIO(encoded)) + + +@pytest.mark.parametrize(('value','expected_encoded'), [ + (0, [0x00]), + (-1, [0x01]), + (1, [0x02]), + (63, [0x7E]), + (-64, [0x7F]), + (64, [0x80, 0x01]), + (-65, [0x81, 0x01]), + (8191, [0xFE, 0x7F]), + (-8192, [0xFF, 0x7F]), + (8192, [0x80, 0x80, 0x01]), + (-8193, [0x81, 0x80, 0x01]), + (1048575, [0xFE, 0xFF, 0x7F]), + (-1048576, [0xFF, 0xFF, 0x7F]), + (1048576, [0x80, 0x80, 0x80, 0x01]), + (-1048577, [0x81, 0x80, 0x80, 0x01]), + (134217727, [0xFE, 0xFF, 0xFF, 0x7F]), + (-134217728, [0xFF, 0xFF, 0xFF, 0x7F]), + (134217728, [0x80, 0x80, 0x80, 0x80, 0x01]), + (-134217729, [0x81, 0x80, 0x80, 0x80, 0x01]), + (2147483647, [0xFE, 0xFF, 0xFF, 0xFF, 0x0F]), + (-2147483648, [0xFF, 0xFF, 0xFF, 0xFF, 0x0F]), +]) +def test_signed_varint_serde(value, expected_encoded): + encoded = VarInt32.encode(value) + assert encoded == b''.join(struct.pack('>B', x) for x in expected_encoded) + assert value == VarInt32.decode(io.BytesIO(encoded)) + + +@pytest.mark.parametrize(('value','expected_encoded'), [ + (0, [0x00]), + (-1, [0x01]), + (1, [0x02]), + (63, [0x7E]), + (-64, [0x7F]), + (64, [0x80, 0x01]), + (-65, [0x81, 0x01]), + (8191, [0xFE, 0x7F]), + (-8192, [0xFF, 0x7F]), + (8192, [0x80, 0x80, 0x01]), + (-8193, [0x81, 0x80, 0x01]), + (1048575, [0xFE, 0xFF, 0x7F]), + (-1048576, [0xFF, 0xFF, 0x7F]), + (1048576, [0x80, 0x80, 0x80, 0x01]), + (-1048577, [0x81, 0x80, 0x80, 0x01]), + (134217727, [0xFE, 0xFF, 0xFF, 0x7F]), + (-134217728, [0xFF, 0xFF, 0xFF, 0x7F]), + (134217728, [0x80, 0x80, 0x80, 0x80, 0x01]), + (-134217729, [0x81, 0x80, 0x80, 0x80, 0x01]), + (2147483647, [0xFE, 0xFF, 0xFF, 0xFF, 0x0F]), + (-2147483648, [0xFF, 0xFF, 0xFF, 0xFF, 0x0F]), + (17179869183, [0xFE, 0xFF, 0xFF, 0xFF, 0x7F]), + (-17179869184, [0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (17179869184, [0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (-17179869185, [0x81, 0x80, 0x80, 0x80, 0x80, 0x01]), + (2199023255551, [0xFE, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (-2199023255552, [0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (2199023255552, [0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (-2199023255553, [0x81, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (281474976710655, [0xFE, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (-281474976710656, [0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (281474976710656, [0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (-281474976710657, [0x81, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 1]), + (36028797018963967, [0xFE, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (-36028797018963968, [0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (36028797018963968, [0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (-36028797018963969, [0x81, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (4611686018427387903, [0xFE, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (-4611686018427387904, [0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x7F]), + (4611686018427387904, [0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (-4611686018427387905, [0x81, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x80, 0x01]), + (9223372036854775807, [0xFE, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x01]), + (-9223372036854775808, [0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x01]), +]) +def test_signed_varlong_serde(value, expected_encoded): + encoded = VarInt64.encode(value) + assert encoded == b''.join(struct.pack('>B', x) for x in expected_encoded) + assert value == VarInt64.decode(io.BytesIO(encoded)) diff --git a/test/test_protocol.py b/test/test_protocol.py deleted file mode 100644 index 45755c4c0..000000000 --- a/test/test_protocol.py +++ /dev/null @@ -1,158 +0,0 @@ -#pylint: skip-file -import io -import struct - -import pytest - -from kafka.protocol.api import RequestHeader -from kafka.protocol.fetch import FetchRequest, FetchResponse -from kafka.protocol.find_coordinator import FindCoordinatorRequest -from kafka.protocol.metadata import MetadataRequest -from kafka.protocol.types import Int16, Int32, Int64, String, UnsignedVarInt32, CompactString, CompactArray, CompactBytes, BitField - - -def test_encode_message_header(): - expect = b''.join([ - struct.pack('>h', 10), # API Key - struct.pack('>h', 0), # API Version - struct.pack('>i', 4), # Correlation Id - struct.pack('>h', len('client3')), # Length of clientId - b'client3', # ClientId - ]) - - req = FindCoordinatorRequest[0]('foo') - header = RequestHeader(req, correlation_id=4, client_id='client3') - assert header.encode() == expect - - -def test_decode_fetch_response_partial(): - encoded = b''.join([ - Int32.encode(1), # Num Topics (Array) - String('utf-8').encode('foobar'), - Int32.encode(2), # Num Partitions (Array) - Int32.encode(0), # Partition id - Int16.encode(0), # Error Code - Int64.encode(1234), # Highwater offset - Int32.encode(52), # MessageSet size - Int64.encode(0), # Msg Offset - Int32.encode(18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - Int64.encode(1), # Msg Offset - struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 8), # Length of value - b'ar', # Value (truncated) - Int32.encode(1), - Int16.encode(0), - Int64.encode(2345), - Int32.encode(52), # MessageSet size - Int64.encode(0), # Msg Offset - Int32.encode(18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - Int64.encode(1), # Msg Offset - struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 8), # Length of value - b'ar', # Value (truncated) - ]) - resp = FetchResponse[0].decode(io.BytesIO(encoded)) - assert len(resp.topics) == 1 - topic, partitions = resp.topics[0] - assert topic == 'foobar' - assert len(partitions) == 2 - - #m1 = MessageSet.decode( - # partitions[0][3], bytes_to_read=len(partitions[0][3])) - #assert len(m1) == 2 - #assert m1[1] == (None, None, PartialMessage()) - - -def test_struct_unrecognized_kwargs(): - try: - _mr = MetadataRequest[0](topicz='foo') - assert False, 'Structs should not allow unrecognized kwargs' - except ValueError: - pass - - -def test_struct_missing_kwargs(): - fr = FetchRequest[0](max_wait_time=100) - assert fr.min_bytes is None - - -def test_unsigned_varint_serde(): - pairs = { - 0: [0], - -1: [0xff, 0xff, 0xff, 0xff, 0x0f], - 1: [1], - 63: [0x3f], - -64: [0xc0, 0xff, 0xff, 0xff, 0x0f], - 64: [0x40], - 8191: [0xff, 0x3f], - -8192: [0x80, 0xc0, 0xff, 0xff, 0x0f], - 8192: [0x80, 0x40], - -8193: [0xff, 0xbf, 0xff, 0xff, 0x0f], - 1048575: [0xff, 0xff, 0x3f], - - } - for value, expected_encoded in pairs.items(): - value &= 0xffffffff - encoded = UnsignedVarInt32.encode(value) - assert encoded == b''.join(struct.pack('>B', x) for x in expected_encoded) - assert value == UnsignedVarInt32.decode(io.BytesIO(encoded)) - - -def test_compact_data_structs(): - cs = CompactString() - encoded = cs.encode(None) - assert encoded == struct.pack('B', 0) - decoded = cs.decode(io.BytesIO(encoded)) - assert decoded is None - assert b'\x01' == cs.encode('') - assert '' == cs.decode(io.BytesIO(b'\x01')) - encoded = cs.encode("foobarbaz") - assert cs.decode(io.BytesIO(encoded)) == "foobarbaz" - - arr = CompactArray(CompactString()) - assert arr.encode(None) == b'\x00' - assert arr.decode(io.BytesIO(b'\x00')) is None - enc = arr.encode([]) - assert enc == b'\x01' - assert [] == arr.decode(io.BytesIO(enc)) - encoded = arr.encode(["foo", "bar", "baz", "quux"]) - assert arr.decode(io.BytesIO(encoded)) == ["foo", "bar", "baz", "quux"] - - enc = CompactBytes.encode(None) - assert enc == b'\x00' - assert CompactBytes.decode(io.BytesIO(b'\x00')) is None - enc = CompactBytes.encode(b'') - assert enc == b'\x01' - assert CompactBytes.decode(io.BytesIO(b'\x01')) == b'' - enc = CompactBytes.encode(b'foo') - assert CompactBytes.decode(io.BytesIO(enc)) == b'foo' - - -@pytest.mark.parametrize(('test_set',), [ - (set([0, 1, 5, 10, 31]),), - (set(range(32)),), -]) -def test_bit_field(test_set): - assert BitField.decode(io.BytesIO(BitField.encode(test_set))) == test_set From bfa9aff0c32844a3511b5f29698ac44ab709bdba Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 14 Dec 2025 06:15:38 -0800 Subject: [PATCH 51/51] Debug log send/recv bytes from protocol parser (#2707) --- kafka/conn.py | 14 ++++++++------ kafka/protocol/parser.py | 9 +++++++-- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index a213a4c53..bfabde407 100755 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -274,9 +274,7 @@ def __init__(self, host, port, afi, **configs): # can use a simple dictionary of correlation_id => request data self.in_flight_requests = dict() - self._protocol = KafkaProtocol( - client_id=self.config['client_id'], - api_version=self.config['api_version']) + self._protocol = self._new_protocol_parser() self.state = ConnectionStates.DISCONNECTED self._reset_reconnect_backoff() self._sock = None @@ -295,6 +293,12 @@ def __init__(self, host, port, afi, **configs): self.config['metric_group_prefix'], self.node_id) + def _new_protocol_parser(self): + return KafkaProtocol( + ident='%s:%d' % (self.host, self.port), + client_id=self.config['client_id'], + api_version=self.config['api_version']) + def _init_sasl_mechanism(self): if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'): self._sasl_mechanism = get_sasl_mechanism(self.config['sasl_mechanism'])(host=self.host, **self.config) @@ -934,9 +938,7 @@ def close(self, error=None): self._api_versions_future = None self._sasl_auth_future = None self._init_sasl_mechanism() - self._protocol = KafkaProtocol( - client_id=self.config['client_id'], - api_version=self.config['api_version']) + self._protocol = self._new_protocol_parser() self._send_buffer = b'' if error is None: error = Errors.Cancelled(str(self)) diff --git a/kafka/protocol/parser.py b/kafka/protocol/parser.py index 84d9e8718..174178acc 100644 --- a/kafka/protocol/parser.py +++ b/kafka/protocol/parser.py @@ -22,7 +22,8 @@ class KafkaProtocol(object): Currently only used to check for 0.8.2 protocol quirks, but may be used for more in the future. """ - def __init__(self, client_id=None, api_version=None): + def __init__(self, client_id=None, api_version=None, ident=''): + self._ident = ident if client_id is None: client_id = self._gen_client_id() self._client_id = client_id @@ -53,7 +54,7 @@ def send_request(self, request, correlation_id=None): Returns: correlation_id """ - log.debug('Sending request %s', request) + log.debug('Sending request %s', request.__class__.__name__) if correlation_id is None: correlation_id = self._next_correlation_id() @@ -71,6 +72,8 @@ def send_bytes(self): """Retrieve all pending bytes to send on the network""" data = b''.join(self.bytes_to_send) self.bytes_to_send = [] + if data: + log.debug('%s Send: %r', self._ident, data) return data def receive_bytes(self, data): @@ -92,6 +95,8 @@ def receive_bytes(self, data): i = 0 n = len(data) responses = [] + if data: + log.debug('%s Recv: %r', self._ident, data) while i < n: # Not receiving is the state of reading the payload header