From 5141d014cf4ba491440c5c272620da5dd882e288 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 6 Mar 2025 07:29:14 -0800 Subject: [PATCH 001/205] 1.1 broker_api_versions --- kafka/protocol/broker_api_versions.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kafka/protocol/broker_api_versions.py b/kafka/protocol/broker_api_versions.py index db7567180..299ab547a 100644 --- a/kafka/protocol/broker_api_versions.py +++ b/kafka/protocol/broker_api_versions.py @@ -23,6 +23,8 @@ # Adds Sasl Authenticate, and additional admin apis (describe/alter log dirs, etc) (1, 0): {0: (0, 5), 1: (0, 6), 2: (0, 2), 3: (0, 5), 4: (0, 1), 5: (0, 0), 6: (0, 4), 7: (0, 1), 8: (0, 3), 9: (0, 3), 10: (0, 1), 11: (0, 2), 12: (0, 1), 13: (0, 1), 14: (0, 1), 15: (0, 1), 16: (0, 1), 17: (0, 1), 18: (0, 1), 19: (0, 2), 20: (0, 1), 21: (0, 0), 22: (0, 0), 23: (0, 0), 24: (0, 0), 25: (0, 0), 26: (0, 0), 27: (0, 0), 28: (0, 0), 29: (0, 0), 30: (0, 0), 31: (0, 0), 32: (0, 0), 33: (0, 0), 34: (0, 0), 35: (0, 0), 36: (0, 0), 37: (0, 0)}, + (1, 1): {0: (0, 5), 1: (0, 7), 2: (0, 2), 3: (0, 5), 4: (0, 1), 5: (0, 0), 6: (0, 4), 7: (0, 1), 8: (0, 3), 9: (0, 3), 10: (0, 1), 11: (0, 2), 12: (0, 1), 13: (0, 1), 14: (0, 1), 15: (0, 1), 16: (0, 1), 17: (0, 1), 18: (0, 1), 19: (0, 2), 20: (0, 1), 21: (0, 0), 22: (0, 0), 23: (0, 0), 24: (0, 0), 25: (0, 0), 26: (0, 0), 27: (0, 0), 28: (0, 0), 29: (0, 0), 30: (0, 0), 31: (0, 0), 32: (0, 1), 33: (0, 0), 34: (0, 0), 35: (0, 0), 36: (0, 0), 37: (0, 0), 38: (0, 0), 39: (0, 0), 40: (0, 0), 41: (0, 0), 42: (0, 0)}, + (2, 0): {0: (0, 6), 1: (0, 8), 2: (0, 3), 3: (0, 6), 4: (0, 1), 5: (0, 0), 6: (0, 4), 7: (0, 1), 8: (0, 4), 9: (0, 4), 10: (0, 2), 11: (0, 3), 12: (0, 2), 13: (0, 2), 14: (0, 2), 15: (0, 2), 16: (0, 2), 17: (0, 1), 18: (0, 2), 19: (0, 3), 20: (0, 2), 21: (0, 1), 22: (0, 1), 23: (0, 1), 24: (0, 1), 25: (0, 1), 26: (0, 1), 27: (0, 0), 28: (0, 1), 29: (0, 1), 30: (0, 1), 31: (0, 1), 32: (0, 2), 33: (0, 1), 34: (0, 1), 35: (0, 1), 36: (0, 0), 37: (0, 1), 38: (0, 1), 39: (0, 1), 40: (0, 1), 41: (0, 1), 42: (0, 1)}, (2, 1): {0: (0, 7), 1: (0, 10), 2: (0, 4), 3: (0, 7), 4: (0, 1), 5: (0, 0), 6: (0, 4), 7: (0, 1), 8: (0, 6), 9: (0, 5), 10: (0, 2), 11: (0, 3), 12: (0, 2), 13: (0, 2), 14: (0, 2), 15: (0, 2), 16: (0, 2), 17: (0, 1), 18: (0, 2), 19: (0, 3), 20: (0, 3), 21: (0, 1), 22: (0, 1), 23: (0, 2), 24: (0, 1), 25: (0, 1), 26: (0, 1), 27: (0, 0), 28: (0, 2), 29: (0, 1), 30: (0, 1), 31: (0, 1), 32: (0, 2), 33: (0, 1), 34: (0, 1), 35: (0, 1), 36: (0, 0), 37: (0, 1), 38: (0, 1), 39: (0, 1), 40: (0, 1), 41: (0, 1), 42: (0, 1)}, From 760f7414cc0250a0048e781f0402332e2b5d3a47 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 7 Mar 2025 18:22:03 -0800 Subject: [PATCH 002/205] Call ApiVersionsRequest during connection, prior to Sasl Handshake (#2493) --- kafka/client_async.py | 102 ++++++----- kafka/conn.py | 314 ++++++++++++++++++--------------- kafka/protocol/api_versions.py | 4 +- test/test_conn.py | 17 +- 4 files changed, 238 insertions(+), 199 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 2597fff61..6fe47c6f7 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -303,7 +303,7 @@ def _can_connect(self, node_id): def _conn_state_change(self, node_id, sock, conn): with self._lock: - if conn.connecting(): + if conn.state is ConnectionStates.CONNECTING: # SSL connections can enter this state 2x (second during Handshake) if node_id not in self._connecting: self._connecting.add(node_id) @@ -315,7 +315,19 @@ def _conn_state_change(self, node_id, sock, conn): if self.cluster.is_bootstrap(node_id): self._last_bootstrap = time.time() - elif conn.connected(): + elif conn.state is ConnectionStates.API_VERSIONS_SEND: + try: + self._selector.register(sock, selectors.EVENT_WRITE, conn) + except KeyError: + self._selector.modify(sock, selectors.EVENT_WRITE, conn) + + elif conn.state in (ConnectionStates.API_VERSIONS_RECV, ConnectionStates.AUTHENTICATING): + try: + self._selector.register(sock, selectors.EVENT_READ, conn) + except KeyError: + self._selector.modify(sock, selectors.EVENT_READ, conn) + + elif conn.state is ConnectionStates.CONNECTED: log.debug("Node %s connected", node_id) if node_id in self._connecting: self._connecting.remove(node_id) @@ -332,6 +344,8 @@ def _conn_state_change(self, node_id, sock, conn): if self.cluster.is_bootstrap(node_id): self._bootstrap_fails = 0 + if self._api_versions is None: + self._api_versions = conn._api_versions else: for node_id in list(self._conns.keys()): @@ -970,15 +984,14 @@ def refresh_done(val_or_error): def get_api_versions(self): """Return the ApiVersions map, if available. - Note: A call to check_version must previously have succeeded and returned - version 0.10.0 or later + Note: Only available after bootstrap; requires broker version 0.10.0 or later. Returns: a map of dict mapping {api_key : (min_version, max_version)}, or None if ApiVersion is not supported by the kafka cluster. """ return self._api_versions - def check_version(self, node_id=None, timeout=None, strict=False): + def check_version(self, node_id=None, timeout=None, **kwargs): """Attempt to guess the version of a Kafka broker. Keyword Arguments: @@ -994,50 +1007,45 @@ def check_version(self, node_id=None, timeout=None, strict=False): Raises: NodeNotReadyError (if node_id is provided) NoBrokersAvailable (if node_id is None) - UnrecognizedBrokerVersion: please file bug if seen! - AssertionError (if strict=True): please file bug if seen! """ timeout = timeout or (self.config['api_version_auto_timeout_ms'] / 1000) - self._lock.acquire() - end = time.time() + timeout - while time.time() < end: - - # It is possible that least_loaded_node falls back to bootstrap, - # which can block for an increasing backoff period - try_node = node_id or self.least_loaded_node() - if try_node is None: - self._lock.release() - raise Errors.NoBrokersAvailable() - if not self._init_connect(try_node): - if try_node == node_id: - raise Errors.NodeNotReadyError("Connection failed to %s" % node_id) - else: + with self._lock: + end = time.time() + timeout + while time.time() < end: + time_remaining = max(end - time.time(), 0) + if node_id is not None and self.connection_delay(node_id) > 0: + sleep_time = min(time_remaining, self.connection_delay(node_id) / 1000.0) + if sleep_time > 0: + time.sleep(sleep_time) continue - - conn = self._conns[try_node] - - # We will intentionally cause socket failures - # These should not trigger metadata refresh - self._refresh_on_disconnects = False - try: - remaining = end - time.time() - version = conn.check_version(timeout=remaining, strict=strict, topics=list(self.config['bootstrap_topics_filter'])) - if not self._api_versions: - self._api_versions = conn.get_api_versions() - self._lock.release() - return version - except Errors.NodeNotReadyError: - # Only raise to user if this is a node-specific request + try_node = node_id or self.least_loaded_node() + if try_node is None: + sleep_time = min(time_remaining, self.least_loaded_node_refresh_ms() / 1000.0) + if sleep_time > 0: + log.warning('No node available during check_version; sleeping %.2f secs', sleep_time) + time.sleep(sleep_time) + continue + log.debug('Attempting to check version with node %s', try_node) + if not self._init_connect(try_node): + if try_node == node_id: + raise Errors.NodeNotReadyError("Connection failed to %s" % node_id) + else: + continue + conn = self._conns[try_node] + + while conn.connecting() and time.time() < end: + timeout_ms = min((end - time.time()) * 1000, 200) + self.poll(timeout_ms=timeout_ms) + + if conn._api_version is not None: + return conn._api_version + + # Timeout + else: if node_id is not None: - self._lock.release() - raise - finally: - self._refresh_on_disconnects = True - - # Timeout - else: - self._lock.release() - raise Errors.NoBrokersAvailable() + raise Errors.NodeNotReadyError(node_id) + else: + raise Errors.NoBrokersAvailable() def api_version(self, operation, max_version=None): """Find the latest version of the protocol operation supported by both @@ -1063,7 +1071,7 @@ def api_version(self, operation, max_version=None): broker_api_versions = self._api_versions api_key = operation[0].API_KEY if broker_api_versions is None or api_key not in broker_api_versions: - raise IncompatibleBrokerVersion( + raise Errors.IncompatibleBrokerVersion( "Kafka broker does not support the '{}' Kafka protocol." .format(operation[0].__name__)) broker_min_version, broker_max_version = broker_api_versions[api_key] @@ -1071,7 +1079,7 @@ def api_version(self, operation, max_version=None): if version < broker_min_version: # max library version is less than min broker version. Currently, # no Kafka versions specify a min msg version. Maybe in the future? - raise IncompatibleBrokerVersion( + raise Errors.IncompatibleBrokerVersion( "No version of the '{}' Kafka protocol is supported by both the client and broker." .format(operation[0].__name__)) return version diff --git a/kafka/conn.py b/kafka/conn.py index 6aa20117e..fd6943171 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -24,8 +24,11 @@ from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate from kafka.oauth.abstract import AbstractTokenProvider -from kafka.protocol.admin import SaslHandShakeRequest, DescribeAclsRequest, DescribeClientQuotasRequest +from kafka.protocol.admin import DescribeAclsRequest, DescribeClientQuotasRequest, ListGroupsRequest, SaslHandShakeRequest +from kafka.protocol.api_versions import ApiVersionsRequest +from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.commit import OffsetFetchRequest +from kafka.protocol.find_coordinator import FindCoordinatorRequest from kafka.protocol.list_offsets import ListOffsetsRequest from kafka.protocol.produce import ProduceRequest from kafka.protocol.metadata import MetadataRequest @@ -92,12 +95,13 @@ class SSLWantWriteError(Exception): class ConnectionStates(object): - DISCONNECTING = '' DISCONNECTED = '' CONNECTING = '' HANDSHAKE = '' CONNECTED = '' AUTHENTICATING = '' + API_VERSIONS_SEND = '' + API_VERSIONS_RECV = '' class BrokerConnection(object): @@ -169,7 +173,7 @@ class BrokerConnection(object): Default: None api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker - api version. Only applies if api_version is None + api version. Only applies if api_version is None. Default: 2000. selector (selectors.BaseSelector): Provide a specific selector implementation to use for I/O multiplexing. Default: selectors.DefaultSelector @@ -215,6 +219,7 @@ class BrokerConnection(object): 'ssl_password': None, 'ssl_ciphers': None, 'api_version': None, + 'api_version_auto_timeout_ms': 2000, 'selector': selectors.DefaultSelector, 'state_change_callback': lambda node_id, sock, conn: True, 'metrics': None, @@ -228,6 +233,12 @@ class BrokerConnection(object): } SECURITY_PROTOCOLS = ('PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL') SASL_MECHANISMS = ('PLAIN', 'GSSAPI', 'OAUTHBEARER', "SCRAM-SHA-256", "SCRAM-SHA-512") + VERSION_CHECKS = ( + ((0, 9), ListGroupsRequest[0]()), + ((0, 8, 2), FindCoordinatorRequest[0]('kafka-python-default-group')), + ((0, 8, 1), OffsetFetchRequest[0]('kafka-python-default-group', [])), + ((0, 8, 0), MetadataRequest[0]([])), + ) def __init__(self, host, port, afi, **configs): self.host = host @@ -236,6 +247,9 @@ def __init__(self, host, port, afi, **configs): self._sock_afi = afi self._sock_addr = None self._api_versions = None + self._api_version = None + self._check_version_idx = None + self._api_versions_idx = 2 self._throttle_time = None self.config = copy.copy(self.DEFAULT_CONFIG) @@ -301,6 +315,7 @@ def __init__(self, host, port, afi, **configs): self._ssl_context = None if self.config['ssl_context'] is not None: self._ssl_context = self.config['ssl_context'] + self._api_versions_future = None self._sasl_auth_future = None self.last_attempt = 0 self._gai = [] @@ -404,17 +419,9 @@ def connect(self): self.config['state_change_callback'](self.node_id, self._sock, self) # _wrap_ssl can alter the connection state -- disconnects on failure self._wrap_ssl() - - elif self.config['security_protocol'] == 'SASL_PLAINTEXT': - log.debug('%s: initiating SASL authentication', self) - self.state = ConnectionStates.AUTHENTICATING - self.config['state_change_callback'](self.node_id, self._sock, self) - else: - # security_protocol PLAINTEXT - log.info('%s: Connection complete.', self) - self.state = ConnectionStates.CONNECTED - self._reset_reconnect_backoff() + log.debug('%s: checking broker Api Versions', self) + self.state = ConnectionStates.API_VERSIONS_SEND self.config['state_change_callback'](self.node_id, self._sock, self) # Connection failed @@ -433,15 +440,25 @@ def connect(self): if self.state is ConnectionStates.HANDSHAKE: if self._try_handshake(): log.debug('%s: completed SSL handshake.', self) - if self.config['security_protocol'] == 'SASL_SSL': - log.debug('%s: initiating SASL authentication', self) - self.state = ConnectionStates.AUTHENTICATING - else: - log.info('%s: Connection complete.', self) - self.state = ConnectionStates.CONNECTED - self._reset_reconnect_backoff() + log.debug('%s: checking broker Api Versions', self) + self.state = ConnectionStates.API_VERSIONS_SEND self.config['state_change_callback'](self.node_id, self._sock, self) + if self.state in (ConnectionStates.API_VERSIONS_SEND, ConnectionStates.API_VERSIONS_RECV): + if self._try_api_versions_check(): + # _try_api_versions_check has side-effects: possibly disconnected on socket errors + if self.state in (ConnectionStates.API_VERSIONS_SEND, ConnectionStates.API_VERSIONS_RECV): + if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'): + log.debug('%s: initiating SASL authentication', self) + self.state = ConnectionStates.AUTHENTICATING + self.config['state_change_callback'](self.node_id, self._sock, self) + else: + # security_protocol PLAINTEXT + log.info('%s: Connection complete.', self) + self.state = ConnectionStates.CONNECTED + self._reset_reconnect_backoff() + self.config['state_change_callback'](self.node_id, self._sock, self) + if self.state is ConnectionStates.AUTHENTICATING: assert self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL') if self._try_authenticate(): @@ -522,6 +539,87 @@ def _try_handshake(self): return False + def _try_api_versions_check(self): + if self._api_versions_future is None: + if self.config['api_version'] is not None: + self._api_version = self.config['api_version'] + self._api_versions = BROKER_API_VERSIONS[self._api_version] + return True + elif self._check_version_idx is None: + request = ApiVersionsRequest[self._api_versions_idx]() + future = Future() + response = self._send(request, blocking=True, request_timeout_ms=(self.config['api_version_auto_timeout_ms'] * 0.8)) + response.add_callback(self._handle_api_versions_response, future) + response.add_errback(self._handle_api_versions_failure, future) + self._api_versions_future = future + self.state = ConnectionStates.API_VERSIONS_RECV + self.config['state_change_callback'](self.node_id, self._sock, self) + elif self._check_version_idx < len(self.VERSION_CHECKS): + version, request = self.VERSION_CHECKS[self._check_version_idx] + future = Future() + response = self._send(request, blocking=True, request_timeout_ms=(self.config['api_version_auto_timeout_ms'] * 0.8)) + response.add_callback(self._handle_check_version_response, future, version) + response.add_errback(self._handle_check_version_failure, future) + self._api_versions_future = future + self.state = ConnectionStates.API_VERSIONS_RECV + self.config['state_change_callback'](self.node_id, self._sock, self) + else: + raise 'Unable to determine broker version.' + + for r, f in self.recv(): + f.success(r) + + # A connection error during blocking send could trigger close() which will reset the future + if self._api_versions_future is None: + return False + elif self._api_versions_future.failed(): + ex = self._api_versions_future.exception + if not isinstance(ex, Errors.KafkaConnectionError): + raise ex + return self._api_versions_future.succeeded() + + def _handle_api_versions_response(self, future, response): + error_type = Errors.for_code(response.error_code) + # if error_type i UNSUPPORTED_VERSION: retry w/ latest version from response + if error_type is not Errors.NoError: + future.failure(error_type()) + if error_type is Errors.UnsupportedVersionError: + self._api_versions_idx -= 1 + if self._api_versions_idx >= 0: + self._api_versions_future = None + self.state = ConnectionStates.API_VERSIONS_SEND + self.config['state_change_callback'](self.node_id, self._sock, self) + else: + self.close(error=error_type()) + return + self._api_versions = dict([ + (api_key, (min_version, max_version)) + for api_key, min_version, max_version in response.api_versions + ]) + self._api_version = self._infer_broker_version_from_api_versions(self._api_versions) + log.info('Broker version identified as %s', '.'.join(map(str, self._api_version))) + future.success(self._api_version) + self.connect() + + def _handle_api_versions_failure(self, future, ex): + future.failure(ex) + self._check_version_idx = 0 + # after failure connection is closed, so state should already be DISCONNECTED + + def _handle_check_version_response(self, future, version, _response): + log.info('Broker version identified as %s', '.'.join(map(str, version))) + log.info('Set configuration api_version=%s to skip auto' + ' check_version requests on startup', version) + self._api_versions = BROKER_API_VERSIONS[version] + self._api_version = version + future.success(version) + self.connect() + + def _handle_check_version_failure(self, future, ex): + future.failure(ex) + self._check_version_idx += 1 + # after failure connection is closed, so state should already be DISCONNECTED + def _try_authenticate(self): assert self.config['api_version'] is None or self.config['api_version'] >= (0, 10, 0) @@ -529,7 +627,7 @@ def _try_authenticate(self): # Build a SaslHandShakeRequest message request = SaslHandShakeRequest[0](self.config['sasl_mechanism']) future = Future() - sasl_response = self._send(request) + sasl_response = self._send(request, blocking=True) sasl_response.add_callback(self._handle_sasl_handshake_response, future) sasl_response.add_errback(lambda f, e: f.failure(e), future) self._sasl_auth_future = future @@ -554,23 +652,28 @@ def _handle_sasl_handshake_response(self, future, response): return future.failure(error_type(self)) if self.config['sasl_mechanism'] not in response.enabled_mechanisms: - return future.failure( + future.failure( Errors.UnsupportedSaslMechanismError( 'Kafka broker does not support %s sasl mechanism. Enabled mechanisms are: %s' % (self.config['sasl_mechanism'], response.enabled_mechanisms))) elif self.config['sasl_mechanism'] == 'PLAIN': - return self._try_authenticate_plain(future) + self._try_authenticate_plain(future) elif self.config['sasl_mechanism'] == 'GSSAPI': - return self._try_authenticate_gssapi(future) + self._try_authenticate_gssapi(future) elif self.config['sasl_mechanism'] == 'OAUTHBEARER': - return self._try_authenticate_oauth(future) + self._try_authenticate_oauth(future) elif self.config['sasl_mechanism'].startswith("SCRAM-SHA-"): - return self._try_authenticate_scram(future) + self._try_authenticate_scram(future) else: - return future.failure( + future.failure( Errors.UnsupportedSaslMechanismError( 'kafka-python does not support SASL mechanism %s' % self.config['sasl_mechanism'])) + assert future.is_done, 'SASL future not complete after mechanism processing!' + if future.failed(): + self.close(error=future.exception) + else: + self.connect() def _send_bytes(self, data): """Send some data via non-blocking IO @@ -901,7 +1004,17 @@ def connecting(self): different states, such as SSL handshake, authorization, etc).""" return self.state in (ConnectionStates.CONNECTING, ConnectionStates.HANDSHAKE, - ConnectionStates.AUTHENTICATING) + ConnectionStates.AUTHENTICATING, + ConnectionStates.API_VERSIONS_SEND, + ConnectionStates.API_VERSIONS_RECV) + + def initializing(self): + """Returns True if socket is connected but full connection is not complete. + During this time the connection may send api requests to the broker to + check api versions and perform SASL authentication.""" + return self.state in (ConnectionStates.AUTHENTICATING, + ConnectionStates.API_VERSIONS_SEND, + ConnectionStates.API_VERSIONS_RECV) def disconnected(self): """Return True iff socket is closed""" @@ -949,6 +1062,7 @@ def close(self, error=None): return log.log(logging.ERROR if error else logging.INFO, '%s: Closing connection. %s', self, error or '') self._update_reconnect_backoff() + self._api_versions_future = None self._sasl_auth_future = None self._protocol = KafkaProtocol( client_id=self.config['client_id'], @@ -975,8 +1089,7 @@ def close(self, error=None): def _can_send_recv(self): """Return True iff socket is ready for requests / responses""" - return self.state in (ConnectionStates.AUTHENTICATING, - ConnectionStates.CONNECTED) + return self.connected() or self.initializing() def send(self, request, blocking=True, request_timeout_ms=None): """Queue request for async network send, return Future() @@ -1218,16 +1331,6 @@ def next_ifr_request_timeout_ms(self): else: return float('inf') - def _handle_api_versions_response(self, response): - error_type = Errors.for_code(response.error_code) - if error_type is not Errors.NoError: - return False - self._api_versions = dict([ - (api_key, (min_version, max_version)) - for api_key, min_version, max_version in response.api_versions - ]) - return self._api_versions - def get_api_versions(self): if self._api_versions is not None: return self._api_versions @@ -1242,6 +1345,20 @@ def _infer_broker_version_from_api_versions(self, api_versions): test_cases = [ # format (, ) # Make sure to update consumer_integration test check when adding newer versions. + # ((3, 9), FetchRequest[17]), + # ((3, 8), ProduceRequest[11]), + # ((3, 7), FetchRequest[16]), + # ((3, 6), AddPartitionsToTxnRequest[4]), + # ((3, 5), FetchRequest[15]), + # ((3, 4), StopReplicaRequest[3]), # broker-internal api... + # ((3, 3), DescribeAclsRequest[3]), + # ((3, 2), JoinGroupRequest[9]), + # ((3, 1), FetchRequest[13]), + # ((3, 0), ListOffsetsRequest[7]), + # ((2, 8), ProduceRequest[9]), + # ((2, 7), FetchRequest[12]), + # ((2, 6), ListGroupsRequest[4]), + # ((2, 5), JoinGroupRequest[7]), ((2, 6), DescribeClientQuotasRequest[0]), ((2, 5), DescribeAclsRequest[2]), ((2, 4), ProduceRequest[8]), @@ -1268,121 +1385,24 @@ def _infer_broker_version_from_api_versions(self, api_versions): # so if all else fails, choose that return (0, 10, 0) - def check_version(self, timeout=2, strict=False, topics=[]): + def check_version(self, timeout=2, **kwargs): """Attempt to guess the broker version. + Keyword Arguments: + timeout (numeric, optional): Maximum number of seconds to block attempting + to connect and check version. Default 2 + Note: This is a blocking call. Returns: version tuple, i.e. (3, 9), (2, 4), etc ... + + Raises: NodeNotReadyError on timeout """ timeout_at = time.time() + timeout - log.info('Probing node %s broker version', self.node_id) - # Monkeypatch some connection configurations to avoid timeouts - override_config = { - 'request_timeout_ms': timeout * 1000, - 'max_in_flight_requests_per_connection': 5 - } - stashed = {} - for key in override_config: - stashed[key] = self.config[key] - self.config[key] = override_config[key] - - def reset_override_configs(): - for key in stashed: - self.config[key] = stashed[key] - - # kafka kills the connection when it doesn't recognize an API request - # so we can send a test request and then follow immediately with a - # vanilla MetadataRequest. If the server did not recognize the first - # request, both will be failed with a ConnectionError that wraps - # socket.error (32, 54, or 104) - from kafka.protocol.admin import ListGroupsRequest - from kafka.protocol.api_versions import ApiVersionsRequest - from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS - from kafka.protocol.commit import OffsetFetchRequest - from kafka.protocol.find_coordinator import FindCoordinatorRequest - - test_cases = [ - # All cases starting from 0.10 will be based on ApiVersionsResponse - ((0, 11), ApiVersionsRequest[1]()), - ((0, 10, 0), ApiVersionsRequest[0]()), - ((0, 9), ListGroupsRequest[0]()), - ((0, 8, 2), FindCoordinatorRequest[0]('kafka-python-default-group')), - ((0, 8, 1), OffsetFetchRequest[0]('kafka-python-default-group', [])), - ((0, 8, 0), MetadataRequest[0](topics)), - ] - - for version, request in test_cases: - if not self.connect_blocking(timeout_at - time.time()): - reset_override_configs() - raise Errors.NodeNotReadyError() - f = self.send(request) - # HACK: sleeping to wait for socket to send bytes - time.sleep(0.1) - # when broker receives an unrecognized request API - # it abruptly closes our socket. - # so we attempt to send a second request immediately - # that we believe it will definitely recognize (metadata) - # the attempt to write to a disconnected socket should - # immediately fail and allow us to infer that the prior - # request was unrecognized - mr = self.send(MetadataRequest[0](topics)) - - if not (f.is_done and mr.is_done) and self._sock is not None: - selector = self.config['selector']() - selector.register(self._sock, selectors.EVENT_READ) - while not (f.is_done and mr.is_done): - selector.select(1) - for response, future in self.recv(): - future.success(response) - selector.close() - - if f.succeeded(): - if version >= (0, 10, 0): - # Starting from 0.10 kafka broker we determine version - # by looking at ApiVersionsResponse - api_versions = self._handle_api_versions_response(f.value) - if not api_versions: - continue - version = self._infer_broker_version_from_api_versions(api_versions) - else: - if version not in BROKER_API_VERSIONS: - raise Errors.UnrecognizedBrokerVersion(version) - self._api_versions = BROKER_API_VERSIONS[version] - log.info('Broker version identified as %s', '.'.join(map(str, version))) - log.info('Set configuration api_version=%s to skip auto' - ' check_version requests on startup', version) - break - - # Only enable strict checking to verify that we understand failure - # modes. For most users, the fact that the request failed should be - # enough to rule out a particular broker version. - if strict: - # If the socket flush hack did not work (which should force the - # connection to close and fail all pending requests), then we - # get a basic Request Timeout. This is not ideal, but we'll deal - if isinstance(f.exception, Errors.RequestTimedOutError): - pass - - # 0.9 brokers do not close the socket on unrecognized api - # requests (bug...). In this case we expect to see a correlation - # id mismatch - elif (isinstance(f.exception, Errors.CorrelationIdError) and - version > (0, 9)): - pass - elif six.PY2: - assert isinstance(f.exception.args[0], socket.error) - assert f.exception.args[0].errno in (32, 54, 104) - else: - assert isinstance(f.exception.args[0], ConnectionError) - log.info("Broker is not v%s -- it did not recognize %s", - version, request.__class__.__name__) + if not self.connect_blocking(timeout_at - time.time()): + raise Errors.NodeNotReadyError() else: - reset_override_configs() - raise Errors.UnrecognizedBrokerVersion() - - reset_override_configs() - return version + return self._api_version def __str__(self): return "" % ( diff --git a/kafka/protocol/api_versions.py b/kafka/protocol/api_versions.py index 9a782928b..dc0aa588e 100644 --- a/kafka/protocol/api_versions.py +++ b/kafka/protocol/api_versions.py @@ -76,8 +76,8 @@ class ApiVersionsRequest_v1(Request): class ApiVersionsRequest_v2(Request): API_KEY = 18 API_VERSION = 2 - RESPONSE_TYPE = ApiVersionsResponse_v1 - SCHEMA = ApiVersionsRequest_v0.SCHEMA + RESPONSE_TYPE = ApiVersionsResponse_v2 + SCHEMA = ApiVersionsRequest_v1.SCHEMA ApiVersionsRequest = [ diff --git a/test/test_conn.py b/test/test_conn.py index 47f5c428e..959cbb4dc 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -15,6 +15,13 @@ 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): @@ -27,13 +34,16 @@ def dns_lookup(mocker): def _socket(mocker): socket = mocker.MagicMock() socket.connect_ex.return_value = 0 + socket.send.side_effect = lambda d: len(d) + socket.recv.side_effect = BlockingIOError("mocked recv") mocker.patch('socket.socket', return_value=socket) return socket @pytest.fixture -def conn(_socket, dns_lookup): +def conn(_socket, dns_lookup, mocker): conn = BrokerConnection('localhost', 9092, socket.AF_INET) + mocker.patch.object(conn, '_try_api_versions_check', return_value=True) return conn @@ -217,12 +227,13 @@ def test_recv_disconnected(_socket, conn): conn.send(req) # Empty data on recv means the socket is disconnected + _socket.recv.side_effect = None _socket.recv.return_value = b'' # Attempt to receive should mark connection as disconnected - assert conn.connected() + assert conn.connected(), 'Not connected: %s' % conn.state conn.recv() - assert conn.disconnected() + assert conn.disconnected(), 'Not disconnected: %s' % conn.state def test_recv(_socket, conn): From f046b0cfe180c6b01b8937ce98211dc5df70cea4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 10 Mar 2025 17:33:19 -0700 Subject: [PATCH 003/205] Use thread-specific client_id in test_group --- test/test_consumer_group.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index c1ef978e2..ec0f41832 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -56,6 +56,7 @@ def consumer_thread(i): consumers[i] = KafkaConsumer(topic, bootstrap_servers=connect_str, group_id=group_id, + client_id="consumer_thread-%s" % i, heartbeat_interval_ms=500) while not stop[i].is_set(): for tp, records in six.itervalues(consumers[i].poll(timeout_ms=200)): From 8320cc89436e4b1de50f34e11bcc35fe3c82bfbb Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 11:54:44 -0700 Subject: [PATCH 004/205] Refactor Sasl authentication with SaslMechanism abstract base class; support SaslAuthenticate (#2515) --- kafka/conn.py | 337 +++++++--------------------- kafka/protocol/admin.py | 35 --- kafka/protocol/sasl_authenticate.py | 42 ++++ kafka/protocol/sasl_handshake.py | 39 ++++ kafka/sasl/__init__.py | 26 +++ kafka/sasl/abc.py | 27 +++ kafka/sasl/gssapi.py | 73 ++++++ kafka/sasl/oauth.py | 39 ++++ kafka/sasl/plain.py | 36 +++ kafka/{ => sasl}/scram.py | 75 +++++-- 10 files changed, 422 insertions(+), 307 deletions(-) create mode 100644 kafka/protocol/sasl_authenticate.py create mode 100644 kafka/protocol/sasl_handshake.py create mode 100644 kafka/sasl/__init__.py create mode 100644 kafka/sasl/abc.py create mode 100644 kafka/sasl/gssapi.py create mode 100644 kafka/sasl/oauth.py create mode 100644 kafka/sasl/plain.py rename kafka/{ => sasl}/scram.py (52%) diff --git a/kafka/conn.py b/kafka/conn.py index fd6943171..988f4399f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -24,18 +24,20 @@ from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate from kafka.oauth.abstract import AbstractTokenProvider -from kafka.protocol.admin import DescribeAclsRequest, DescribeClientQuotasRequest, ListGroupsRequest, SaslHandShakeRequest +from kafka.protocol.admin import DescribeAclsRequest, DescribeClientQuotasRequest, ListGroupsRequest from kafka.protocol.api_versions import ApiVersionsRequest from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.commit import OffsetFetchRequest +from kafka.protocol.fetch import FetchRequest from kafka.protocol.find_coordinator import FindCoordinatorRequest from kafka.protocol.list_offsets import ListOffsetsRequest -from kafka.protocol.produce import ProduceRequest from kafka.protocol.metadata import MetadataRequest -from kafka.protocol.fetch import FetchRequest from kafka.protocol.parser import KafkaProtocol +from kafka.protocol.produce import ProduceRequest +from kafka.protocol.sasl_authenticate import SaslAuthenticateRequest +from kafka.protocol.sasl_handshake import SaslHandshakeRequest from kafka.protocol.types import Int32, Int8 -from kafka.scram import ScramClient +from kafka.sasl import get_sasl_mechanism from kafka.version import __version__ @@ -48,10 +50,6 @@ DEFAULT_KAFKA_PORT = 9092 -SASL_QOP_AUTH = 1 -SASL_QOP_AUTH_INT = 2 -SASL_QOP_AUTH_CONF = 4 - try: import ssl ssl_available = True @@ -77,15 +75,6 @@ class SSLWantReadError(Exception): class SSLWantWriteError(Exception): pass -# needed for SASL_GSSAPI authentication: -try: - import gssapi - from gssapi.raw.misc import GSSError -except (ImportError, OSError): - #no gssapi available, will disable gssapi mechanism - gssapi = None - GSSError = None - AFI_NAMES = { socket.AF_UNSPEC: "unspecified", @@ -232,7 +221,6 @@ class BrokerConnection(object): 'sasl_oauth_token_provider': None } SECURITY_PROTOCOLS = ('PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL') - SASL_MECHANISMS = ('PLAIN', 'GSSAPI', 'OAUTHBEARER', "SCRAM-SHA-256", "SCRAM-SHA-512") VERSION_CHECKS = ( ((0, 9), ListGroupsRequest[0]()), ((0, 8, 2), FindCoordinatorRequest[0]('kafka-python-default-group')), @@ -271,26 +259,13 @@ def __init__(self, host, port, afi, **configs): assert self.config['security_protocol'] in self.SECURITY_PROTOCOLS, ( 'security_protocol must be in ' + ', '.join(self.SECURITY_PROTOCOLS)) + self._sasl_mechanism = None if self.config['security_protocol'] in ('SSL', 'SASL_SSL'): assert ssl_available, "Python wasn't built with SSL support" if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'): - assert self.config['sasl_mechanism'] in self.SASL_MECHANISMS, ( - 'sasl_mechanism must be in ' + ', '.join(self.SASL_MECHANISMS)) - if self.config['sasl_mechanism'] in ('PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512'): - assert self.config['sasl_plain_username'] is not None, ( - 'sasl_plain_username required for PLAIN or SCRAM sasl' - ) - assert self.config['sasl_plain_password'] is not None, ( - 'sasl_plain_password required for PLAIN or SCRAM sasl' - ) - if self.config['sasl_mechanism'] == 'GSSAPI': - assert gssapi is not None, 'GSSAPI lib not available' - assert self.config['sasl_kerberos_service_name'] is not None, 'sasl_kerberos_service_name required for GSSAPI sasl' - if self.config['sasl_mechanism'] == 'OAUTHBEARER': - token_provider = self.config['sasl_oauth_token_provider'] - assert token_provider is not None, 'sasl_oauth_token_provider required for OAUTHBEARER sasl' - assert callable(getattr(token_provider, "token", None)), 'sasl_oauth_token_provider must implement method #token()' + self._sasl_mechanism = get_sasl_mechanism(self.config['sasl_mechanism'])(**self.config) + # This is not a general lock / this class is not generally thread-safe yet # However, to avoid pushing responsibility for maintaining # per-connection locks to the upstream client, we will use this lock to @@ -620,12 +595,22 @@ def _handle_check_version_failure(self, future, ex): self._check_version_idx += 1 # after failure connection is closed, so state should already be DISCONNECTED - def _try_authenticate(self): - assert self.config['api_version'] is None or self.config['api_version'] >= (0, 10, 0) + def _sasl_handshake_version(self): + if self._api_versions is None: + raise RuntimeError('_api_versions not set') + if SaslHandshakeRequest[0].API_KEY not in self._api_versions: + raise Errors.UnsupportedVersionError('SaslHandshake') + + # Build a SaslHandshakeRequest message + min_version, max_version = self._api_versions[SaslHandshakeRequest[0].API_KEY] + if min_version > 1: + raise Errors.UnsupportedVersionError('SaslHandshake %s' % min_version) + return min(max_version, 1) + def _try_authenticate(self): if self._sasl_auth_future is None: - # Build a SaslHandShakeRequest message - request = SaslHandShakeRequest[0](self.config['sasl_mechanism']) + version = self._sasl_handshake_version() + request = SaslHandshakeRequest[version](self.config['sasl_mechanism']) future = Future() sasl_response = self._send(request, blocking=True) sasl_response.add_callback(self._handle_sasl_handshake_response, future) @@ -656,19 +641,9 @@ def _handle_sasl_handshake_response(self, future, response): Errors.UnsupportedSaslMechanismError( 'Kafka broker does not support %s sasl mechanism. Enabled mechanisms are: %s' % (self.config['sasl_mechanism'], response.enabled_mechanisms))) - elif self.config['sasl_mechanism'] == 'PLAIN': - self._try_authenticate_plain(future) - elif self.config['sasl_mechanism'] == 'GSSAPI': - self._try_authenticate_gssapi(future) - elif self.config['sasl_mechanism'] == 'OAUTHBEARER': - self._try_authenticate_oauth(future) - elif self.config['sasl_mechanism'].startswith("SCRAM-SHA-"): - self._try_authenticate_scram(future) else: - future.failure( - Errors.UnsupportedSaslMechanismError( - 'kafka-python does not support SASL mechanism %s' % - self.config['sasl_mechanism'])) + self._sasl_authenticate(future) + assert future.is_done, 'SASL future not complete after mechanism processing!' if future.failed(): self.close(error=future.exception) @@ -727,224 +702,72 @@ def _recv_bytes_blocking(self, n): finally: self._sock.settimeout(0.0) - def _try_authenticate_plain(self, future): - if self.config['security_protocol'] == 'SASL_PLAINTEXT': - log.warning('%s: Sending username and password in the clear', self) - - data = b'' - # Send PLAIN credentials per RFC-4616 - msg = bytes('\0'.join([self.config['sasl_plain_username'], - self.config['sasl_plain_username'], - self.config['sasl_plain_password']]).encode('utf-8')) - size = Int32.encode(len(msg)) - - err = None - close = False - with self._lock: - if not self._can_send_recv(): - err = Errors.NodeNotReadyError(str(self)) - close = False - else: - try: - self._send_bytes_blocking(size + msg) - - # The server will send a zero sized message (that is Int32(0)) on success. - # The connection is closed on failure - data = self._recv_bytes_blocking(4) - - except (ConnectionError, TimeoutError) as e: - log.exception("%s: Error receiving reply from server", self) - err = Errors.KafkaConnectionError("%s: %s" % (self, e)) - close = True - - if err is not None: - if close: + def _send_sasl_authenticate(self, sasl_auth_bytes): + version = self._sasl_handshake_version() + if version == 1: + request = SaslAuthenticateRequest[0](sasl_auth_bytes) + self._send(request, blocking=True) + else: + try: + self._send_bytes_blocking(Int32.encode(len(sasl_auth_bytes)) + sasl_auth_bytes) + except (ConnectionError, TimeoutError) as e: + log.exception("%s: Error sending sasl auth bytes to server", self) + err = Errors.KafkaConnectionError("%s: %s" % (self, e)) self.close(error=err) - return future.failure(err) - - if data != b'\x00\x00\x00\x00': - error = Errors.AuthenticationFailedError('Unrecognized response during authentication') - return future.failure(error) - - log.info('%s: Authenticated as %s via PLAIN', self, self.config['sasl_plain_username']) - return future.success(True) - - def _try_authenticate_scram(self, future): - if self.config['security_protocol'] == 'SASL_PLAINTEXT': - log.warning('%s: Exchanging credentials in the clear', self) - - scram_client = ScramClient( - self.config['sasl_plain_username'], self.config['sasl_plain_password'], self.config['sasl_mechanism'] - ) - - err = None - close = False - with self._lock: - if not self._can_send_recv(): - err = Errors.NodeNotReadyError(str(self)) - close = False - else: - try: - client_first = scram_client.first_message().encode('utf-8') - size = Int32.encode(len(client_first)) - self._send_bytes_blocking(size + client_first) - - (data_len,) = struct.unpack('>i', self._recv_bytes_blocking(4)) - server_first = self._recv_bytes_blocking(data_len).decode('utf-8') - scram_client.process_server_first_message(server_first) - client_final = scram_client.final_message().encode('utf-8') - size = Int32.encode(len(client_final)) - self._send_bytes_blocking(size + client_final) + def _recv_sasl_authenticate(self): + version = self._sasl_handshake_version() + # GSSAPI mechanism does not get a final recv in old non-framed mode + if version == 0 and self._sasl_mechanism.is_done(): + return b'' - (data_len,) = struct.unpack('>i', self._recv_bytes_blocking(4)) - server_final = self._recv_bytes_blocking(data_len).decode('utf-8') - scram_client.process_server_final_message(server_final) + try: + data = self._recv_bytes_blocking(4) + nbytes = Int32.decode(io.BytesIO(data)) + data += self._recv_bytes_blocking(nbytes) + except (ConnectionError, TimeoutError) as e: + log.exception("%s: Error receiving sasl auth bytes from server", self) + err = Errors.KafkaConnectionError("%s: %s" % (self, e)) + self.close(error=err) + return - except (ConnectionError, TimeoutError) as e: - log.exception("%s: Error receiving reply from server", self) - err = Errors.KafkaConnectionError("%s: %s" % (self, e)) - close = True + if version == 1: + ((correlation_id, response),) = self._protocol.receive_bytes(data) + (future, timestamp, _timeout) = self.in_flight_requests.pop(correlation_id) + latency_ms = (time.time() - timestamp) * 1000 + if self._sensors: + self._sensors.request_time.record(latency_ms) + log.debug('%s Response %d (%s ms): %s', self, correlation_id, latency_ms, response) - if err is not None: - if close: - self.close(error=err) - return future.failure(err) - - log.info( - '%s: Authenticated as %s via %s', self, self.config['sasl_plain_username'], self.config['sasl_mechanism'] - ) - return future.success(True) - - def _try_authenticate_gssapi(self, future): - kerberos_damin_name = self.config['sasl_kerberos_domain_name'] or self.host - auth_id = self.config['sasl_kerberos_service_name'] + '@' + kerberos_damin_name - gssapi_name = gssapi.Name( - auth_id, - name_type=gssapi.NameType.hostbased_service - ).canonicalize(gssapi.MechType.kerberos) - log.debug('%s: GSSAPI name: %s', self, gssapi_name) + error_type = Errors.for_code(response.error_code) + if error_type is not Errors.NoError: + log.error("%s: SaslAuthenticate error: %s (%s)", + self, error_type.__name__, response.error_message) + self.close(error=error_type(response.error_message)) + return + return response.auth_bytes + else: + # unframed bytes w/ SaslHandhake v0 + return data[4:] - err = None - close = False - with self._lock: + def _sasl_authenticate(self, future): + while not self._sasl_mechanism.is_done(): + send_token = self._sasl_mechanism.auth_bytes() + self._send_sasl_authenticate(send_token) if not self._can_send_recv(): - err = Errors.NodeNotReadyError(str(self)) - close = False - else: - # Establish security context and negotiate protection level - # For reference RFC 2222, section 7.2.1 - try: - # Exchange tokens until authentication either succeeds or fails - client_ctx = gssapi.SecurityContext(name=gssapi_name, usage='initiate') - received_token = None - while not client_ctx.complete: - # calculate an output token from kafka token (or None if first iteration) - output_token = client_ctx.step(received_token) - - # pass output token to kafka, or send empty response if the security - # context is complete (output token is None in that case) - if output_token is None: - self._send_bytes_blocking(Int32.encode(0)) - else: - msg = output_token - size = Int32.encode(len(msg)) - self._send_bytes_blocking(size + msg) - - # The server will send a token back. Processing of this token either - # establishes a security context, or it needs further token exchange. - # The gssapi will be able to identify the needed next step. - # The connection is closed on failure. - header = self._recv_bytes_blocking(4) - (token_size,) = struct.unpack('>i', header) - received_token = self._recv_bytes_blocking(token_size) - - # Process the security layer negotiation token, sent by the server - # once the security context is established. - - # unwraps message containing supported protection levels and msg size - msg = client_ctx.unwrap(received_token).message - # Kafka currently doesn't support integrity or confidentiality security layers, so we - # simply set QoP to 'auth' only (first octet). We reuse the max message size proposed - # by the server - msg = Int8.encode(SASL_QOP_AUTH & Int8.decode(io.BytesIO(msg[0:1]))) + msg[1:] - # add authorization identity to the response, GSS-wrap and send it - msg = client_ctx.wrap(msg + auth_id.encode(), False).message - size = Int32.encode(len(msg)) - self._send_bytes_blocking(size + msg) - - except (ConnectionError, TimeoutError) as e: - log.exception("%s: Error receiving reply from server", self) - err = Errors.KafkaConnectionError("%s: %s" % (self, e)) - close = True - except Exception as e: - err = e - close = True - - if err is not None: - if close: - self.close(error=err) - return future.failure(err) - - log.info('%s: Authenticated as %s via GSSAPI', self, gssapi_name) - return future.success(True) - - def _try_authenticate_oauth(self, future): - data = b'' + return future.failure(Errors.KafkaConnectionError("%s: Connection failure during Sasl Authenticate" % self)) - msg = bytes(self._build_oauth_client_request().encode("utf-8")) - size = Int32.encode(len(msg)) - - err = None - close = False - with self._lock: - if not self._can_send_recv(): - err = Errors.NodeNotReadyError(str(self)) - close = False + recv_token = self._recv_sasl_authenticate() + if recv_token is None: + return future.failure(Errors.KafkaConnectionError("%s: Connection failure during Sasl Authenticate" % self)) else: - try: - # Send SASL OAuthBearer request with OAuth token - self._send_bytes_blocking(size + msg) - - # The server will send a zero sized message (that is Int32(0)) on success. - # The connection is closed on failure - data = self._recv_bytes_blocking(4) - - except (ConnectionError, TimeoutError) as e: - log.exception("%s: Error receiving reply from server", self) - err = Errors.KafkaConnectionError("%s: %s" % (self, e)) - close = True - - if err is not None: - if close: - self.close(error=err) - return future.failure(err) - - if data != b'\x00\x00\x00\x00': - error = Errors.AuthenticationFailedError('Unrecognized response during authentication') - return future.failure(error) - - log.info('%s: Authenticated via OAuth', self) - return future.success(True) - - def _build_oauth_client_request(self): - token_provider = self.config['sasl_oauth_token_provider'] - return "n,,\x01auth=Bearer {}{}\x01\x01".format(token_provider.token(), self._token_extensions()) - - def _token_extensions(self): - """ - Return a string representation of the OPTIONAL key-value pairs that can be sent with an OAUTHBEARER - initial request. - """ - token_provider = self.config['sasl_oauth_token_provider'] + self._sasl_mechanism.receive(recv_token) - # Only run if the #extensions() method is implemented by the clients Token Provider class - # Builds up a string separated by \x01 via a dict of key value pairs - if callable(getattr(token_provider, "extensions", None)) and len(token_provider.extensions()) > 0: - msg = "\x01".join(["{}={}".format(k, v) for k, v in token_provider.extensions().items()]) - return "\x01" + msg + if self._sasl_mechanism.is_authenticated(): + log.info('%s: Authenticated via %s', self, self.config['sasl_mechanism']) + return future.success(True) else: - return "" + return future.failure(Errors.AuthenticationFailedError('Failed to authenticate via SASL %s' % self.config['sasl_mechanism'])) def blacked_out(self): """ diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index c237ef7e0..058325cb1 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -346,41 +346,6 @@ class DescribeGroupsRequest_v3(Request): ] -class SaslHandShakeResponse_v0(Response): - API_KEY = 17 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('enabled_mechanisms', Array(String('utf-8'))) - ) - - -class SaslHandShakeResponse_v1(Response): - API_KEY = 17 - API_VERSION = 1 - SCHEMA = SaslHandShakeResponse_v0.SCHEMA - - -class SaslHandShakeRequest_v0(Request): - API_KEY = 17 - API_VERSION = 0 - RESPONSE_TYPE = SaslHandShakeResponse_v0 - SCHEMA = Schema( - ('mechanism', String('utf-8')) - ) - - -class SaslHandShakeRequest_v1(Request): - API_KEY = 17 - API_VERSION = 1 - RESPONSE_TYPE = SaslHandShakeResponse_v1 - SCHEMA = SaslHandShakeRequest_v0.SCHEMA - - -SaslHandShakeRequest = [SaslHandShakeRequest_v0, SaslHandShakeRequest_v1] -SaslHandShakeResponse = [SaslHandShakeResponse_v0, SaslHandShakeResponse_v1] - - class DescribeAclsResponse_v0(Response): API_KEY = 29 API_VERSION = 0 diff --git a/kafka/protocol/sasl_authenticate.py b/kafka/protocol/sasl_authenticate.py new file mode 100644 index 000000000..528bb3cc6 --- /dev/null +++ b/kafka/protocol/sasl_authenticate.py @@ -0,0 +1,42 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Bytes, Int16, Int64, Schema, String + + +class SaslAuthenticateResponse_v0(Response): + API_KEY = 36 + API_VERSION = 0 + SCHEMA = Schema( + ('error_code', Int16), + ('error_message', String('utf-8')), + ('auth_bytes', Bytes)) + + +class SaslAuthenticateResponse_v1(Response): + API_KEY = 36 + API_VERSION = 1 + SCHEMA = Schema( + ('error_code', Int16), + ('error_message', String('utf-8')), + ('auth_bytes', Bytes), + ('session_lifetime_ms', Int64)) + + +class SaslAuthenticateRequest_v0(Request): + API_KEY = 36 + API_VERSION = 0 + RESPONSE_TYPE = SaslAuthenticateResponse_v0 + SCHEMA = Schema( + ('auth_bytes', Bytes)) + + +class SaslAuthenticateRequest_v1(Request): + API_KEY = 36 + API_VERSION = 1 + RESPONSE_TYPE = SaslAuthenticateResponse_v1 + SCHEMA = SaslAuthenticateRequest_v0.SCHEMA + + +SaslAuthenticateRequest = [SaslAuthenticateRequest_v0, SaslAuthenticateRequest_v1] +SaslAuthenticateResponse = [SaslAuthenticateResponse_v0, SaslAuthenticateResponse_v1] diff --git a/kafka/protocol/sasl_handshake.py b/kafka/protocol/sasl_handshake.py new file mode 100644 index 000000000..e91c856ca --- /dev/null +++ b/kafka/protocol/sasl_handshake.py @@ -0,0 +1,39 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Int16, Schema, String + + +class SaslHandshakeResponse_v0(Response): + API_KEY = 17 + API_VERSION = 0 + SCHEMA = Schema( + ('error_code', Int16), + ('enabled_mechanisms', Array(String('utf-8'))) + ) + + +class SaslHandshakeResponse_v1(Response): + API_KEY = 17 + API_VERSION = 1 + SCHEMA = SaslHandshakeResponse_v0.SCHEMA + + +class SaslHandshakeRequest_v0(Request): + API_KEY = 17 + API_VERSION = 0 + RESPONSE_TYPE = SaslHandshakeResponse_v0 + SCHEMA = Schema( + ('mechanism', String('utf-8')) + ) + + +class SaslHandshakeRequest_v1(Request): + API_KEY = 17 + API_VERSION = 1 + RESPONSE_TYPE = SaslHandshakeResponse_v1 + SCHEMA = SaslHandshakeRequest_v0.SCHEMA + + +SaslHandshakeRequest = [SaslHandshakeRequest_v0, SaslHandshakeRequest_v1] +SaslHandshakeResponse = [SaslHandshakeResponse_v0, SaslHandshakeResponse_v1] diff --git a/kafka/sasl/__init__.py b/kafka/sasl/__init__.py new file mode 100644 index 000000000..e36d1dfbd --- /dev/null +++ b/kafka/sasl/__init__.py @@ -0,0 +1,26 @@ +from __future__ import absolute_import + +from kafka.sasl.gssapi import SaslMechanismGSSAPI +from kafka.sasl.oauth import SaslMechanismOAuth +from kafka.sasl.plain import SaslMechanismPlain +from kafka.sasl.scram import SaslMechanismScram + + +SASL_MECHANISMS = {} + + +def register_sasl_mechanism(name, klass, overwrite=False): + if not overwrite and name in SASL_MECHANISMS: + raise ValueError('Sasl mechanism %s already defined!' % name) + SASL_MECHANISMS[name] = klass + + +def get_sasl_mechanism(name): + return SASL_MECHANISMS[name] + + +register_sasl_mechanism('GSSAPI', SaslMechanismGSSAPI) +register_sasl_mechanism('OAUTHBEARER', SaslMechanismOAuth) +register_sasl_mechanism('PLAIN', SaslMechanismPlain) +register_sasl_mechanism('SCRAM-SHA-256', SaslMechanismScram) +register_sasl_mechanism('SCRAM-SHA-512', SaslMechanismScram) diff --git a/kafka/sasl/abc.py b/kafka/sasl/abc.py new file mode 100644 index 000000000..7baef3b78 --- /dev/null +++ b/kafka/sasl/abc.py @@ -0,0 +1,27 @@ +from __future__ import absolute_import + +import abc + + +class SaslMechanism(object): + __metaclass__ = abc.ABCMeta + + @abc.abstractmethod + def __init__(self, **config): + pass + + @abc.abstractmethod + def auth_bytes(self): + pass + + @abc.abstractmethod + def receive(self, auth_bytes): + pass + + @abc.abstractmethod + def is_done(self): + pass + + @abc.abstractmethod + def is_authenticated(self): + pass diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py new file mode 100644 index 000000000..b40c37535 --- /dev/null +++ b/kafka/sasl/gssapi.py @@ -0,0 +1,73 @@ +from __future__ import absolute_import + +# needed for SASL_GSSAPI authentication: +try: + import gssapi + from gssapi.raw.misc import GSSError +except (ImportError, OSError): + #no gssapi available, will disable gssapi mechanism + gssapi = None + GSSError = None + +from kafka.sasl.abc import SaslMechanism + + +class SaslMechanismGSSAPI(SaslMechanism): + # Establish security context and negotiate protection level + # For reference RFC 2222, section 7.2.1 + + SASL_QOP_AUTH = 1 + SASL_QOP_AUTH_INT = 2 + SASL_QOP_AUTH_CONF = 4 + + def __init__(self, **config): + assert gssapi is not None, 'GSSAPI lib not available' + assert config['sasl_kerberos_service_name'] is not None, 'sasl_kerberos_service_name required for GSSAPI sasl' + self._is_done = False + self._is_authenticated = False + self.kerberos_damin_name = config['sasl_kerberos_domain_name'] or config['host'] + self.auth_id = config['sasl_kerberos_service_name'] + '@' + kerberos_damin_name + self.gssapi_name = gssapi.Name(auth_id, name_type=gssapi.NameType.hostbased_service).canonicalize(gssapi.MechType.kerberos) + self._client_ctx = gssapi.SecurityContext(name=self.gssapi_name, usage='initiate') + self._next_token = self._client_ctx.step(None) + + def auth_bytes(self): + # GSSAPI Auth does not have a final broker->client message + # so mark is_done after the final auth_bytes are provided + # in practice we'll still receive a response when using SaslAuthenticate + # but not when using the prior unframed approach. + if self._client_ctx.complete: + self._is_done = True + self._is_authenticated = True + return self._next_token or b'' + + def receive(self, auth_bytes): + if not self._client_ctx.complete: + # The server will send a token back. Processing of this token either + # establishes a security context, or it needs further token exchange. + # The gssapi will be able to identify the needed next step. + self._next_token = self._client_ctx.step(auth_bytes) + elif self._is_done: + # The final step of gssapi is send, so we do not expect any additional bytes + # however, allow an empty message to support SaslAuthenticate response + if auth_bytes != b'': + raise ValueError("Unexpected receive auth_bytes after sasl/gssapi completion") + else: + # unwraps message containing supported protection levels and msg size + msg = client_ctx.unwrap(received_token).message + # Kafka currently doesn't support integrity or confidentiality security layers, so we + # simply set QoP to 'auth' only (first octet). We reuse the max message size proposed + # by the server + message_parts = [ + Int8.encode(self.SASL_QOP_AUTH & Int8.decode(io.BytesIO(msg[0:1]))), + msg[:1], + self.auth_id.encode(), + ] + # add authorization identity to the response, and GSS-wrap + self._next_token = self._client_ctx.wrap(b''.join(message_parts), False).message + + def is_done(self): + return self._is_done + + def is_authenticated(self): + return self._is_authenticated diff --git a/kafka/sasl/oauth.py b/kafka/sasl/oauth.py new file mode 100644 index 000000000..7bbc7dd43 --- /dev/null +++ b/kafka/sasl/oauth.py @@ -0,0 +1,39 @@ +from __future__ import absolute_import + +from kafka.sasl.abc import SaslMechanism + + +class SaslMechanismOAuth(SaslMechanism): + + def __init__(self, **config): + self.token_provider = config['sasl_oauth_token_provider'] + assert self.token_provider is not None, 'sasl_oauth_token_provider required for OAUTHBEARER sasl' + assert callable(getattr(self.token_provider, 'token', None)), 'sasl_oauth_token_provider must implement method #token()' + self._is_done = False + self._is_authenticated = False + + def auth_bytes(self): + token = self.token_provider.token() + extensions = self._token_extensions() + return "n,,\x01auth=Bearer {}{}\x01\x01".format(token, extensions).encode('utf-8') + + def receive(self, auth_bytes): + self._is_done = True + self._is_authenticated = auth_bytes == b'' + + def is_done(self): + return self._is_done + + def is_authenticated(self): + return self._is_authenticated + + def _token_extensions(self): + """ + Return a string representation of the OPTIONAL key-value pairs that can be sent with an OAUTHBEARER + initial request. + """ + # Only run if the #extensions() method is implemented by the clients Token Provider class + # Builds up a string separated by \x01 via a dict of key value pairs + extensions = getattr(self.token_provider, 'extensions', lambda: [])() + msg = '\x01'.join(['{}={}'.format(k, v) for k, v in extensions.items()]) + return '\x01' + msg if msg else '' diff --git a/kafka/sasl/plain.py b/kafka/sasl/plain.py new file mode 100644 index 000000000..f2bae6751 --- /dev/null +++ b/kafka/sasl/plain.py @@ -0,0 +1,36 @@ +from __future__ import absolute_import + +import logging + +from kafka.sasl.abc import SaslMechanism + + +log = logging.getLogger(__name__) + + +class SaslMechanismPlain(SaslMechanism): + + def __init__(self, **config): + if config['security_protocol'] == 'SASL_PLAINTEXT': + log.warning('Sending username and password in the clear') + assert config['sasl_plain_username'] is not None, 'sasl_plain_username required for PLAIN sasl' + assert config['sasl_plain_password'] is not None, 'sasl_plain_password required for PLAIN sasl' + + self.username = config['sasl_plain_username'] + self.password = config['sasl_plain_password'] + self._is_done = False + self._is_authenticated = False + + def auth_bytes(self): + # Send PLAIN credentials per RFC-4616 + return bytes('\0'.join([self.username, self.username, self.password]).encode('utf-8')) + + def receive(self, auth_bytes): + self._is_done = True + self._is_authenticated = auth_bytes == b'' + + def is_done(self): + return self._is_done + + def is_authenticated(self): + return self._is_authenticated diff --git a/kafka/scram.py b/kafka/sasl/scram.py similarity index 52% rename from kafka/scram.py rename to kafka/sasl/scram.py index 7f003750c..0bae8c928 100644 --- a/kafka/scram.py +++ b/kafka/sasl/scram.py @@ -3,11 +3,17 @@ import base64 import hashlib import hmac +import logging import uuid + +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)) @@ -16,6 +22,47 @@ def xor_bytes(left, right): return bytes(lb ^ rb for lb, rb in zip(left, right)) +class SaslMechanismScram(SaslMechanism): + + def __init__(self, **config): + assert config['sasl_plain_username'] is not None, 'sasl_plain_username required for SCRAM sasl' + assert config['sasl_plain_password'] is not None, 'sasl_plain_password required for SCRAM sasl' + if config['security_protocol'] == 'SASL_PLAINTEXT': + log.warning('Exchanging credentials in the clear during Sasl Authentication') + + self._scram_client = ScramClient( + config['sasl_plain_username'], + config['sasl_plain_password'], + config['sasl_mechanism'] + ) + self._state = 0 + + def auth_bytes(self): + if self._state == 0: + return self._scram_client.first_message() + elif self._state == 1: + return self._scram_client.final_message() + else: + raise ValueError('No auth_bytes for state: %s' % self._state) + + def receive(self, auth_bytes): + if self._state == 0: + self._scram_client.process_server_first_message(auth_bytes) + elif self._state == 1: + self._scram_client.process_server_final_message(auth_bytes) + else: + raise ValueError('Cannot receive bytes in state: %s' % self._state) + self._state += 1 + return self.is_done() + + def is_done(self): + return self._state == 2 + + def is_authenticated(self): + # receive raises if authentication fails...? + return self._state == 2 + + class ScramClient: MECHANISMS = { 'SCRAM-SHA-256': hashlib.sha256, @@ -23,10 +70,10 @@ class ScramClient: } def __init__(self, user, password, mechanism): - self.nonce = str(uuid.uuid4()).replace('-', '') - self.auth_message = '' + self.nonce = str(uuid.uuid4()).replace('-', '').encode('utf-8') + self.auth_message = b'' self.salted_password = None - self.user = user + self.user = user.encode('utf-8') self.password = password.encode('utf-8') self.hashfunc = self.MECHANISMS[mechanism] self.hashname = ''.join(mechanism.lower().split('-')[1:3]) @@ -38,18 +85,18 @@ def __init__(self, user, password, mechanism): self.server_signature = None def first_message(self): - client_first_bare = 'n={},r={}'.format(self.user, self.nonce) + client_first_bare = b'n=' + self.user + b',r=' + self.nonce self.auth_message += client_first_bare - return 'n,,' + client_first_bare + return b'n,,' + client_first_bare def process_server_first_message(self, server_first_message): - self.auth_message += ',' + server_first_message - params = dict(pair.split('=', 1) for pair in server_first_message.split(',')) - server_nonce = params['r'] + self.auth_message += b',' + server_first_message + params = dict(pair.split('=', 1) for pair in server_first_message.decode('utf-8').split(',')) + server_nonce = params['r'].encode('utf-8') if not server_nonce.startswith(self.nonce): raise ValueError("Server nonce, did not start with client nonce!") self.nonce = server_nonce - self.auth_message += ',c=biws,r=' + self.nonce + self.auth_message += b',c=biws,r=' + self.nonce salt = base64.b64decode(params['s'].encode('utf-8')) iterations = int(params['i']) @@ -57,10 +104,10 @@ def process_server_first_message(self, server_first_message): self.client_key = self.hmac(self.salted_password, b'Client Key') self.stored_key = self.hashfunc(self.client_key).digest() - self.client_signature = self.hmac(self.stored_key, self.auth_message.encode('utf-8')) + self.client_signature = self.hmac(self.stored_key, self.auth_message) self.client_proof = xor_bytes(self.client_key, self.client_signature) self.server_key = self.hmac(self.salted_password, b'Server Key') - self.server_signature = self.hmac(self.server_key, self.auth_message.encode('utf-8')) + self.server_signature = self.hmac(self.server_key, self.auth_message) def hmac(self, key, msg): return hmac.new(key, msg, digestmod=self.hashfunc).digest() @@ -71,11 +118,9 @@ def create_salted_password(self, salt, iterations): ) def final_message(self): - return 'c=biws,r={},p={}'.format(self.nonce, base64.b64encode(self.client_proof).decode('utf-8')) + return b'c=biws,r=' + self.nonce + b',p=' + base64.b64encode(self.client_proof) def process_server_final_message(self, server_final_message): - params = dict(pair.split('=', 1) for pair in server_final_message.split(',')) + params = dict(pair.split('=', 1) for pair in server_final_message.decode('utf-8').split(',')) if self.server_signature != base64.b64decode(params['v'].encode('utf-8')): raise ValueError("Server sent wrong signature!") - - From 6e1b9e59ac69904d016c06b1a7bdfefb87681625 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 14:18:17 -0700 Subject: [PATCH 005/205] Cleanup sasl mechanism configuration checks; fix gssapi bugs; add sasl_kerberos_name config (#2520) --- kafka/admin/client.py | 4 ++++ kafka/client_async.py | 4 ++++ kafka/conn.py | 4 ++++ kafka/consumer/group.py | 4 ++++ kafka/producer/kafka.py | 4 ++++ kafka/sasl/gssapi.py | 17 ++++++++++++----- kafka/sasl/oauth.py | 2 +- kafka/sasl/plain.py | 6 +++--- kafka/sasl/scram.py | 8 ++++---- 9 files changed, 40 insertions(+), 13 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 29ee6cd9a..27ad69312 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -142,6 +142,9 @@ class KafkaAdminClient(object): Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. sasl_plain_password (str): password for sasl PLAIN and SCRAM authentication. Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. + sasl_kerberos_name (str or gssapi.Name): Constructed gssapi.Name for use with + sasl mechanism handshake. If provided, sasl_kerberos_service_name and + sasl_kerberos_domain name are ignored. Default: None. sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI @@ -181,6 +184,7 @@ class KafkaAdminClient(object): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, + 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, diff --git a/kafka/client_async.py b/kafka/client_async.py index 6fe47c6f7..3892c2759 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -163,6 +163,9 @@ class KafkaClient(object): Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. sasl_plain_password (str): password for sasl PLAIN and SCRAM authentication. Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. + sasl_kerberos_name (str or gssapi.Name): Constructed gssapi.Name for use with + sasl mechanism handshake. If provided, sasl_kerberos_service_name and + sasl_kerberos_domain name are ignored. Default: None. sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI @@ -206,6 +209,7 @@ class KafkaClient(object): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, + 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None diff --git a/kafka/conn.py b/kafka/conn.py index 988f4399f..857b13a57 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -178,6 +178,9 @@ class BrokerConnection(object): Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. sasl_plain_password (str): password for sasl PLAIN and SCRAM authentication. Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. + sasl_kerberos_name (str or gssapi.Name): Constructed gssapi.Name for use with + sasl mechanism handshake. If provided, sasl_kerberos_service_name and + sasl_kerberos_domain name are ignored. Default: None. sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI @@ -216,6 +219,7 @@ class BrokerConnection(object): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, + 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 6f23bec8a..16fd7c005 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -250,6 +250,9 @@ class KafkaConsumer(six.Iterator): Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. sasl_plain_password (str): password for sasl PLAIN and SCRAM authentication. Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. + sasl_kerberos_name (str or gssapi.Name): Constructed gssapi.Name for use with + sasl mechanism handshake. If provided, sasl_kerberos_service_name and + sasl_kerberos_domain name are ignored. Default: None. sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI @@ -317,6 +320,7 @@ class KafkaConsumer(six.Iterator): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, + 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 233bc3dce..1c075eba0 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -289,6 +289,9 @@ class KafkaProducer(object): Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. sasl_plain_password (str): password for sasl PLAIN and SCRAM authentication. Required if sasl_mechanism is PLAIN or one of the SCRAM mechanisms. + sasl_kerberos_name (str or gssapi.Name): Constructed gssapi.Name for use with + sasl mechanism handshake. If provided, sasl_kerberos_service_name and + sasl_kerberos_domain name are ignored. Default: None. sasl_kerberos_service_name (str): Service name to include in GSSAPI sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI @@ -347,6 +350,7 @@ class KafkaProducer(object): 'sasl_mechanism': None, 'sasl_plain_username': None, 'sasl_plain_password': None, + 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index b40c37535..1be3de4a4 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -22,12 +22,19 @@ class SaslMechanismGSSAPI(SaslMechanism): def __init__(self, **config): assert gssapi is not None, 'GSSAPI lib not available' - assert config['sasl_kerberos_service_name'] is not None, 'sasl_kerberos_service_name required for GSSAPI sasl' + if 'sasl_kerberos_name' not in config and 'sasl_kerberos_service_name' not in config: + raise ValueError('sasl_kerberos_service_name or sasl_kerberos_name required for GSSAPI sasl configuration') self._is_done = False self._is_authenticated = False - self.kerberos_damin_name = config['sasl_kerberos_domain_name'] or config['host'] - self.auth_id = config['sasl_kerberos_service_name'] + '@' + kerberos_damin_name - self.gssapi_name = gssapi.Name(auth_id, name_type=gssapi.NameType.hostbased_service).canonicalize(gssapi.MechType.kerberos) + if config.get('sasl_kerberos_name', None) is not None: + self.auth_id = str(config['sasl_kerberos_name']) + else: + kerberos_domain_name = config.get('sasl_kerberos_domain_name', '') or config.get('host', '') + self.auth_id = config['sasl_kerberos_service_name'] + '@' + kerberos_domain_name + if isinstance(config.get('sasl_kerberos_name', None), gssapi.Name): + self.gssapi_name = config['sasl_kerberos_name'] + else: + self.gssapi_name = gssapi.Name(self.auth_id, name_type=gssapi.NameType.hostbased_service).canonicalize(gssapi.MechType.kerberos) self._client_ctx = gssapi.SecurityContext(name=self.gssapi_name, usage='initiate') self._next_token = self._client_ctx.step(None) @@ -54,7 +61,7 @@ def receive(self, auth_bytes): raise ValueError("Unexpected receive auth_bytes after sasl/gssapi completion") else: # unwraps message containing supported protection levels and msg size - msg = client_ctx.unwrap(received_token).message + msg = self._client_ctx.unwrap(auth_bytes).message # Kafka currently doesn't support integrity or confidentiality security layers, so we # simply set QoP to 'auth' only (first octet). We reuse the max message size proposed # by the server diff --git a/kafka/sasl/oauth.py b/kafka/sasl/oauth.py index 7bbc7dd43..fce630a77 100644 --- a/kafka/sasl/oauth.py +++ b/kafka/sasl/oauth.py @@ -6,8 +6,8 @@ class SaslMechanismOAuth(SaslMechanism): def __init__(self, **config): + assert 'sasl_oauth_token_provider' in config, 'sasl_oauth_token_provider required for OAUTHBEARER sasl' self.token_provider = config['sasl_oauth_token_provider'] - assert self.token_provider is not None, 'sasl_oauth_token_provider required for OAUTHBEARER sasl' assert callable(getattr(self.token_provider, 'token', None)), 'sasl_oauth_token_provider must implement method #token()' self._is_done = False self._is_authenticated = False diff --git a/kafka/sasl/plain.py b/kafka/sasl/plain.py index f2bae6751..e59d23013 100644 --- a/kafka/sasl/plain.py +++ b/kafka/sasl/plain.py @@ -11,10 +11,10 @@ class SaslMechanismPlain(SaslMechanism): def __init__(self, **config): - if config['security_protocol'] == 'SASL_PLAINTEXT': + if config.get('security_protocol', '') == 'SASL_PLAINTEXT': log.warning('Sending username and password in the clear') - assert config['sasl_plain_username'] is not None, 'sasl_plain_username required for PLAIN sasl' - assert config['sasl_plain_password'] is not None, 'sasl_plain_password required for PLAIN sasl' + assert 'sasl_plain_username' in config, 'sasl_plain_username required for PLAIN sasl' + assert 'sasl_plain_password' in config, 'sasl_plain_password required for PLAIN sasl' self.username = config['sasl_plain_username'] self.password = config['sasl_plain_password'] diff --git a/kafka/sasl/scram.py b/kafka/sasl/scram.py index 0bae8c928..734885927 100644 --- a/kafka/sasl/scram.py +++ b/kafka/sasl/scram.py @@ -23,11 +23,11 @@ def xor_bytes(left, right): class SaslMechanismScram(SaslMechanism): - def __init__(self, **config): - assert config['sasl_plain_username'] is not None, 'sasl_plain_username required for SCRAM sasl' - assert config['sasl_plain_password'] is not None, 'sasl_plain_password required for SCRAM sasl' - if config['security_protocol'] == 'SASL_PLAINTEXT': + assert 'sasl_plain_username' in config, 'sasl_plain_username required for SCRAM sasl' + assert 'sasl_plain_password' in config, 'sasl_plain_password required for SCRAM sasl' + assert config.get('sasl_mechanism', '') in ScramClient.MECHANISMS, 'Unrecognized SCRAM mechanism' + if config.get('security_protocol', '') == 'SASL_PLAINTEXT': log.warning('Exchanging credentials in the clear during Sasl Authentication') self._scram_client = ScramClient( From 7e9b86b7ba350ec701ffc65fa06d730c7e5f05ae Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 14:44:13 -0700 Subject: [PATCH 006/205] Document api_version_auto_timeout_ms default; override in group tests --- kafka/client_async.py | 1 + kafka/consumer/group.py | 1 + kafka/producer/kafka.py | 1 + test/test_consumer_group.py | 2 ++ 4 files changed, 5 insertions(+) diff --git a/kafka/client_async.py b/kafka/client_async.py index 3892c2759..bd34c3b2d 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -150,6 +150,7 @@ class KafkaClient(object): api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker api version. Only applies if api_version set to None. + Default: 2000 selector (selectors.BaseSelector): Provide a specific selector implementation to use for I/O multiplexing. Default: selectors.DefaultSelector diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 16fd7c005..27be4588d 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -224,6 +224,7 @@ class KafkaConsumer(six.Iterator): api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker api version. Only applies if api_version set to None. + Default: 2000 connections_max_idle_ms: Close idle connections after the number of milliseconds specified by this config. The broker closes idle connections after connections.max.idle.ms, so this avoids hitting diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 1c075eba0..668387aac 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -272,6 +272,7 @@ class KafkaProducer(object): api_version_auto_timeout_ms (int): number of milliseconds to throw a timeout exception from the constructor when checking the broker api version. Only applies if api_version set to None. + Default: 2000 metric_reporters (list): A list of classes to use as metrics reporters. Implementing the AbstractMetricsReporter interface allows plugging in classes that will be notified of new metric creation. Default: [] diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index ec0f41832..bc04eed48 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -57,6 +57,7 @@ def consumer_thread(i): bootstrap_servers=connect_str, group_id=group_id, client_id="consumer_thread-%s" % i, + api_version_auto_timeout_ms=30000, heartbeat_interval_ms=500) while not stop[i].is_set(): for tp, records in six.itervalues(consumers[i].poll(timeout_ms=200)): @@ -156,6 +157,7 @@ def test_heartbeat_thread(kafka_broker, topic): consumer = KafkaConsumer(topic, bootstrap_servers=get_connect_str(kafka_broker), group_id=group_id, + api_version_auto_timeout_ms=30000, heartbeat_interval_ms=500) # poll until we have joined group / have assignment From ef731924c7565d4520c85d318f7ef7015303f393 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 16:04:53 -0700 Subject: [PATCH 007/205] Remove tox.ini; update testing docs --- .github/workflows/python-package.yml | 1 - docs/tests.rst | 64 +++++++--------------------- pytest.ini | 2 + requirements-dev.txt | 1 - tox.ini | 49 --------------------- 5 files changed, 17 insertions(+), 100 deletions(-) create mode 100644 pytest.ini delete mode 100644 tox.ini diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index bc0724e4a..1b0d71c89 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -65,7 +65,6 @@ jobs: sudo apt install -y libsnappy-dev libzstd-dev python -m pip install --upgrade pip pip install -r requirements-dev.txt - pip install tox-gh-actions - name: Pylint run: pylint --recursive=y --errors-only --exit-zero kafka test - name: Setup java diff --git a/docs/tests.rst b/docs/tests.rst index 561179ca5..988afca65 100644 --- a/docs/tests.rst +++ b/docs/tests.rst @@ -6,12 +6,14 @@ Tests .. image:: https://travis-ci.org/dpkp/kafka-python.svg?branch=master :target: https://travis-ci.org/dpkp/kafka-python -Test environments are managed via tox. The test suite is run via pytest. +The test suite is run via pytest. -Linting is run via pylint, but is generally skipped on pypy due to pylint -compatibility / performance issues. +Linting is run via pylint, but is currently skipped during CI/CD due to +accumulated debt. We'd like to transition to ruff! For test coverage details, see https://coveralls.io/github/dpkp/kafka-python +Coverage reporting is currently disabled as we have transitioned from travis +to GH Actions and have not yet re-enabled coveralls integration. The test suite includes unit tests that mock network interfaces, as well as integration tests that setup and teardown kafka broker (and zookeeper) @@ -21,30 +23,21 @@ fixtures for client / consumer / producer testing. Unit tests ------------------ -To run the tests locally, install tox: +To run the tests locally, install test dependencies: .. code:: bash - pip install tox + pip install -r requirements-dev.txt -For more details, see https://tox.readthedocs.io/en/latest/install.html - -Then simply run tox, optionally setting the python environment. -If unset, tox will loop through all environments. +Then simply run pytest (or make test) from your preferred python + virtualenv. .. code:: bash - tox -e py27 - tox -e py35 - - # run protocol tests only - tox -- -v test.test_protocol - - # re-run the last failing test, dropping into pdb - tox -e py27 -- --lf --pdb + # run protocol tests only (via pytest) + pytest test/test_protocol.py - # see available (pytest) options - tox -e py27 -- --help + # Run conn tests only (via make) + PYTESTS=test/test_conn.py make test Integration tests @@ -52,35 +45,8 @@ Integration tests .. code:: bash - KAFKA_VERSION=0.8.2.2 tox -e py27 - KAFKA_VERSION=1.0.1 tox -e py36 - - -Integration tests start Kafka and Zookeeper fixtures. This requires downloading -kafka server binaries: - -.. code:: bash - - ./build_integration.sh - -By default, this will install the broker versions listed in build_integration.sh's `ALL_RELEASES` -into the servers/ directory. To install a specific version, set the `KAFKA_VERSION` variable: - -.. code:: bash - - KAFKA_VERSION=1.0.1 ./build_integration.sh + KAFKA_VERSION=3.9.0 make test -Then to run the tests against a specific Kafka version, simply set the `KAFKA_VERSION` -env variable to the server build you want to use for testing: - -.. code:: bash - - KAFKA_VERSION=1.0.1 tox -e py36 - -To test against the kafka source tree, set KAFKA_VERSION=trunk -[optionally set SCALA_VERSION (defaults to the value set in `build_integration.sh`)] - -.. code:: bash - SCALA_VERSION=2.12 KAFKA_VERSION=trunk ./build_integration.sh - KAFKA_VERSION=trunk tox -e py36 +Integration tests start Kafka and Zookeeper fixtures. Make will download +kafka server binaries automatically if needed. diff --git a/pytest.ini b/pytest.ini new file mode 100644 index 000000000..f54588733 --- /dev/null +++ b/pytest.ini @@ -0,0 +1,2 @@ +[pytest] +log_format = %(created)f %(filename)-23s %(threadName)s %(message)s diff --git a/requirements-dev.txt b/requirements-dev.txt index e272d1ff7..e49608a4d 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -13,6 +13,5 @@ pytest-pylint python-snappy Sphinx sphinx-rtd-theme -tox xxhash zstandard diff --git a/tox.ini b/tox.ini deleted file mode 100644 index 71e443dec..000000000 --- a/tox.ini +++ /dev/null @@ -1,49 +0,0 @@ -[tox] -envlist = py{38,39,310,311,312,py}, docs - -[pytest] -testpaths = kafka test -addopts = --durations=10 -log_format = %(created)f %(filename)-23s %(threadName)s %(message)s - -[gh-actions] -python = - 3.8: py38 - 3.9: py39 - 3.10: py310 - 3.11: py311 - 3.12: py312 - pypy-3.9: pypy - -[testenv] -deps = - pytest - pytest-cov - pylint - pytest-pylint - pytest-mock - mock - python-snappy - zstandard - lz4 - xxhash - crc32c -commands = - pytest {posargs:--cov=kafka --cov-config=.covrc} -setenv = - CRC32C_SW_MODE = auto - PROJECT_ROOT = {toxinidir} -passenv = KAFKA_* - -[testenv:pypy] -# pylint is super slow on pypy... -commands = pytest {posargs:--cov=kafka --cov-config=.covrc} - -[testenv:docs] -deps = - sphinx_rtd_theme - sphinx - -commands = - sphinx-apidoc -o docs/apidoc/ kafka/ - sphinx-build -b html docs/ docs/_build From 0ae708a66a76bb5722f021e512d5d383393f5405 Mon Sep 17 00:00:00 2001 From: "Romain Geissler @ Amadeus" Date: Wed, 12 Mar 2025 00:56:40 +0100 Subject: [PATCH 008/205] Make the "mock" dependency optional (only used in Python < 3.3). (#2518) --- pyproject.toml | 2 +- requirements-dev.txt | 2 +- test/record/test_default_records.py | 5 ++++- test/record/test_legacy_records.py | 5 ++++- test/test_conn.py | 5 ++++- test/test_consumer_integration.py | 5 ++++- 6 files changed, 18 insertions(+), 6 deletions(-) diff --git a/pyproject.toml b/pyproject.toml index 48be87ffd..ddd40a08e 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -37,7 +37,7 @@ crc32c = ["crc32c"] lz4 = ["lz4"] snappy = ["python-snappy"] zstd = ["zstandard"] -testing = ["pytest", "mock", "pytest-mock"] +testing = ["pytest", "mock; python_version < '3.3'", "pytest-mock"] [tool.setuptools] include-package-data = false diff --git a/requirements-dev.txt b/requirements-dev.txt index e49608a4d..6cfb6d83b 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -3,7 +3,7 @@ crc32c docker-py flake8 lz4 -mock +mock; python_version < '3.3' py pylint pytest diff --git a/test/record/test_default_records.py b/test/record/test_default_records.py index c3a7b02c8..e1c840fa6 100644 --- a/test/record/test_default_records.py +++ b/test/record/test_default_records.py @@ -1,7 +1,10 @@ # -*- coding: utf-8 -*- from __future__ import unicode_literals import pytest -from mock import patch +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 43970f7c9..b15b53704 100644 --- a/test/record/test_legacy_records.py +++ b/test/record/test_legacy_records.py @@ -1,6 +1,9 @@ from __future__ import unicode_literals import pytest -from mock import patch +try: + from unittest.mock import patch +except ImportError: + from mock import patch from kafka.record.legacy_records import ( LegacyRecordBatch, LegacyRecordBatchBuilder ) diff --git a/test/test_conn.py b/test/test_conn.py index 959cbb4dc..ea88fd04c 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -4,7 +4,10 @@ from errno import EALREADY, EINPROGRESS, EISCONN, ECONNRESET import socket -import mock +try: + from unittest import mock +except ImportError: + import mock import pytest from kafka.conn import BrokerConnection, ConnectionStates, collect_hosts diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index 5aeb63d1d..af8ec6829 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -1,7 +1,10 @@ import logging import time -from 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 From 02dd98fe2d066b5b0e822b352c873ea715fec9ba Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 17:00:46 -0700 Subject: [PATCH 009/205] Support AWS_MSK_IAM authentication (#2519) Co-authored-by: Matt Oberle --- kafka/sasl/__init__.py | 2 + kafka/sasl/msk.py | 233 +++++++++++++++++++++++++++++++++++++++++ test/sasl/test_msk.py | 67 ++++++++++++ 3 files changed, 302 insertions(+) create mode 100644 kafka/sasl/msk.py create mode 100644 test/sasl/test_msk.py diff --git a/kafka/sasl/__init__.py b/kafka/sasl/__init__.py index e36d1dfbd..8677f60d2 100644 --- a/kafka/sasl/__init__.py +++ b/kafka/sasl/__init__.py @@ -1,6 +1,7 @@ from __future__ import absolute_import from kafka.sasl.gssapi import SaslMechanismGSSAPI +from kafka.sasl.msk import SaslMechanismAwsMskIam from kafka.sasl.oauth import SaslMechanismOAuth from kafka.sasl.plain import SaslMechanismPlain from kafka.sasl.scram import SaslMechanismScram @@ -24,3 +25,4 @@ def get_sasl_mechanism(name): register_sasl_mechanism('PLAIN', SaslMechanismPlain) register_sasl_mechanism('SCRAM-SHA-256', SaslMechanismScram) register_sasl_mechanism('SCRAM-SHA-512', SaslMechanismScram) +register_sasl_mechanism('AWS_MSK_IAM', SaslMechanismAwsMskIam) diff --git a/kafka/sasl/msk.py b/kafka/sasl/msk.py new file mode 100644 index 000000000..db56b4801 --- /dev/null +++ b/kafka/sasl/msk.py @@ -0,0 +1,233 @@ +from __future__ import absolute_import + +import datetime +import hashlib +import hmac +import json +import string + +# needed for AWS_MSK_IAM authentication: +try: + from botocore.session import Session as BotoSession +except ImportError: + # no botocore available, will disable AWS_MSK_IAM mechanism + BotoSession = None + +from kafka.sasl.abc import SaslMechanism +from kafka.vendor.six.moves import urllib + + +class SaslMechanismAwsMskIam(SaslMechanism): + def __init__(self, **config): + assert BotoSession is not None, 'AWS_MSK_IAM requires the "botocore" package' + assert config.get('security_protocol', '') == 'SASL_SSL', 'AWS_MSK_IAM requires SASL_SSL' + assert 'host' in config, 'AWS_MSK_IAM requires host configuration' + self.host = config['host'] + self._auth = None + self._is_done = False + self._is_authenticated = False + + def auth_bytes(self): + session = BotoSession() + credentials = session.get_credentials().get_frozen_credentials() + client = AwsMskIamClient( + host=self.host, + access_key=credentials.access_key, + secret_key=credentials.secret_key, + region=session.get_config_variable('region'), + token=credentials.token, + ) + return client.first_message() + + def receive(self, auth_bytes): + self._is_done = True + self._is_authenticated = auth_bytes != b'' + self._auth = auth_bytes.deode('utf-8') + + def is_done(self): + return self._is_done + + def is_authenticated(self): + return self._is_authenticated + + def auth_details(self): + if not self.is_authenticated: + raise RuntimeError('Not authenticated yet!') + return 'Authenticated via SASL / AWS_MSK_IAM %s' % (self._auth,) + + +class AwsMskIamClient: + UNRESERVED_CHARS = string.ascii_letters + string.digits + '-._~' + + def __init__(self, host, access_key, secret_key, region, token=None): + """ + Arguments: + host (str): The hostname of the broker. + access_key (str): An AWS_ACCESS_KEY_ID. + secret_key (str): An AWS_SECRET_ACCESS_KEY. + region (str): An AWS_REGION. + token (Optional[str]): An AWS_SESSION_TOKEN if using temporary + credentials. + """ + self.algorithm = 'AWS4-HMAC-SHA256' + self.expires = '900' + self.hashfunc = hashlib.sha256 + self.headers = [ + ('host', host) + ] + self.version = '2020_10_22' + + self.service = 'kafka-cluster' + self.action = '{}:Connect'.format(self.service) + + now = datetime.datetime.utcnow() + self.datestamp = now.strftime('%Y%m%d') + self.timestamp = now.strftime('%Y%m%dT%H%M%SZ') + + self.host = host + self.access_key = access_key + self.secret_key = secret_key + self.region = region + self.token = token + + @property + def _credential(self): + return '{0.access_key}/{0._scope}'.format(self) + + @property + def _scope(self): + return '{0.datestamp}/{0.region}/{0.service}/aws4_request'.format(self) + + @property + def _signed_headers(self): + """ + Returns (str): + An alphabetically sorted, semicolon-delimited list of lowercase + request header names. + """ + return ';'.join(sorted(k.lower() for k, _ in self.headers)) + + @property + def _canonical_headers(self): + """ + Returns (str): + A newline-delited list of header names and values. + Header names are lowercased. + """ + return '\n'.join(map(':'.join, self.headers)) + '\n' + + @property + def _canonical_request(self): + """ + Returns (str): + An AWS Signature Version 4 canonical request in the format: + \n + \n + \n + \n + \n + + """ + # The hashed_payload is always an empty string for MSK. + hashed_payload = self.hashfunc(b'').hexdigest() + return '\n'.join(( + 'GET', + '/', + self._canonical_querystring, + self._canonical_headers, + self._signed_headers, + hashed_payload, + )) + + @property + def _canonical_querystring(self): + """ + Returns (str): + A '&'-separated list of URI-encoded key/value pairs. + """ + params = [] + params.append(('Action', self.action)) + params.append(('X-Amz-Algorithm', self.algorithm)) + params.append(('X-Amz-Credential', self._credential)) + params.append(('X-Amz-Date', self.timestamp)) + params.append(('X-Amz-Expires', self.expires)) + if self.token: + params.append(('X-Amz-Security-Token', self.token)) + params.append(('X-Amz-SignedHeaders', self._signed_headers)) + + return '&'.join(self._uriencode(k) + '=' + self._uriencode(v) for k, v in params) + + @property + def _signing_key(self): + """ + Returns (bytes): + An AWS Signature V4 signing key generated from the secret_key, date, + region, service, and request type. + """ + key = self._hmac(('AWS4' + self.secret_key).encode('utf-8'), self.datestamp) + key = self._hmac(key, self.region) + key = self._hmac(key, self.service) + key = self._hmac(key, 'aws4_request') + return key + + @property + def _signing_str(self): + """ + Returns (str): + A string used to sign the AWS Signature V4 payload in the format: + \n + \n + \n + + """ + canonical_request_hash = self.hashfunc(self._canonical_request.encode('utf-8')).hexdigest() + return '\n'.join((self.algorithm, self.timestamp, self._scope, canonical_request_hash)) + + def _uriencode(self, msg): + """ + Arguments: + msg (str): A string to URI-encode. + + Returns (str): + The URI-encoded version of the provided msg, following the encoding + rules specified: https://github.com/aws/aws-msk-iam-auth#uriencode + """ + return urllib.parse.quote(msg, safe=self.UNRESERVED_CHARS) + + def _hmac(self, key, msg): + """ + Arguments: + key (bytes): A key to use for the HMAC digest. + msg (str): A value to include in the HMAC digest. + Returns (bytes): + An HMAC digest of the given key and msg. + """ + return hmac.new(key, msg.encode('utf-8'), digestmod=self.hashfunc).digest() + + def first_message(self): + """ + Returns (bytes): + An encoded JSON authentication payload that can be sent to the + broker. + """ + signature = hmac.new( + self._signing_key, + self._signing_str.encode('utf-8'), + digestmod=self.hashfunc, + ).hexdigest() + msg = { + 'version': self.version, + 'host': self.host, + 'user-agent': 'kafka-python', + 'action': self.action, + 'x-amz-algorithm': self.algorithm, + 'x-amz-credential': self._credential, + 'x-amz-date': self.timestamp, + 'x-amz-signedheaders': self._signed_headers, + 'x-amz-expires': self.expires, + 'x-amz-signature': signature, + } + if self.token: + msg['x-amz-security-token'] = self.token + + return json.dumps(msg, separators=(',', ':')).encode('utf-8') diff --git a/test/sasl/test_msk.py b/test/sasl/test_msk.py new file mode 100644 index 000000000..297ca84ce --- /dev/null +++ b/test/sasl/test_msk.py @@ -0,0 +1,67 @@ +import datetime +import json + +from kafka.sasl.msk import AwsMskIamClient + +try: + from unittest import mock +except ImportError: + import mock + + +def client_factory(token=None): + now = datetime.datetime.utcfromtimestamp(1629321911) + with mock.patch('kafka.sasl.msk.datetime') as mock_dt: + mock_dt.datetime.utcnow = mock.Mock(return_value=now) + return AwsMskIamClient( + host='localhost', + access_key='XXXXXXXXXXXXXXXXXXXX', + secret_key='XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX', + region='us-east-1', + token=token, + ) + + +def test_aws_msk_iam_client_permanent_credentials(): + client = client_factory(token=None) + msg = client.first_message() + assert msg + assert isinstance(msg, bytes) + actual = json.loads(msg) + + expected = { + 'version': '2020_10_22', + 'host': 'localhost', + 'user-agent': 'kafka-python', + 'action': 'kafka-cluster:Connect', + 'x-amz-algorithm': 'AWS4-HMAC-SHA256', + 'x-amz-credential': 'XXXXXXXXXXXXXXXXXXXX/20210818/us-east-1/kafka-cluster/aws4_request', + 'x-amz-date': '20210818T212511Z', + 'x-amz-signedheaders': 'host', + 'x-amz-expires': '900', + 'x-amz-signature': '0fa42ae3d5693777942a7a4028b564f0b372bafa2f71c1a19ad60680e6cb994b', + } + assert actual == expected + + +def test_aws_msk_iam_client_temporary_credentials(): + client = client_factory(token='XXXXX') + msg = client.first_message() + assert msg + assert isinstance(msg, bytes) + actual = json.loads(msg) + + expected = { + 'version': '2020_10_22', + 'host': 'localhost', + 'user-agent': 'kafka-python', + 'action': 'kafka-cluster:Connect', + 'x-amz-algorithm': 'AWS4-HMAC-SHA256', + 'x-amz-credential': 'XXXXXXXXXXXXXXXXXXXX/20210818/us-east-1/kafka-cluster/aws4_request', + 'x-amz-date': '20210818T212511Z', + 'x-amz-signedheaders': 'host', + 'x-amz-expires': '900', + 'x-amz-signature': 'b0619c50b7ecb4a7f6f92bd5f733770df5710e97b25146f97015c0b1db783b05', + 'x-amz-security-token': 'XXXXX', + } + assert actual == expected From 57f17824c31244ada3bcb9ceea751c5ffa1b4b0a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 17:01:34 -0700 Subject: [PATCH 010/205] Fix lint issues via ruff check (#2522) --- kafka/client_async.py | 2 +- kafka/conn.py | 25 +++++++++---------- kafka/consumer/fetcher.py | 4 +-- .../assignors/sticky/sticky_assignor.py | 1 - kafka/coordinator/consumer.py | 2 +- kafka/metrics/metric_name.py | 2 +- kafka/metrics/quota.py | 2 +- kafka/producer/kafka.py | 4 +-- kafka/protocol/api_versions.py | 2 ++ kafka/protocol/commit.py | 2 +- kafka/protocol/find_coordinator.py | 2 +- kafka/protocol/offset_for_leader_epoch.py | 10 ++++---- kafka/protocol/sasl_authenticate.py | 2 +- kafka/sasl/gssapi.py | 4 ++- test/test_admin.py | 8 +++--- test/test_admin_integration.py | 4 +-- test/test_cluster.py | 2 -- test/test_fetcher.py | 2 +- test/test_object_conversion.py | 4 +-- test/test_protocol.py | 6 ++--- test/test_sender.py | 1 - 21 files changed, 44 insertions(+), 47 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index bd34c3b2d..b72c05dac 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -19,7 +19,7 @@ from kafka.vendor import six from kafka.cluster import ClusterMetadata -from kafka.conn import BrokerConnection, ConnectionStates, collect_hosts, get_ip_port_afi +from kafka.conn import BrokerConnection, ConnectionStates, get_ip_port_afi from kafka import errors as Errors from kafka.future import Future from kafka.metrics import AnonMeasurable diff --git a/kafka/conn.py b/kafka/conn.py index 857b13a57..7287a4840 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -14,7 +14,6 @@ from kafka.vendor import selectors34 as selectors import socket -import struct import threading import time @@ -23,7 +22,6 @@ import kafka.errors as Errors from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate -from kafka.oauth.abstract import AbstractTokenProvider from kafka.protocol.admin import DescribeAclsRequest, DescribeClientQuotasRequest, ListGroupsRequest from kafka.protocol.api_versions import ApiVersionsRequest from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS @@ -36,7 +34,7 @@ from kafka.protocol.produce import ProduceRequest from kafka.protocol.sasl_authenticate import SaslAuthenticateRequest from kafka.protocol.sasl_handshake import SaslHandshakeRequest -from kafka.protocol.types import Int32, Int8 +from kafka.protocol.types import Int32 from kafka.sasl import get_sasl_mechanism from kafka.version import __version__ @@ -1151,7 +1149,8 @@ def timed_out_ifrs(self): def next_ifr_request_timeout_ms(self): with self._lock: if self.in_flight_requests: - get_timeout = lambda v: v[2] + def get_timeout(v): + return v[2] next_timeout = min(map(get_timeout, self.in_flight_requests.values())) return max(0, (next_timeout - time.time()) * 1000) @@ -1159,11 +1158,11 @@ def next_ifr_request_timeout_ms(self): return float('inf') def get_api_versions(self): - if self._api_versions is not None: - return self._api_versions - - version = self.check_version() - # _api_versions is set as a side effect of check_versions() + # _api_versions is set as a side effect of first connection + # which should typically be bootstrap, but call check_version + # if that hasn't happened yet + if self._api_versions is None: + self.check_version() return self._api_versions def _infer_broker_version_from_api_versions(self, api_versions): @@ -1201,11 +1200,11 @@ def _infer_broker_version_from_api_versions(self, api_versions): ] # Get the best match of test cases - for broker_version, struct in sorted(test_cases, reverse=True): - if struct.API_KEY not in api_versions: + for broker_version, proto_struct in sorted(test_cases, reverse=True): + if proto_struct.API_KEY not in api_versions: continue - min_version, max_version = api_versions[struct.API_KEY] - if min_version <= struct.API_VERSION <= max_version: + min_version, max_version = api_versions[proto_struct.API_KEY] + if min_version <= proto_struct.API_VERSION <= max_version: return broker_version # We know that ApiVersionsResponse is only supported in 0.10+ diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index eefac5ba7..a833a5b79 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -501,7 +501,7 @@ def _unpack_records(self, tp, records): # If unpacking raises StopIteration, it is erroneously # caught by the generator. We want all exceptions to be raised # back to the user. See Issue 545 - except StopIteration as e: + except StopIteration: log.exception('StopIteration raised unpacking messageset') raise RuntimeError('StopIteration raised unpacking messageset') @@ -1001,7 +1001,7 @@ def build_next(self, next_partitions): log.debug("Built full fetch %s for node %s with %s partition(s).", self.next_metadata, self.node_id, len(next_partitions)) self.session_partitions = next_partitions - return FetchRequestData(next_partitions, None, self.next_metadata); + return FetchRequestData(next_partitions, None, self.next_metadata) prev_tps = set(self.session_partitions.keys()) next_tps = set(next_partitions.keys()) diff --git a/kafka/coordinator/assignors/sticky/sticky_assignor.py b/kafka/coordinator/assignors/sticky/sticky_assignor.py index dce714f1a..6e79c597e 100644 --- a/kafka/coordinator/assignors/sticky/sticky_assignor.py +++ b/kafka/coordinator/assignors/sticky/sticky_assignor.py @@ -2,7 +2,6 @@ from collections import defaultdict, namedtuple from copy import deepcopy -from kafka.cluster import ClusterMetadata 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 diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 36c91ee42..9c662ce7f 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -796,7 +796,7 @@ def _handle_offset_fetch_response(self, future, response): elif error_type is Errors.GroupAuthorizationFailedError: future.failure(error) else: - log.error("Unknown error fetching offsets for %s: %s", tp, error) + log.error("Unknown error fetching offsets: %s", error) future.failure(error) return diff --git a/kafka/metrics/metric_name.py b/kafka/metrics/metric_name.py index b5acd1662..32a7e3a4b 100644 --- a/kafka/metrics/metric_name.py +++ b/kafka/metrics/metric_name.py @@ -93,7 +93,7 @@ def __eq__(self, other): return True if other is None: return False - return (type(self) == type(other) and + return (isinstance(self, type(other)) and self.group == other.group and self.name == other.name and self.tags == other.tags) diff --git a/kafka/metrics/quota.py b/kafka/metrics/quota.py index 4d1b0d6cb..237edf841 100644 --- a/kafka/metrics/quota.py +++ b/kafka/metrics/quota.py @@ -34,7 +34,7 @@ def __hash__(self): def __eq__(self, other): if self is other: return True - return (type(self) == type(other) and + return (isinstance(self, type(other)) and self.bound == other.bound and self.is_upper_bound() == other.is_upper_bound()) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 668387aac..2a70700c4 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -612,8 +612,8 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest if headers is None: headers = [] - assert type(headers) == list - assert all(type(item) == tuple and len(item) == 2 and type(item[0]) == str and type(item[1]) == bytes for item in headers) + assert isinstance(headers, list) + assert all(isinstance(item, tuple) and len(item) == 2 and isinstance(item[0], str) and isinstance(item[1], bytes) for item in headers) message_size = self._estimate_size_in_bytes(key_bytes, value_bytes, headers) self._ensure_valid_record_size(message_size) diff --git a/kafka/protocol/api_versions.py b/kafka/protocol/api_versions.py index dc0aa588e..7e2e61251 100644 --- a/kafka/protocol/api_versions.py +++ b/kafka/protocol/api_versions.py @@ -1,5 +1,7 @@ from __future__ import absolute_import +from io import BytesIO + from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int16, Int32, Schema diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py index 53c2466fe..a0439e7ef 100644 --- a/kafka/protocol/commit.py +++ b/kafka/protocol/commit.py @@ -1,7 +1,7 @@ from __future__ import absolute_import from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String +from kafka.protocol.types import Array, Int16, Int32, Int64, Schema, String class OffsetCommitResponse_v0(Response): diff --git a/kafka/protocol/find_coordinator.py b/kafka/protocol/find_coordinator.py index a68a23902..be5b45ded 100644 --- a/kafka/protocol/find_coordinator.py +++ b/kafka/protocol/find_coordinator.py @@ -1,7 +1,7 @@ from __future__ import absolute_import from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String +from kafka.protocol.types import Int8, Int16, Int32, Schema, String class FindCoordinatorResponse_v0(Response): diff --git a/kafka/protocol/offset_for_leader_epoch.py b/kafka/protocol/offset_for_leader_epoch.py index afe8284eb..8465588a3 100644 --- a/kafka/protocol/offset_for_leader_epoch.py +++ b/kafka/protocol/offset_for_leader_epoch.py @@ -4,7 +4,7 @@ from kafka.protocol.types import Array, CompactArray, CompactString, Int16, Int32, Int64, Schema, String, TaggedFields -class OffsetForLeaderEpochResponse_v0(Request): +class OffsetForLeaderEpochResponse_v0(Response): API_KEY = 23 API_VERSION = 0 SCHEMA = Schema( @@ -16,7 +16,7 @@ class OffsetForLeaderEpochResponse_v0(Request): ('end_offset', Int64)))))) -class OffsetForLeaderEpochResponse_v1(Request): +class OffsetForLeaderEpochResponse_v1(Response): API_KEY = 23 API_VERSION = 1 SCHEMA = Schema( @@ -29,7 +29,7 @@ class OffsetForLeaderEpochResponse_v1(Request): ('end_offset', Int64)))))) -class OffsetForLeaderEpochResponse_v2(Request): +class OffsetForLeaderEpochResponse_v2(Response): API_KEY = 23 API_VERSION = 2 SCHEMA = Schema( @@ -43,13 +43,13 @@ class OffsetForLeaderEpochResponse_v2(Request): ('end_offset', Int64)))))) -class OffsetForLeaderEpochResponse_v3(Request): +class OffsetForLeaderEpochResponse_v3(Response): API_KEY = 23 API_VERSION = 3 SCHEMA = OffsetForLeaderEpochResponse_v2.SCHEMA -class OffsetForLeaderEpochResponse_v4(Request): +class OffsetForLeaderEpochResponse_v4(Response): API_KEY = 23 API_VERSION = 4 SCHEMA = Schema( diff --git a/kafka/protocol/sasl_authenticate.py b/kafka/protocol/sasl_authenticate.py index 528bb3cc6..a2b9b1988 100644 --- a/kafka/protocol/sasl_authenticate.py +++ b/kafka/protocol/sasl_authenticate.py @@ -1,7 +1,7 @@ from __future__ import absolute_import from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Bytes, Int16, Int64, Schema, String +from kafka.protocol.types import Bytes, Int16, Int64, Schema, String class SaslAuthenticateResponse_v0(Response): diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index 1be3de4a4..60b658c77 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -1,5 +1,7 @@ from __future__ import absolute_import +import struct + # needed for SASL_GSSAPI authentication: try: import gssapi @@ -66,7 +68,7 @@ def receive(self, auth_bytes): # simply set QoP to 'auth' only (first octet). We reuse the max message size proposed # by the server message_parts = [ - Int8.encode(self.SASL_QOP_AUTH & Int8.decode(io.BytesIO(msg[0:1]))), + struct.pack('>b', self.SASL_QOP_AUTH & struct.unpack('>b', msg[0:1])), msg[:1], self.auth_id.encode(), ] diff --git a/test/test_admin.py b/test/test_admin.py index 279f85abf..cdb74242e 100644 --- a/test/test_admin.py +++ b/test/test_admin.py @@ -6,7 +6,7 @@ def test_config_resource(): with pytest.raises(KeyError): - bad_resource = kafka.admin.ConfigResource('something', 'foo') + _bad_resource = kafka.admin.ConfigResource('something', 'foo') good_resource = kafka.admin.ConfigResource('broker', 'bar') assert good_resource.resource_type == kafka.admin.ConfigResourceType.BROKER assert good_resource.name == 'bar' @@ -59,11 +59,11 @@ def test_acl_resource(): def test_new_topic(): with pytest.raises(IllegalArgumentError): - bad_topic = kafka.admin.NewTopic('foo', -1, -1) + _bad_topic = kafka.admin.NewTopic('foo', -1, -1) with pytest.raises(IllegalArgumentError): - bad_topic = kafka.admin.NewTopic('foo', 1, -1) + _bad_topic = kafka.admin.NewTopic('foo', 1, -1) with pytest.raises(IllegalArgumentError): - bad_topic = kafka.admin.NewTopic('foo', 1, 1, {1: [1, 1, 1]}) + _bad_topic = kafka.admin.NewTopic('foo', 1, 1, {1: [1, 1, 1]}) good_topic = kafka.admin.NewTopic('foo', 1, 2) assert good_topic.name == 'foo' assert good_topic.num_partitions == 1 diff --git a/test/test_admin_integration.py b/test/test_admin_integration.py index bd2fd216e..2f6b76598 100644 --- a/test/test_admin_integration.py +++ b/test/test_admin_integration.py @@ -140,7 +140,7 @@ def test_describe_configs_invalid_broker_id_raises(kafka_admin_client): broker_id = "str" with pytest.raises(ValueError): - configs = kafka_admin_client.describe_configs([ConfigResource(ConfigResourceType.BROKER, broker_id)]) + kafka_admin_client.describe_configs([ConfigResource(ConfigResourceType.BROKER, broker_id)]) @pytest.mark.skipif(env_kafka_version() < (0, 11), reason='Describe consumer group requires broker >=0.11') @@ -148,7 +148,7 @@ def test_describe_consumer_group_does_not_exist(kafka_admin_client): """Tests that the describe consumer group call fails if the group coordinator is not available """ with pytest.raises(GroupCoordinatorNotAvailableError): - group_description = kafka_admin_client.describe_consumer_groups(['test']) + kafka_admin_client.describe_consumer_groups(['test']) @pytest.mark.skipif(env_kafka_version() < (0, 11), reason='Describe consumer group requires broker >=0.11') diff --git a/test/test_cluster.py b/test/test_cluster.py index b55bdc5ad..f0a2f83d6 100644 --- a/test/test_cluster.py +++ b/test/test_cluster.py @@ -1,8 +1,6 @@ # pylint: skip-file from __future__ import absolute_import -import pytest - from kafka.cluster import ClusterMetadata from kafka.protocol.metadata import MetadataResponse diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 7e948e3cb..479f6e22b 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -10,7 +10,7 @@ from kafka.client_async import KafkaClient from kafka.consumer.fetcher import ( - CompletedFetch, ConsumerRecord, Fetcher, NoOffsetForPartitionError + CompletedFetch, ConsumerRecord, Fetcher ) from kafka.consumer.subscription_state import SubscriptionState import kafka.errors as Errors diff --git a/test/test_object_conversion.py b/test/test_object_conversion.py index 9b1ff2131..a48eb0601 100644 --- a/test/test_object_conversion.py +++ b/test/test_object_conversion.py @@ -207,7 +207,7 @@ def test_with_metadata_response(): assert len(obj['topics']) == 2 assert obj['topics'][0]['error_code'] == 0 assert obj['topics'][0]['topic'] == 'testtopic1' - assert obj['topics'][0]['is_internal'] == False + assert obj['topics'][0]['is_internal'] is False assert len(obj['topics'][0]['partitions']) == 2 assert obj['topics'][0]['partitions'][0]['error_code'] == 0 assert obj['topics'][0]['partitions'][0]['partition'] == 0 @@ -224,7 +224,7 @@ def test_with_metadata_response(): assert obj['topics'][1]['error_code'] == 0 assert obj['topics'][1]['topic'] == 'other-test-topic' - assert obj['topics'][1]['is_internal'] == True + assert obj['topics'][1]['is_internal'] is True assert len(obj['topics'][1]['partitions']) == 1 assert obj['topics'][1]['partitions'][0]['error_code'] == 0 assert obj['topics'][1]['partitions'][0]['partition'] == 0 diff --git a/test/test_protocol.py b/test/test_protocol.py index 6f94c74e1..d0cc7ed0a 100644 --- a/test/test_protocol.py +++ b/test/test_protocol.py @@ -2,8 +2,6 @@ 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 @@ -273,7 +271,7 @@ def test_decode_fetch_response_partial(): def test_struct_unrecognized_kwargs(): try: - mr = MetadataRequest[0](topicz='foo') + _mr = MetadataRequest[0](topicz='foo') assert False, 'Structs should not allow unrecognized kwargs' except ValueError: pass @@ -331,6 +329,6 @@ def test_compact_data_structs(): assert CompactBytes.decode(io.BytesIO(b'\x00')) is None enc = CompactBytes.encode(b'') assert enc == b'\x01' - assert CompactBytes.decode(io.BytesIO(b'\x01')) is b'' + 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/test_sender.py b/test/test_sender.py index 83a26cd39..3da1a9f42 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -5,7 +5,6 @@ import io from kafka.client_async import KafkaClient -from kafka.cluster import ClusterMetadata from kafka.metrics import Metrics from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.produce import ProduceRequest From 7f1bbb98a839ddecdd5db3e5283b9801309bafc0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 11 Mar 2025 17:15:05 -0700 Subject: [PATCH 011/205] feat: Add SSPI (Kerberos for Windows) authentication mechanism (#2521) Co-authored-by: Emmanuel --- kafka/conn.py | 2 +- kafka/sasl/__init__.py | 10 +++- kafka/sasl/abc.py | 5 ++ kafka/sasl/gssapi.py | 13 +++-- kafka/sasl/oauth.py | 5 ++ kafka/sasl/plain.py | 5 ++ kafka/sasl/scram.py | 7 +++ kafka/sasl/sspi.py | 111 +++++++++++++++++++++++++++++++++++++++++ 8 files changed, 151 insertions(+), 7 deletions(-) create mode 100644 kafka/sasl/sspi.py diff --git a/kafka/conn.py b/kafka/conn.py index 7287a4840..6992bb5c2 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -766,7 +766,7 @@ def _sasl_authenticate(self, future): self._sasl_mechanism.receive(recv_token) if self._sasl_mechanism.is_authenticated(): - log.info('%s: Authenticated via %s', self, self.config['sasl_mechanism']) + log.info('%s: %s', self, self._sasl_mechanism.auth_details()) return future.success(True) else: return future.failure(Errors.AuthenticationFailedError('Failed to authenticate via SASL %s' % self.config['sasl_mechanism'])) diff --git a/kafka/sasl/__init__.py b/kafka/sasl/__init__.py index 8677f60d2..90f05e733 100644 --- a/kafka/sasl/__init__.py +++ b/kafka/sasl/__init__.py @@ -1,10 +1,13 @@ from __future__ import absolute_import +import platform + from kafka.sasl.gssapi import SaslMechanismGSSAPI from kafka.sasl.msk import SaslMechanismAwsMskIam from kafka.sasl.oauth import SaslMechanismOAuth from kafka.sasl.plain import SaslMechanismPlain from kafka.sasl.scram import SaslMechanismScram +from kafka.sasl.sspi import SaslMechanismSSPI SASL_MECHANISMS = {} @@ -20,9 +23,12 @@ def get_sasl_mechanism(name): return SASL_MECHANISMS[name] -register_sasl_mechanism('GSSAPI', SaslMechanismGSSAPI) +register_sasl_mechanism('AWS_MSK_IAM', SaslMechanismAwsMskIam) +if platform.system() == 'Windows': + register_sasl_mechanism('GSSAPI', SaslMechanismSSPI) +else: + register_sasl_mechanism('GSSAPI', SaslMechanismGSSAPI) register_sasl_mechanism('OAUTHBEARER', SaslMechanismOAuth) register_sasl_mechanism('PLAIN', SaslMechanismPlain) register_sasl_mechanism('SCRAM-SHA-256', SaslMechanismScram) register_sasl_mechanism('SCRAM-SHA-512', SaslMechanismScram) -register_sasl_mechanism('AWS_MSK_IAM', SaslMechanismAwsMskIam) diff --git a/kafka/sasl/abc.py b/kafka/sasl/abc.py index 7baef3b78..8977c7c23 100644 --- a/kafka/sasl/abc.py +++ b/kafka/sasl/abc.py @@ -25,3 +25,8 @@ def is_done(self): @abc.abstractmethod def is_authenticated(self): pass + + def auth_details(self): + if not self.is_authenticated: + raise RuntimeError('Not authenticated yet!') + return 'Authenticated via SASL' diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index 60b658c77..be84269da 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -1,7 +1,5 @@ from __future__ import absolute_import -import struct - # needed for SASL_GSSAPI authentication: try: import gssapi @@ -67,10 +65,12 @@ def receive(self, auth_bytes): # Kafka currently doesn't support integrity or confidentiality security layers, so we # 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] message_parts = [ - struct.pack('>b', self.SASL_QOP_AUTH & struct.unpack('>b', msg[0:1])), + bytes(client_flags & server_flags), msg[:1], - self.auth_id.encode(), + self.auth_id.encode('utf-8'), ] # add authorization identity to the response, and GSS-wrap self._next_token = self._client_ctx.wrap(b''.join(message_parts), False).message @@ -80,3 +80,8 @@ def is_done(self): def is_authenticated(self): return self._is_authenticated + + def auth_details(self): + if not self.is_authenticated: + raise RuntimeError('Not authenticated yet!') + return 'Authenticated as %s to %s via SASL / GSSAPI' % (self._client_ctx.initiator_name, self._client_ctx.target_name) diff --git a/kafka/sasl/oauth.py b/kafka/sasl/oauth.py index fce630a77..d4f643d84 100644 --- a/kafka/sasl/oauth.py +++ b/kafka/sasl/oauth.py @@ -37,3 +37,8 @@ def _token_extensions(self): extensions = getattr(self.token_provider, 'extensions', lambda: [])() msg = '\x01'.join(['{}={}'.format(k, v) for k, v in extensions.items()]) return '\x01' + msg if msg else '' + + def auth_details(self): + if not self.is_authenticated: + raise RuntimeError('Not authenticated yet!') + return 'Authenticated via SASL / OAuth' diff --git a/kafka/sasl/plain.py b/kafka/sasl/plain.py index e59d23013..81443f5fe 100644 --- a/kafka/sasl/plain.py +++ b/kafka/sasl/plain.py @@ -34,3 +34,8 @@ def is_done(self): def is_authenticated(self): return self._is_authenticated + + def auth_details(self): + if not self.is_authenticated: + raise RuntimeError('Not authenticated yet!') + return 'Authenticated as %s via SASL / Plain' % self.username diff --git a/kafka/sasl/scram.py b/kafka/sasl/scram.py index 734885927..d8cd071a7 100644 --- a/kafka/sasl/scram.py +++ b/kafka/sasl/scram.py @@ -30,6 +30,8 @@ def __init__(self, **config): if config.get('security_protocol', '') == 'SASL_PLAINTEXT': log.warning('Exchanging credentials in the clear during Sasl Authentication') + self.username = config['sasl_plain_username'] + self.mechanism = config['sasl_mechanism'] self._scram_client = ScramClient( config['sasl_plain_username'], config['sasl_plain_password'], @@ -62,6 +64,11 @@ def is_authenticated(self): # receive raises if authentication fails...? return self._state == 2 + def auth_details(self): + if not self.is_authenticated: + raise RuntimeError('Not authenticated yet!') + return 'Authenticated as %s via SASL / %s' % (self.username, self.mechanism) + class ScramClient: MECHANISMS = { diff --git a/kafka/sasl/sspi.py b/kafka/sasl/sspi.py new file mode 100644 index 000000000..f4c95d037 --- /dev/null +++ b/kafka/sasl/sspi.py @@ -0,0 +1,111 @@ +from __future__ import absolute_import + +import logging + +# Windows-only +try: + import sspi + import pywintypes + import sspicon + import win32security +except ImportError: + sspi = None + +from kafka.sasl.abc import SaslMechanism + + +log = logging.getLogger(__name__) + + +class SaslMechanismSSPI(SaslMechanism): + # Establish security context and negotiate protection level + # For reference see RFC 4752, section 3 + + SASL_QOP_AUTH = 1 + SASL_QOP_AUTH_INT = 2 + SASL_QOP_AUTH_CONF = 4 + + def __init__(self, **config): + assert sspi is not None, 'No GSSAPI lib available (gssapi or sspi)' + if 'sasl_kerberos_name' not in config and 'sasl_kerberos_service_name' not in config: + raise ValueError('sasl_kerberos_service_name or sasl_kerberos_name required for GSSAPI sasl configuration') + self._is_done = False + self._is_authenticated = False + if config.get('sasl_kerberos_name', None) is not None: + self.auth_id = str(config['sasl_kerberos_name']) + else: + kerberos_domain_name = config.get('sasl_kerberos_domain_name', '') or config.get('host', '') + self.auth_id = config['sasl_kerberos_service_name'] + '/' + kerberos_domain_name + scheme = "Kerberos" # Do not try with Negotiate for SASL authentication. Tokens are different. + # https://docs.microsoft.com/en-us/windows/win32/secauthn/context-requirements + flags = ( + sspicon.ISC_REQ_MUTUAL_AUTH | # mutual authentication + sspicon.ISC_REQ_INTEGRITY | # check for integrity + sspicon.ISC_REQ_SEQUENCE_DETECT | # enable out-of-order messages + sspicon.ISC_REQ_CONFIDENTIALITY # request confidentiality + ) + self._client_ctx = sspi.ClientAuth(scheme, targetspn=self.auth_id, scflags=flags) + self._next_token = self._client_ctx.step(None) + + def auth_bytes(self): + # GSSAPI Auth does not have a final broker->client message + # so mark is_done after the final auth_bytes are provided + # in practice we'll still receive a response when using SaslAuthenticate + # but not when using the prior unframed approach. + if self._client_ctx.authenticated: + self._is_done = True + self._is_authenticated = True + return self._next_token or b'' + + def receive(self, auth_bytes): + log.debug("Received token from server (size %s)", len(auth_bytes)) + if not self._client_ctx.authenticated: + # calculate an output token from kafka token (or None on first iteration) + # https://docs.microsoft.com/en-us/windows/win32/api/sspi/nf-sspi-initializesecuritycontexta + # https://docs.microsoft.com/en-us/windows/win32/secauthn/initializesecuritycontext--kerberos + # authorize method will wrap for us our token in sspi structures + error, auth = self._client_ctx.authorize(auth_bytes) + if len(auth) > 0 and len(auth[0].Buffer): + log.debug("Got token from context") + # this buffer must be sent to the server whatever the result is + self._next_token = auth[0].Buffer + else: + log.debug("Got no token, exchange finished") + # seems to be the end of the loop + self._next_token = b'' + elif self._is_done: + # The final step of gssapi is send, so we do not expect any additional bytes + # however, allow an empty message to support SaslAuthenticate response + if auth_bytes != b'': + raise ValueError("Unexpected receive auth_bytes after sasl/gssapi completion") + else: + # Process the security layer negotiation token, sent by the server + # once the security context is established. + + # The following part is required by SASL, but not by classic Kerberos. + # See RFC 4752 + + # unwraps message containing supported protection levels and msg size + msg, _was_encrypted = self._client_ctx.unwrap(auth_bytes) + + # Kafka currently doesn't support integrity or confidentiality security layers, so we + # 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] + message_parts = [ + bytes(client_flags & server_flags), + msg[:1], + self.auth_id.encode('utf-8'), + ] + # add authorization identity to the response, and GSS-wrap + self._next_token = self._client_ctx.wrap(b''.join(message_parts), False) + + def is_done(self): + return self._is_done + + def is_authenticated(self): + return self._is_authenticated + + def auth_details(self): + return 'Authenticated as %s to %s via SASL / SSPI/GSSAPI \\o/' % (self._client_ctx.initiator_name, self._client_ctx.service_name) From 87700497a286425a544fe6c44c1e26a38d2c29f4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 09:30:13 -0700 Subject: [PATCH 012/205] socket.setblocking for eventlet/gevent compatibility --- kafka/conn.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/kafka/conn.py b/kafka/conn.py index 6992bb5c2..588b5fd86 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -679,6 +679,7 @@ def _send_bytes(self, data): return total_sent def _send_bytes_blocking(self, data): + self._sock.setblocking(True) self._sock.settimeout(self.config['request_timeout_ms'] / 1000) total_sent = 0 try: @@ -690,8 +691,10 @@ def _send_bytes_blocking(self, data): return total_sent finally: self._sock.settimeout(0.0) + self._sock.setblocking(False) def _recv_bytes_blocking(self, n): + self._sock.setblocking(True) self._sock.settimeout(self.config['request_timeout_ms'] / 1000) try: data = b'' @@ -703,6 +706,7 @@ def _recv_bytes_blocking(self, n): return data finally: self._sock.settimeout(0.0) + self._sock.setblocking(False) def _send_sasl_authenticate(self, sasl_auth_bytes): version = self._sasl_handshake_version() From e6b7d3144bcc3ff0ea201b1033ce2deadc09fe10 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 13:01:09 -0700 Subject: [PATCH 013/205] Attempt to fix metadata race condition when partitioning in producer.send (#2523) --- kafka/producer/kafka.py | 58 ++++++++++++++++++++++++++--------------- 1 file changed, 37 insertions(+), 21 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 2a70700c4..f7e2b5fa4 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -1,4 +1,4 @@ -from __future__ import absolute_import +from __future__ import absolute_import, division import atexit import copy @@ -538,7 +538,7 @@ def close(self, timeout=None): def partitions_for(self, topic): """Returns set of all known partitions for the topic.""" - max_wait = self.config['max_block_ms'] / 1000.0 + max_wait = self.config['max_block_ms'] / 1000 return self._wait_on_metadata(topic, max_wait) def _max_usable_produce_magic(self): @@ -596,19 +596,29 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest assert not (value is None and key is None), 'Need at least one: key or value' key_bytes = value_bytes = None try: - self._wait_on_metadata(topic, self.config['max_block_ms'] / 1000.0) - - key_bytes = self._serialize( - self.config['key_serializer'], - topic, key) - value_bytes = self._serialize( - self.config['value_serializer'], - topic, value) - assert type(key_bytes) in (bytes, bytearray, memoryview, type(None)) - assert type(value_bytes) in (bytes, bytearray, memoryview, type(None)) - - partition = self._partition(topic, partition, key, value, - key_bytes, value_bytes) + assigned_partition = None + elapsed = 0.0 + begin = time.time() + timeout = self.config['max_block_ms'] / 1000 + while assigned_partition is None and elapsed < timeout: + elapsed = time.time() - begin + self._wait_on_metadata(topic, timeout - elapsed) + + key_bytes = self._serialize( + self.config['key_serializer'], + topic, key) + value_bytes = self._serialize( + self.config['value_serializer'], + topic, value) + assert type(key_bytes) in (bytes, bytearray, memoryview, type(None)) + assert type(value_bytes) in (bytes, bytearray, memoryview, type(None)) + + assigned_partition = self._partition(topic, partition, key, value, + key_bytes, value_bytes) + if assigned_partition is None: + raise Errors.KafkaTimeoutError("Failed to assign partition for message after %s secs." % timeout) + else: + partition = assigned_partition if headers is None: headers = [] @@ -710,6 +720,10 @@ def _wait_on_metadata(self, topic, max_wait): if partitions is not None: return partitions + if elapsed >= max_wait: + raise Errors.KafkaTimeoutError( + "Failed to update metadata after %.1f secs." % (max_wait,)) + if not metadata_event: metadata_event = threading.Event() @@ -720,13 +734,13 @@ def _wait_on_metadata(self, topic, max_wait): future.add_both(lambda e, *args: e.set(), metadata_event) self._sender.wakeup() metadata_event.wait(max_wait - elapsed) - elapsed = time.time() - begin if not metadata_event.is_set(): raise Errors.KafkaTimeoutError( "Failed to update metadata after %.1f secs." % (max_wait,)) elif topic in self._metadata.unauthorized_topics: raise Errors.TopicAuthorizationFailedError(topic) else: + elapsed = time.time() - begin log.debug("_wait_on_metadata woke after %s secs.", elapsed) def _serialize(self, f, topic, data): @@ -738,16 +752,18 @@ def _serialize(self, f, topic, data): def _partition(self, topic, partition, key, value, serialized_key, serialized_value): + all_partitions = self._metadata.partitions_for_topic(topic) + available = self._metadata.available_partitions_for_topic(topic) + if all_partitions is None or available is None: + return None if partition is not None: assert partition >= 0 - assert partition in self._metadata.partitions_for_topic(topic), 'Unrecognized partition' + assert partition in all_partitions, 'Unrecognized partition' return partition - all_partitions = sorted(self._metadata.partitions_for_topic(topic)) - available = list(self._metadata.available_partitions_for_topic(topic)) return self.config['partitioner'](serialized_key, - all_partitions, - available) + sorted(all_partitions), + list(available)) def metrics(self, raw=False): """Get metrics on producer performance. From 09462d03f79d68325ae3de7e96da791418eb94c7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 13:02:58 -0700 Subject: [PATCH 014/205] fixup TopicAuthorizationFailedError construction --- kafka/consumer/fetcher.py | 2 +- kafka/producer/kafka.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index a833a5b79..ebc6f6ba5 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -920,7 +920,7 @@ def _parse_fetched_data(self, completed_fetch): elif error_type is Errors.TopicAuthorizationFailedError: log.warning("Not authorized to read from topic %s.", tp.topic) - raise Errors.TopicAuthorizationFailedError(set(tp.topic)) + raise Errors.TopicAuthorizationFailedError(set([tp.topic])) elif error_type.is_retriable: log.debug("Retriable error fetching partition %s: %s", tp, error_type()) if error_type.invalid_metadata: diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index f7e2b5fa4..e30e9b7be 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -738,7 +738,7 @@ def _wait_on_metadata(self, topic, max_wait): raise Errors.KafkaTimeoutError( "Failed to update metadata after %.1f secs." % (max_wait,)) elif topic in self._metadata.unauthorized_topics: - raise Errors.TopicAuthorizationFailedError(topic) + raise Errors.TopicAuthorizationFailedError(set([topic])) else: elapsed = time.time() - begin log.debug("_wait_on_metadata woke after %s secs.", elapsed) From 6f8de582688aeebc9d49d756bea2b68750a0eb44 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 13:40:59 -0700 Subject: [PATCH 015/205] mask unused afi var --- kafka/client_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index b72c05dac..e4a96ee44 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -408,7 +408,7 @@ def _should_recycle_connection(self, conn): if broker is None: return False - host, _, afi = get_ip_port_afi(broker.host) + host, _, _ = get_ip_port_afi(broker.host) if conn.host != host or conn.port != broker.port: log.info("Broker metadata change detected for node %s" " from %s:%s to %s:%s", conn.node_id, conn.host, conn.port, From 5c7fc8b6ac29d75dacb64d8782a92fd58e315cf2 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 13:47:13 -0700 Subject: [PATCH 016/205] Remove unused partial KIP-467 implementation (ProduceResponse batch error details) (#2524) --- kafka/producer/record_accumulator.py | 14 ++++++++------ kafka/producer/sender.py | 20 +++++++++----------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index f13c21b9f..4f08b8c08 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -68,17 +68,19 @@ def try_append(self, timestamp_ms, key, value, headers): sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1) return future - def done(self, base_offset=None, timestamp_ms=None, exception=None, log_start_offset=None, global_error=None): - level = logging.DEBUG if exception is None else logging.WARNING - log.log(level, "Produced messages to topic-partition %s with base offset" - " %s log start offset %s and error %s.", self.topic_partition, base_offset, - log_start_offset, global_error) # trace + def done(self, base_offset=None, timestamp_ms=None, exception=None, log_start_offset=None): if self.produce_future.is_done: log.warning('Batch is already closed -- ignoring batch.done()') return elif exception is None: + log.debug("Produced messages to topic-partition %s with base offset" + " %s log start offset %s.", self.topic_partition, base_offset, + log_start_offset) # trace self.produce_future.success((base_offset, timestamp_ms, log_start_offset)) else: + log.warning("Failed to produce messages to topic-partition %s with base offset" + " %s log start offset %s and error %s.", self.topic_partition, base_offset, + log_start_offset, exception) # trace self.produce_future.failure(exception) def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full): @@ -109,7 +111,7 @@ def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full) if error: self.records.close() - self.done(-1, None, Errors.KafkaTimeoutError( + self.done(base_offset=-1, exception=Errors.KafkaTimeoutError( "Batch for %s containing %s record(s) expired: %s" % ( self.topic_partition, self.records.next_offset(), error))) return True diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 3dd52ba76..0e2ea577e 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -182,7 +182,7 @@ def add_topic(self, topic): def _failed_produce(self, batches, node_id, error): log.error("Error sending produce request to node %d: %s", node_id, error) # trace for batch in batches: - self._complete_batch(batch, error, -1, None) + self._complete_batch(batch, error, -1) def _handle_produce_response(self, node_id, send_time, batches, response): """Handle a produce response.""" @@ -194,7 +194,6 @@ def _handle_produce_response(self, node_id, send_time, batches, response): for topic, partitions in response.topics: for partition_info in partitions: - global_error = None log_start_offset = None if response.API_VERSION < 2: partition, error_code, offset = partition_info @@ -204,19 +203,19 @@ def _handle_produce_response(self, node_id, send_time, batches, response): elif 5 <= response.API_VERSION <= 7: partition, error_code, offset, ts, log_start_offset = partition_info else: - # the ignored parameter is record_error of type list[(batch_index: int, error_message: str)] - partition, error_code, offset, ts, log_start_offset, _, global_error = partition_info + # Currently unused / TODO: KIP-467 + partition, error_code, offset, ts, log_start_offset, _record_errors, _global_error = partition_info tp = TopicPartition(topic, partition) error = Errors.for_code(error_code) batch = batches_by_partition[tp] - self._complete_batch(batch, error, offset, ts, log_start_offset, global_error) + self._complete_batch(batch, error, offset, timestamp_ms=ts, log_start_offset=log_start_offset) else: # this is the acks = 0 case, just complete all requests for batch in batches: - self._complete_batch(batch, None, -1, None) + self._complete_batch(batch, None, -1) - def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_start_offset=None, global_error=None): + def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_start_offset=None): """Complete or retry the given batch of records. Arguments: @@ -224,8 +223,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star 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 - log_start_offset (int): The start offset of the log at the time this produce response was created - global_error (str): The summarising error message + log_start_offset (int, optional): The start offset of the log at the time this produce response was created """ # Standardize no-error to None if error is Errors.NoError: @@ -237,7 +235,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star " retrying (%d attempts left). Error: %s", batch.topic_partition, self.config['retries'] - batch.attempts - 1, - global_error or error) + error) self._accumulator.reenqueue(batch) self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) else: @@ -245,7 +243,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star error = error(batch.topic_partition.topic) # tell the user the result of their request - batch.done(base_offset, timestamp_ms, error, log_start_offset, global_error) + batch.done(base_offset, timestamp_ms, error, log_start_offset) self._accumulator.deallocate(batch) if error is not None: self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) From e90514343949e3712e21a4435128de14fa8224a9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 13:47:46 -0700 Subject: [PATCH 017/205] Move kafka.oauth.AbstractTokenProvider -> kafka.sasl.oauth.AbstractTokenProvider (#2525) --- kafka/admin/client.py | 4 ++-- kafka/client_async.py | 4 ++-- kafka/conn.py | 4 ++-- kafka/consumer/group.py | 4 ++-- kafka/oauth/__init__.py | 3 --- kafka/oauth/abstract.py | 42 ----------------------------------- kafka/producer/kafka.py | 4 ++-- kafka/sasl/oauth.py | 49 ++++++++++++++++++++++++++++++++++++++--- 8 files changed, 56 insertions(+), 58 deletions(-) delete mode 100644 kafka/oauth/__init__.py delete mode 100644 kafka/oauth/abstract.py diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 27ad69312..c46bc7f3a 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -149,8 +149,8 @@ class KafkaAdminClient(object): sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI sasl mechanism handshake. Default: one of bootstrap servers - sasl_oauth_token_provider (AbstractTokenProvider): OAuthBearer token provider - instance. (See kafka.oauth.abstract). Default: None + sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer + token provider instance. Default: None kafka_client (callable): Custom class / callable for creating KafkaClient instances """ diff --git a/kafka/client_async.py b/kafka/client_async.py index e4a96ee44..c8a8ca4ad 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -171,8 +171,8 @@ class KafkaClient(object): sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI sasl mechanism handshake. Default: one of bootstrap servers - sasl_oauth_token_provider (AbstractTokenProvider): OAuthBearer token provider - instance. (See kafka.oauth.abstract). Default: None + sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer + token provider instance. Default: None """ DEFAULT_CONFIG = { diff --git a/kafka/conn.py b/kafka/conn.py index 588b5fd86..7af7459da 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -183,8 +183,8 @@ class BrokerConnection(object): sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI sasl mechanism handshake. Default: one of bootstrap servers - sasl_oauth_token_provider (AbstractTokenProvider): OAuthBearer token provider - instance. (See kafka.oauth.abstract). Default: None + sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer + token provider instance. Default: None """ DEFAULT_CONFIG = { diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 27be4588d..d517acf13 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -258,8 +258,8 @@ class KafkaConsumer(six.Iterator): sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI sasl mechanism handshake. Default: one of bootstrap servers - sasl_oauth_token_provider (AbstractTokenProvider): OAuthBearer token provider - instance. (See kafka.oauth.abstract). Default: None + sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer + token provider instance. Default: None kafka_client (callable): Custom class / callable for creating KafkaClient instances Note: diff --git a/kafka/oauth/__init__.py b/kafka/oauth/__init__.py deleted file mode 100644 index 8c8349564..000000000 --- a/kafka/oauth/__init__.py +++ /dev/null @@ -1,3 +0,0 @@ -from __future__ import absolute_import - -from kafka.oauth.abstract import AbstractTokenProvider diff --git a/kafka/oauth/abstract.py b/kafka/oauth/abstract.py deleted file mode 100644 index 8d89ff51d..000000000 --- a/kafka/oauth/abstract.py +++ /dev/null @@ -1,42 +0,0 @@ -from __future__ import absolute_import - -import abc - -# This statement is compatible with both Python 2.7 & 3+ -ABC = abc.ABCMeta('ABC', (object,), {'__slots__': ()}) - -class AbstractTokenProvider(ABC): - """ - A Token Provider must be used for the SASL OAuthBearer protocol. - - The implementation should ensure token reuse so that multiple - calls at connect time do not create multiple tokens. The implementation - should also periodically refresh the token in order to guarantee - that each call returns an unexpired token. A timeout error should - be returned after a short period of inactivity so that the - broker can log debugging info and retry. - - Token Providers MUST implement the token() method - """ - - def __init__(self, **config): - pass - - @abc.abstractmethod - def token(self): - """ - Returns a (str) ID/Access Token to be sent to the Kafka - client. - """ - pass - - def extensions(self): - """ - This is an OPTIONAL method that may be implemented. - - Returns a map of key-value pairs that can - be sent with the SASL/OAUTHBEARER initial client request. If - not implemented, the values are ignored. This feature is only available - in Kafka >= 2.1.0. - """ - return {} diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index e30e9b7be..1b9b12817 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -297,8 +297,8 @@ class KafkaProducer(object): sasl mechanism handshake. Default: 'kafka' sasl_kerberos_domain_name (str): kerberos domain name to use in GSSAPI sasl mechanism handshake. Default: one of bootstrap servers - sasl_oauth_token_provider (AbstractTokenProvider): OAuthBearer token provider - instance. (See kafka.oauth.abstract). Default: None + sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer + token provider instance. Default: None kafka_client (callable): Custom class / callable for creating KafkaClient instances Note: diff --git a/kafka/sasl/oauth.py b/kafka/sasl/oauth.py index d4f643d84..4041a93bd 100644 --- a/kafka/sasl/oauth.py +++ b/kafka/sasl/oauth.py @@ -1,5 +1,7 @@ from __future__ import absolute_import +import abc + from kafka.sasl.abc import SaslMechanism @@ -7,8 +9,9 @@ class SaslMechanismOAuth(SaslMechanism): def __init__(self, **config): assert 'sasl_oauth_token_provider' in config, 'sasl_oauth_token_provider required for OAUTHBEARER sasl' + assert isinstance(config['sasl_oauth_token_provider'], AbstractTokenProvider), \ + 'sasl_oauth_token_provider must implement kafka.sasl.oauth.AbstractTokenProvider' self.token_provider = config['sasl_oauth_token_provider'] - assert callable(getattr(self.token_provider, 'token', None)), 'sasl_oauth_token_provider must implement method #token()' self._is_done = False self._is_authenticated = False @@ -32,9 +35,8 @@ def _token_extensions(self): Return a string representation of the OPTIONAL key-value pairs that can be sent with an OAUTHBEARER initial request. """ - # Only run if the #extensions() method is implemented by the clients Token Provider class # Builds up a string separated by \x01 via a dict of key value pairs - extensions = getattr(self.token_provider, 'extensions', lambda: [])() + extensions = self.token_provider.extensions() msg = '\x01'.join(['{}={}'.format(k, v) for k, v in extensions.items()]) return '\x01' + msg if msg else '' @@ -42,3 +44,44 @@ def auth_details(self): if not self.is_authenticated: raise RuntimeError('Not authenticated yet!') return 'Authenticated via SASL / OAuth' + +# This statement is compatible with both Python 2.7 & 3+ +ABC = abc.ABCMeta('ABC', (object,), {'__slots__': ()}) + +class AbstractTokenProvider(ABC): + """ + A Token Provider must be used for the SASL OAuthBearer protocol. + + The implementation should ensure token reuse so that multiple + calls at connect time do not create multiple tokens. The implementation + should also periodically refresh the token in order to guarantee + that each call returns an unexpired token. A timeout error should + be returned after a short period of inactivity so that the + broker can log debugging info and retry. + + Token Providers MUST implement the token() method + """ + + def __init__(self, **config): + pass + + @abc.abstractmethod + def token(self): + """ + Returns a (str) ID/Access Token to be sent to the Kafka + client. + """ + pass + + def extensions(self): + """ + This is an OPTIONAL method that may be implemented. + + Returns a map of key-value pairs that can + be sent with the SASL/OAUTHBEARER initial client request. If + not implemented, the values are ignored. This feature is only available + in Kafka >= 2.1.0. + + All returned keys and values should be type str + """ + return {} From 227a3124bf39f3717d8b7428094413a3f6d62489 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 14:57:31 -0700 Subject: [PATCH 018/205] Improve ClusterMetadata docs re: node_id/broker_id str/int types --- kafka/cluster.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index c28d36d20..fd5abe852 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -93,7 +93,7 @@ def broker_metadata(self, broker_id): """Get BrokerMetadata Arguments: - broker_id (int): node_id for a broker to check + broker_id (int or str): node_id for a broker to check Returns: BrokerMetadata or None if not found @@ -148,7 +148,7 @@ def partitions_for_broker(self, broker_id): """Return TopicPartitions for which the broker is a leader. Arguments: - broker_id (int): node id for a broker + broker_id (int or str): node id for a broker Returns: set: {TopicPartition, ...} @@ -163,7 +163,7 @@ def coordinator_for_group(self, group): group (str): name of consumer group Returns: - int: node_id for group coordinator + node_id (int or str) for group coordinator, -1 if coordinator unknown None if the group does not exist. """ return self._groups.get(group) From efb554d8ecdd41957a28774863644280ff98b20f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 16:01:23 -0700 Subject: [PATCH 019/205] Cache servers/dist in github actions workflow (#2527) --- .github/workflows/python-package.yml | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 1b0d71c89..ec4b8ec1e 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -72,8 +72,20 @@ jobs: with: distribution: temurin java-version: 23 - - name: Pull Kafka release + - name: Restore cached kafka releases + id: cache-servers-dist-restore + uses: actions/cache/restore@v4 + with: + path: servers/dist + key: servers-dist-${{ matrix.kafka }} + - name: Install Kafka release run: make servers/${{ matrix.kafka }}/kafka-bin + - name: Update kafka release cache + id: cache-servers-dist-save + uses: actions/cache/save@v4 + with: + path: servers/dist + key: ${{ steps.cache-servers-dist-restore.outputs.cache-primary-key }} - name: Pytest run: make test env: From 36287cb4fa8ab453f75500770e0af17cfca0f816 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 16:01:58 -0700 Subject: [PATCH 020/205] Timeout coordinator poll / ensure_coordinator_ready / ensure_active_group (#2526) --- kafka/consumer/group.py | 17 +++++---- kafka/coordinator/base.py | 56 ++++++++++++++++++++++------- kafka/coordinator/consumer.py | 66 +++++++++++++++++++++-------------- 3 files changed, 95 insertions(+), 44 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index d517acf13..284d52f04 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -694,7 +694,8 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): Returns: dict: Map of topic to list of records (may be empty). """ - self._coordinator.poll() + begin = time.time() + self._coordinator.poll(timeout_ms=timeout_ms) # Fetch positions if we have partitions we're subscribed to that we # don't know the offset for @@ -720,7 +721,8 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): if len(futures): self._client.poll(timeout_ms=0) - timeout_ms = min(timeout_ms, self._coordinator.time_to_next_poll() * 1000) + timeout_ms -= (time.time() - begin) * 1000 + timeout_ms = max(0, min(timeout_ms, self._coordinator.time_to_next_poll() * 1000)) self._client.poll(timeout_ms=timeout_ms) # after the long poll, we should check whether the group needs to rebalance # prior to returning data so that the group can stabilize faster @@ -1134,7 +1136,7 @@ def _update_fetch_positions(self, partitions): self._fetcher.update_fetch_positions(partitions) def _message_generator_v2(self): - timeout_ms = 1000 * (self._consumer_timeout - time.time()) + 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): # Generators are stateful, and it is possible that the tp / records @@ -1154,17 +1156,20 @@ def _message_generator_v2(self): def _message_generator(self): assert self.assignment() or self.subscription() is not None, 'No topic subscription or manual partition assignment' + + def inner_poll_ms(): + return max(0, min((1000 * (self._consumer_timeout - time.time())), self.config['retry_backoff_ms'])) + while time.time() < self._consumer_timeout: - self._coordinator.poll() + self._coordinator.poll(timeout_ms=inner_poll_ms()) # Fetch offsets for any subscribed partitions that we arent tracking yet if not self._subscription.has_all_fetch_positions(): partitions = self._subscription.missing_fetch_positions() self._update_fetch_positions(partitions) - poll_ms = min((1000 * (self._consumer_timeout - time.time())), self.config['retry_backoff_ms']) - self._client.poll(timeout_ms=poll_ms) + self._client.poll(timeout_ms=inner_poll_ms()) # after the long poll, we should check whether the group needs to rebalance # prior to returning data so that the group can stabilize faster diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index a30b5a9b8..0d4aedb88 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -234,10 +234,25 @@ def coordinator(self): else: return self.coordinator_id - def ensure_coordinator_ready(self): - """Block until the coordinator for this group is known - (and we have an active connection -- java client uses unsent queue). + def ensure_coordinator_ready(self, timeout_ms=None): + """Block until the coordinator for this group is known. + + Keyword Arguments: + timeout_ms (numeric, optional): Maximum number of milliseconds to + block waiting to find coordinator. Default: None. + + Raises: KafkaTimeoutError if timeout_ms is not None """ + elapsed = 0.0 # noqa: F841 + begin = time.time() + def inner_timeout_ms(): + if timeout_ms is None: + return None + elapsed = (time.time() - begin) * 1000 + if elapsed >= timeout_ms: + raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') + return max(0, timeout_ms - elapsed) + with self._client._lock, self._lock: while self.coordinator_unknown(): @@ -251,16 +266,16 @@ def ensure_coordinator_ready(self): continue future = self.lookup_coordinator() - self._client.poll(future=future) + self._client.poll(future=future, timeout_ms=inner_timeout_ms()) if future.failed(): if future.retriable(): if getattr(future.exception, 'invalid_metadata', False): log.debug('Requesting metadata for group coordinator request: %s', future.exception) metadata_update = self._client.cluster.request_update() - self._client.poll(future=metadata_update) + self._client.poll(future=metadata_update, timeout_ms=inner_timeout_ms()) else: - time.sleep(self.config['retry_backoff_ms'] / 1000) + time.sleep(min(inner_timeout_ms(), self.config['retry_backoff_ms']) / 1000) else: raise future.exception # pylint: disable-msg=raising-bad-type @@ -339,14 +354,31 @@ def _handle_join_failure(self, _): with self._lock: self.state = MemberState.UNJOINED - def ensure_active_group(self): - """Ensure that the group is active (i.e. joined and synced)""" + def ensure_active_group(self, timeout_ms=None): + """Ensure that the group is active (i.e. joined and synced) + + Keyword Arguments: + timeout_ms (numeric, optional): Maximum number of milliseconds to + block waiting to join group. Default: None. + + Raises: KafkaTimeoutError if timeout_ms is not None + """ with self._client._lock, self._lock: if self._heartbeat_thread is None: self._start_heartbeat_thread() + elapsed = 0.0 # noqa: F841 + begin = time.time() + def inner_timeout_ms(): + if timeout_ms is None: + return None + elapsed = (time.time() - begin) * 1000 + if elapsed >= timeout_ms: + raise Errors.KafkaTimeoutError() + return max(0, timeout_ms - elapsed) + while self.need_rejoin() or self._rejoin_incomplete(): - self.ensure_coordinator_ready() + self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) # call on_join_prepare if needed. We set a flag # to make sure that we do not call it a second @@ -367,7 +399,7 @@ def ensure_active_group(self): while not self.coordinator_unknown(): if not self._client.in_flight_request_count(self.coordinator_id): break - self._client.poll(timeout_ms=200) + self._client.poll(timeout_ms=min(200, inner_timeout_ms())) else: continue @@ -400,7 +432,7 @@ def ensure_active_group(self): else: future = self.join_future - self._client.poll(future=future) + self._client.poll(future=future, timeout_ms=inner_timeout_ms()) if future.succeeded(): self._on_join_complete(self._generation.generation_id, @@ -419,7 +451,7 @@ def ensure_active_group(self): continue elif not future.retriable(): raise exception # pylint: disable-msg=raising-bad-type - time.sleep(self.config['retry_backoff_ms'] / 1000) + time.sleep(min(inner_timeout_ms(), self.config['retry_backoff_ms']) / 1000) def _rejoin_incomplete(self): return self.join_future is not None diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 9c662ce7f..73cf25297 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -258,7 +258,7 @@ def _on_join_complete(self, generation, member_id, protocol, self._subscription.listener, self.group_id, assigned) - def poll(self): + def poll(self, timeout_ms=None): """ Poll for coordinator events. Only applicable if group_id is set, and broker version supports GroupCoordinators. This ensures that the @@ -269,31 +269,45 @@ def poll(self): if self.group_id is None: return - self._invoke_completed_offset_commit_callbacks() - self.ensure_coordinator_ready() - - if self.config['api_version'] >= (0, 9) and self._subscription.partitions_auto_assigned(): - if self.need_rejoin(): - # due to a race condition between the initial metadata fetch and the - # initial rebalance, we need to ensure that the metadata is fresh - # before joining initially, and then request the metadata update. If - # metadata update arrives while the rebalance is still pending (for - # example, when the join group is still inflight), then we will lose - # track of the fact that we need to rebalance again to reflect the - # change to the topic subscription. Without ensuring that the - # metadata is fresh, any metadata update that changes the topic - # subscriptions and arrives while a rebalance is in progress will - # essentially be ignored. See KAFKA-3949 for the complete - # description of the problem. - if self._subscription.subscribed_pattern: - metadata_update = self._client.cluster.request_update() - self._client.poll(future=metadata_update) - - self.ensure_active_group() - - self.poll_heartbeat() - - self._maybe_auto_commit_offsets_async() + elapsed = 0.0 # noqa: F841 + begin = time.time() + def inner_timeout_ms(): + if timeout_ms is None: + return None + elapsed = (time.time() - begin) * 1000 + if elapsed >= timeout_ms: + raise Errors.KafkaTimeoutError() + return max(0, timeout_ms - elapsed) + + try: + self._invoke_completed_offset_commit_callbacks() + self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + + if self.config['api_version'] >= (0, 9) and self._subscription.partitions_auto_assigned(): + if self.need_rejoin(): + # due to a race condition between the initial metadata fetch and the + # initial rebalance, we need to ensure that the metadata is fresh + # before joining initially, and then request the metadata update. If + # metadata update arrives while the rebalance is still pending (for + # example, when the join group is still inflight), then we will lose + # track of the fact that we need to rebalance again to reflect the + # change to the topic subscription. Without ensuring that the + # metadata is fresh, any metadata update that changes the topic + # subscriptions and arrives while a rebalance is in progress will + # essentially be ignored. See KAFKA-3949 for the complete + # description of the problem. + if self._subscription.subscribed_pattern: + metadata_update = self._client.cluster.request_update() + self._client.poll(future=metadata_update, timeout_ms=inner_timeout_ms()) + + self.ensure_active_group(timeout_ms=inner_timeout_ms()) + + self.poll_heartbeat() + + self._maybe_auto_commit_offsets_async() + + except Errors.KafkaTimeoutError: + return def time_to_next_poll(self): """Return seconds (float) remaining until :meth:`.poll` should be called again""" From a621bdf9f8b5bfc7d0890dc797735682f6c21950 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 21:27:59 -0700 Subject: [PATCH 021/205] Only mark coordinator dead if connection_delay > 0 (#2530) --- kafka/coordinator/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 0d4aedb88..89752bb27 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -228,7 +228,7 @@ def coordinator(self): """ if self.coordinator_id is None: return None - elif self._client.is_disconnected(self.coordinator_id): + elif self._client.is_disconnected(self.coordinator_id) and self._client.connection_delay(self.coordinator_id) > 0: self.coordinator_dead('Node Disconnected') return None else: From fdf9b22d8ed967b9e5d0fdcdecabdd485542d6d6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 21:29:32 -0700 Subject: [PATCH 022/205] Add inner_timeout_ms handler to fetcher; add fallback (#2529) --- kafka/consumer/fetcher.py | 38 ++++++++++++++++++++++---------------- kafka/coordinator/base.py | 26 ++++++++++++++++---------- 2 files changed, 38 insertions(+), 26 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index ebc6f6ba5..2600d7f69 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -1,4 +1,4 @@ -from __future__ import absolute_import +from __future__ import absolute_import, division import collections import copy @@ -246,7 +246,7 @@ def _reset_offset(self, partition): else: log.debug("Could not find offset for partition %s since it is probably deleted" % (partition,)) - def _retrieve_offsets(self, timestamps, timeout_ms=float("inf")): + def _retrieve_offsets(self, timestamps, timeout_ms=None): """Fetch offset for each partition passed in ``timestamps`` map. Blocks until offsets are obtained, a non-retriable exception is raised @@ -266,29 +266,38 @@ def _retrieve_offsets(self, timestamps, timeout_ms=float("inf")): if not timestamps: return {} - start_time = time.time() - remaining_ms = timeout_ms + elapsed = 0.0 # noqa: F841 + begin = time.time() + def inner_timeout_ms(fallback=None): + if timeout_ms is None: + return fallback + elapsed = (time.time() - begin) * 1000 + if elapsed >= timeout_ms: + raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') + ret = max(0, timeout_ms - elapsed) + if fallback is not None: + return min(ret, fallback) + return ret + timestamps = copy.copy(timestamps) - while remaining_ms > 0: + while True: if not timestamps: return {} future = self._send_list_offsets_requests(timestamps) - self._client.poll(future=future, timeout_ms=remaining_ms) + self._client.poll(future=future, timeout_ms=inner_timeout_ms()) if future.succeeded(): return future.value if not future.retriable(): raise future.exception # pylint: disable-msg=raising-bad-type - elapsed_ms = (time.time() - start_time) * 1000 - remaining_ms = timeout_ms - elapsed_ms - if remaining_ms < 0: - break - if future.exception.invalid_metadata: refresh_future = self._client.cluster.request_update() - self._client.poll(future=refresh_future, timeout_ms=remaining_ms) + self._client.poll(future=refresh_future, timeout_ms=inner_timeout_ms()) + + if not future.is_done: + break # Issue #1780 # Recheck partition existence after after a successful metadata refresh @@ -299,10 +308,7 @@ def _retrieve_offsets(self, timestamps, timeout_ms=float("inf")): log.debug("Removed partition %s from offsets retrieval" % (unknown_partition, )) timestamps.pop(unknown_partition) else: - time.sleep(self.config['retry_backoff_ms'] / 1000.0) - - elapsed_ms = (time.time() - start_time) * 1000 - remaining_ms = timeout_ms - elapsed_ms + time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) raise Errors.KafkaTimeoutError( "Failed to get offsets by timestamps in %s ms" % (timeout_ms,)) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 89752bb27..aa8d05e31 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -245,13 +245,16 @@ def ensure_coordinator_ready(self, timeout_ms=None): """ elapsed = 0.0 # noqa: F841 begin = time.time() - def inner_timeout_ms(): + def inner_timeout_ms(fallback=None): if timeout_ms is None: - return None + return fallback elapsed = (time.time() - begin) * 1000 if elapsed >= timeout_ms: raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') - return max(0, timeout_ms - elapsed) + ret = max(0, timeout_ms - elapsed) + if fallback is not None: + return min(ret, fallback) + return ret with self._client._lock, self._lock: while self.coordinator_unknown(): @@ -275,7 +278,7 @@ def inner_timeout_ms(): metadata_update = self._client.cluster.request_update() self._client.poll(future=metadata_update, timeout_ms=inner_timeout_ms()) else: - time.sleep(min(inner_timeout_ms(), self.config['retry_backoff_ms']) / 1000) + time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) else: raise future.exception # pylint: disable-msg=raising-bad-type @@ -369,13 +372,16 @@ def ensure_active_group(self, timeout_ms=None): elapsed = 0.0 # noqa: F841 begin = time.time() - def inner_timeout_ms(): + def inner_timeout_ms(fallback=None): if timeout_ms is None: - return None + return fallback elapsed = (time.time() - begin) * 1000 if elapsed >= timeout_ms: - raise Errors.KafkaTimeoutError() - return max(0, timeout_ms - elapsed) + raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') + ret = max(0, timeout_ms - elapsed) + if fallback is not None: + return min(ret, fallback) + return ret while self.need_rejoin() or self._rejoin_incomplete(): self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) @@ -399,7 +405,7 @@ def inner_timeout_ms(): while not self.coordinator_unknown(): if not self._client.in_flight_request_count(self.coordinator_id): break - self._client.poll(timeout_ms=min(200, inner_timeout_ms())) + self._client.poll(timeout_ms=inner_timeout_ms(200)) else: continue @@ -451,7 +457,7 @@ def inner_timeout_ms(): continue elif not future.retriable(): raise exception # pylint: disable-msg=raising-bad-type - time.sleep(min(inner_timeout_ms(), self.config['retry_backoff_ms']) / 1000) + time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) def _rejoin_incomplete(self): return self.join_future is not None From 693b21017ee3b9f52389666a28e1c286ba2cdf6d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 13 Mar 2025 07:50:12 -0700 Subject: [PATCH 023/205] Move inner_timeout_ms def to kafka.util timeout_ms_fn --- kafka/consumer/fetcher.py | 15 ++------------- kafka/coordinator/base.py | 29 +++-------------------------- kafka/coordinator/consumer.py | 13 ++----------- kafka/util.py | 18 ++++++++++++++++++ 4 files changed, 25 insertions(+), 50 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 2600d7f69..376079bff 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -19,6 +19,7 @@ from kafka.record import MemoryRecords from kafka.serializer import Deserializer from kafka.structs import TopicPartition, OffsetAndMetadata, OffsetAndTimestamp +from kafka.util import timeout_ms_fn log = logging.getLogger(__name__) @@ -266,19 +267,7 @@ def _retrieve_offsets(self, timestamps, timeout_ms=None): if not timestamps: return {} - elapsed = 0.0 # noqa: F841 - begin = time.time() - def inner_timeout_ms(fallback=None): - if timeout_ms is None: - return fallback - elapsed = (time.time() - begin) * 1000 - if elapsed >= timeout_ms: - raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') - ret = max(0, timeout_ms - elapsed) - if fallback is not None: - return min(ret, fallback) - return ret - + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to find coordinator') timestamps = copy.copy(timestamps) while True: if not timestamps: diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index aa8d05e31..d5a601bf5 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -16,6 +16,7 @@ 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.util import timeout_ms_fn log = logging.getLogger('kafka.coordinator') @@ -243,19 +244,7 @@ def ensure_coordinator_ready(self, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms is not None """ - elapsed = 0.0 # noqa: F841 - begin = time.time() - def inner_timeout_ms(fallback=None): - if timeout_ms is None: - return fallback - elapsed = (time.time() - begin) * 1000 - if elapsed >= timeout_ms: - raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') - ret = max(0, timeout_ms - elapsed) - if fallback is not None: - return min(ret, fallback) - return ret - + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to find group coordinator') with self._client._lock, self._lock: while self.coordinator_unknown(): @@ -366,23 +355,11 @@ def ensure_active_group(self, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms is not None """ + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') with self._client._lock, self._lock: if self._heartbeat_thread is None: self._start_heartbeat_thread() - elapsed = 0.0 # noqa: F841 - begin = time.time() - def inner_timeout_ms(fallback=None): - if timeout_ms is None: - return fallback - elapsed = (time.time() - begin) * 1000 - if elapsed >= timeout_ms: - raise Errors.KafkaTimeoutError('Timeout attempting to find coordinator') - ret = max(0, timeout_ms - elapsed) - if fallback is not None: - return min(ret, fallback) - return ret - while self.need_rejoin() or self._rejoin_incomplete(): self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 73cf25297..bdc9b946c 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -19,7 +19,7 @@ from kafka.metrics.stats import Avg, Count, Max, Rate from kafka.protocol.commit import OffsetCommitRequest, OffsetFetchRequest from kafka.structs import OffsetAndMetadata, TopicPartition -from kafka.util import WeakMethod +from kafka.util import timeout_ms_fn, WeakMethod log = logging.getLogger(__name__) @@ -269,16 +269,7 @@ def poll(self, timeout_ms=None): if self.group_id is None: return - elapsed = 0.0 # noqa: F841 - begin = time.time() - def inner_timeout_ms(): - if timeout_ms is None: - return None - elapsed = (time.time() - begin) * 1000 - if elapsed >= timeout_ms: - raise Errors.KafkaTimeoutError() - return max(0, timeout_ms - elapsed) - + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.poll') try: self._invoke_completed_offset_commit_callbacks() self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) diff --git a/kafka/util.py b/kafka/util.py index e31d99305..6d061193a 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -1,8 +1,10 @@ from __future__ import absolute_import import binascii +import time import weakref +from kafka.errors import KafkaTimeoutError from kafka.vendor import six @@ -22,6 +24,22 @@ def crc32(data): from binascii import crc32 +def timeout_ms_fn(timeout_ms, error_message): + elapsed = 0.0 # noqa: F841 + begin = time.time() + def inner_timeout_ms(fallback=None): + if timeout_ms is None: + return fallback + elapsed = (time.time() - begin) * 1000 + if elapsed >= timeout_ms: + raise KafkaTimeoutError(error_message) + ret = max(0, timeout_ms - elapsed) + if fallback is not None: + return min(ret, fallback) + return ret + return inner_timeout_ms + + class WeakMethod(object): """ Callable that weakly references a method and the object it is bound to. It From 913eddc92928955ec5c9fba9e95f7fedf8e56121 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 21:07:33 -0700 Subject: [PATCH 024/205] reset rejoin_needed after _on_join_complete --- kafka/coordinator/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index d5a601bf5..2b598a11c 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -338,7 +338,6 @@ def _handle_join_success(self, member_assignment_bytes): log.info("Successfully joined group %s with generation %s", self.group_id, self._generation.generation_id) self.state = MemberState.STABLE - self.rejoin_needed = False if self._heartbeat_thread: self._heartbeat_thread.enable() @@ -424,6 +423,7 @@ def ensure_active_group(self, timeout_ms=None): future.value) self.join_future = None self.rejoining = False + self.rejoin_needed = False else: self.join_future = None From b1ff0d813dc9f7471bf13eb7b5447ad595d58d06 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 20:44:19 -0700 Subject: [PATCH 025/205] Check for coordinator.poll failure in KafkaConsumer --- kafka/consumer/group.py | 6 ++++-- kafka/coordinator/consumer.py | 5 +++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 284d52f04..ce66c9606 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -695,7 +695,8 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): dict: Map of topic to list of records (may be empty). """ begin = time.time() - self._coordinator.poll(timeout_ms=timeout_ms) + if not self._coordinator.poll(timeout_ms=timeout_ms): + return {} # Fetch positions if we have partitions we're subscribed to that we # don't know the offset for @@ -1162,7 +1163,8 @@ def inner_poll_ms(): while time.time() < self._consumer_timeout: - self._coordinator.poll(timeout_ms=inner_poll_ms()) + if not self._coordinator.poll(timeout_ms=inner_poll_ms()): + continue # Fetch offsets for any subscribed partitions that we arent tracking yet if not self._subscription.has_all_fetch_positions(): diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index bdc9b946c..92c84024c 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -267,7 +267,7 @@ def poll(self, timeout_ms=None): periodic offset commits if they are enabled. """ if self.group_id is None: - return + return True inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.poll') try: @@ -296,9 +296,10 @@ def poll(self, timeout_ms=None): self.poll_heartbeat() self._maybe_auto_commit_offsets_async() + return True except Errors.KafkaTimeoutError: - return + return False def time_to_next_poll(self): """Return seconds (float) remaining until :meth:`.poll` should be called again""" From 034b4bdabfceba56fa7f8fc96345331c79b37857 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 17:54:13 -0700 Subject: [PATCH 026/205] test_group: shorter timeout, more logging, more sleep --- test/test_consumer_group.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index bc04eed48..7d22346d0 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -57,7 +57,7 @@ def consumer_thread(i): bootstrap_servers=connect_str, group_id=group_id, client_id="consumer_thread-%s" % i, - api_version_auto_timeout_ms=30000, + api_version_auto_timeout_ms=5000, heartbeat_interval_ms=500) while not stop[i].is_set(): for tp, records in six.itervalues(consumers[i].poll(timeout_ms=200)): @@ -73,16 +73,18 @@ def consumer_thread(i): threads[i] = t try: - timeout = time.time() + 35 + timeout = time.time() + 15 while True: for c in range(num_consumers): # Verify all consumers have been created if c not in consumers: + logging.info('%s not in consumers list yet...', c) break # Verify all consumers have an assignment elif not consumers[c].assignment(): + logging.info('Consumer %s does not have assignment yet...', c) break # If all consumers exist and have an assignment @@ -96,8 +98,7 @@ def consumer_thread(i): # New generation assignment is not complete until # coordinator.rejoining = False - rejoining = any([consumer._coordinator.rejoining - for consumer in list(consumers.values())]) + rejoining = set([c for c, consumer in list(consumers.items()) if consumer._coordinator.rejoining]) if not rejoining and len(generations) == 1: for c, consumer in list(consumers.items()): @@ -110,6 +111,7 @@ def consumer_thread(i): logging.info('Rejoining: %s, generations: %s', rejoining, generations) time.sleep(1) assert time.time() < timeout, "timeout waiting for assignments" + time.sleep(1) logging.info('Group stabilized; verifying assignment') group_assignment = set() @@ -157,7 +159,6 @@ def test_heartbeat_thread(kafka_broker, topic): consumer = KafkaConsumer(topic, bootstrap_servers=get_connect_str(kafka_broker), group_id=group_id, - api_version_auto_timeout_ms=30000, heartbeat_interval_ms=500) # poll until we have joined group / have assignment From 215b6262287a6bc827429d1d23ace96894623f7a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 12 Mar 2025 17:11:33 -0700 Subject: [PATCH 027/205] Support client.poll with future and timeout_ms --- kafka/client_async.py | 20 ++++++++++++++------ kafka/consumer/fetcher.py | 4 ++++ kafka/coordinator/base.py | 8 ++++++++ test/fixtures.py | 4 +++- test/test_client_async.py | 12 +++++++----- 5 files changed, 36 insertions(+), 12 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index c8a8ca4ad..c04130c82 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -636,11 +636,14 @@ def poll(self, timeout_ms=None, future=None): Returns: list: responses received (can be empty) """ - if timeout_ms is None: - timeout_ms = self.config['request_timeout_ms'] - elif not isinstance(timeout_ms, (int, float)): + if not isinstance(timeout_ms, (int, float, type(None))): raise TypeError('Invalid type for timeout: %s' % type(timeout_ms)) + begin = time.time() + if timeout_ms is not None: + timeout_at = begin + (timeout_ms / 1000) + else: + timeout_at = begin + (self.config['request_timeout_ms'] / 1000) # Loop for futures, break after first loop if None responses = [] while True: @@ -665,11 +668,12 @@ def poll(self, timeout_ms=None, future=None): if future is not None and future.is_done: timeout = 0 else: + user_timeout_ms = 1000 * max(0, timeout_at - time.time()) idle_connection_timeout_ms = self._idle_expiry_manager.next_check_ms() request_timeout_ms = self._next_ifr_request_timeout_ms() - log.debug("Timeouts: user %f, metadata %f, idle connection %f, request %f", timeout_ms, metadata_timeout_ms, idle_connection_timeout_ms, request_timeout_ms) + log.debug("Timeouts: user %f, metadata %f, idle connection %f, request %f", user_timeout_ms, metadata_timeout_ms, idle_connection_timeout_ms, request_timeout_ms) timeout = min( - timeout_ms, + user_timeout_ms, metadata_timeout_ms, idle_connection_timeout_ms, request_timeout_ms) @@ -683,7 +687,11 @@ def poll(self, timeout_ms=None, future=None): # If all we had was a timeout (future is None) - only do one poll # If we do have a future, we keep looping until it is done - if future is None or future.is_done: + if future is None: + break + elif future.is_done: + break + elif timeout_ms is not None and time.time() >= timeout_at: break return responses diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 376079bff..2179e19fc 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -276,6 +276,10 @@ def _retrieve_offsets(self, timestamps, timeout_ms=None): future = self._send_list_offsets_requests(timestamps) self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + # Timeout w/o future completion + if not future.is_done: + break + if future.succeeded(): return future.value if not future.retriable(): diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 2b598a11c..eb4bf7265 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -260,12 +260,17 @@ def ensure_coordinator_ready(self, timeout_ms=None): future = self.lookup_coordinator() self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + if not future.is_done: + raise Errors.KafkaTimeoutError() + if future.failed(): if future.retriable(): if getattr(future.exception, 'invalid_metadata', False): log.debug('Requesting metadata for group coordinator request: %s', future.exception) metadata_update = self._client.cluster.request_update() self._client.poll(future=metadata_update, timeout_ms=inner_timeout_ms()) + if not metadata_update.is_done: + raise Errors.KafkaTimeoutError() else: time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) else: @@ -416,6 +421,9 @@ def ensure_active_group(self, timeout_ms=None): self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + if not future.is_done: + raise Errors.KafkaTimeoutError() + if future.succeeded(): self._on_join_complete(self._generation.generation_id, self._generation.member_id, diff --git a/test/fixtures.py b/test/fixtures.py index f8e2aa746..c9f138ef5 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -14,7 +14,7 @@ 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 +from kafka.errors import InvalidReplicationFactorError, KafkaTimeoutError from kafka.protocol.admin import CreateTopicsRequest from kafka.protocol.metadata import MetadataRequest from test.testutil import env_kafka_version, random_string @@ -555,6 +555,8 @@ def _failure(error): future.error_on_callbacks = True future.add_errback(_failure) self._client.poll(future=future, timeout_ms=timeout) + if not future.is_done: + raise KafkaTimeoutError() return future.value except Exception as exc: time.sleep(1) diff --git a/test/test_client_async.py b/test/test_client_async.py index 015f39365..8582d8fb7 100644 --- a/test/test_client_async.py +++ b/test/test_client_async.py @@ -228,6 +228,9 @@ def test_poll(mocker): ifr_request_timeout = mocker.patch.object(KafkaClient, '_next_ifr_request_timeout_ms') _poll = mocker.patch.object(KafkaClient, '_poll') cli = KafkaClient(api_version=(0, 9)) + now = time.time() + t = mocker.patch('time.time') + t.return_value = now # metadata timeout wins ifr_request_timeout.return_value = float('inf') @@ -346,17 +349,16 @@ def test_maybe_refresh_metadata_cant_send(mocker, client): t.return_value = now # first poll attempts connection - client.poll(timeout_ms=12345678) - client._poll.assert_called_with(12345.678) + client.poll() + client._poll.assert_called() client._init_connect.assert_called_once_with('foobar') # poll while connecting should not attempt a new connection client._connecting.add('foobar') client._can_connect.reset_mock() - client.poll(timeout_ms=12345678) - client._poll.assert_called_with(12345.678) + client.poll() + client._poll.assert_called() assert not client._can_connect.called - assert not client._metadata_refresh_in_progress From 1c87c8213e4a7b3488884d9205e15d8e7da77096 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 13 Mar 2025 08:21:35 -0700 Subject: [PATCH 028/205] KAFKA-4160: Ensure rebalance listener not called with coordinator lock (#1438) --- kafka/coordinator/base.py | 214 +++++++++++++++++++++----------------- 1 file changed, 116 insertions(+), 98 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index eb4bf7265..021c6de68 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -338,7 +338,36 @@ def time_to_next_heartbeat(self): return float('inf') return self.heartbeat.time_to_next_heartbeat() + def _reset_join_group_future(self): + with self._lock: + self.join_future = None + + def _initiate_join_group(self): + with self._lock: + # we store the join future in case we are woken up by the user + # after beginning the rebalance in the call to poll below. + # This ensures that we do not mistakenly attempt to rejoin + # before the pending rebalance has completed. + if self.join_future is None: + self.state = MemberState.REBALANCING + self.join_future = self._send_join_group_request() + + # handle join completion in the callback so that the + # callback will be invoked even if the consumer is woken up + # before finishing the rebalance + self.join_future.add_callback(self._handle_join_success) + + # we handle failures below after the request finishes. + # If the join completes after having been woken up, the + # exception is ignored and we will rejoin + self.join_future.add_errback(self._handle_join_failure) + + return self.join_future + def _handle_join_success(self, member_assignment_bytes): + # handle join completion in the callback so that the callback + # will be invoked even if the consumer is woken up before + # finishing the rebalance with self._lock: log.info("Successfully joined group %s with generation %s", self.group_id, self._generation.generation_id) @@ -347,6 +376,9 @@ def _handle_join_success(self, member_assignment_bytes): self._heartbeat_thread.enable() def _handle_join_failure(self, _): + # we handle failures below after the request finishes. + # if the join completes after having been woken up, + # the exception is ignored and we will rejoin with self._lock: self.state = MemberState.UNJOINED @@ -360,92 +392,67 @@ def ensure_active_group(self, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms is not None """ inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') - with self._client._lock, self._lock: - if self._heartbeat_thread is None: - self._start_heartbeat_thread() - - while self.need_rejoin() or self._rejoin_incomplete(): - self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) - - # call on_join_prepare if needed. We set a flag - # to make sure that we do not call it a second - # time if the client is woken up before a pending - # rebalance completes. This must be called on each - # iteration of the loop because an event requiring - # a rebalance (such as a metadata refresh which - # changes the matched subscription set) can occur - # while another rebalance is still in progress. - if not self.rejoining: - self._on_join_prepare(self._generation.generation_id, - self._generation.member_id) - self.rejoining = True - - # ensure that there are no pending requests to the coordinator. - # This is important in particular to avoid resending a pending - # JoinGroup request. - while not self.coordinator_unknown(): - if not self._client.in_flight_request_count(self.coordinator_id): - break - self._client.poll(timeout_ms=inner_timeout_ms(200)) - else: - continue - - # we store the join future in case we are woken up by the user - # after beginning the rebalance in the call to poll below. - # This ensures that we do not mistakenly attempt to rejoin - # before the pending rebalance has completed. - if self.join_future is None: - # Fence off the heartbeat thread explicitly so that it cannot - # interfere with the join group. Note that this must come after - # the call to _on_join_prepare since we must be able to continue - # sending heartbeats if that callback takes some time. - self._heartbeat_thread.disable() - - self.state = MemberState.REBALANCING - future = self._send_join_group_request() - - self.join_future = future # this should happen before adding callbacks - - # handle join completion in the callback so that the - # callback will be invoked even if the consumer is woken up - # before finishing the rebalance - future.add_callback(self._handle_join_success) - - # we handle failures below after the request finishes. - # If the join completes after having been woken up, the - # exception is ignored and we will rejoin - future.add_errback(self._handle_join_failure) - - else: - future = self.join_future - - self._client.poll(future=future, timeout_ms=inner_timeout_ms()) - - if not future.is_done: - raise Errors.KafkaTimeoutError() + self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + self._start_heartbeat_thread() + self.join_group(timeout_ms=inner_timeout_ms()) - if future.succeeded(): - self._on_join_complete(self._generation.generation_id, - self._generation.member_id, - self._generation.protocol, - future.value) - self.join_future = None - self.rejoining = False - self.rejoin_needed = False + def join_group(self, timeout_ms=None): + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') + while self.need_rejoin(): + self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + + # call on_join_prepare if needed. We set a flag + # to make sure that we do not call it a second + # time if the client is woken up before a pending + # rebalance completes. This must be called on each + # iteration of the loop because an event requiring + # a rebalance (such as a metadata refresh which + # changes the matched subscription set) can occur + # while another rebalance is still in progress. + if not self.rejoining: + self._on_join_prepare(self._generation.generation_id, + self._generation.member_id) + self.rejoining = True + + # fence off the heartbeat thread explicitly so that it cannot + # interfere with the join group. # Note that this must come after + # the call to onJoinPrepare since we must be able to continue + # sending heartbeats if that callback takes some time. + self._disable_heartbeat_thread() + + # ensure that there are no pending requests to the coordinator. + # This is important in particular to avoid resending a pending + # JoinGroup request. + while not self.coordinator_unknown(): + if not self._client.in_flight_request_count(self.coordinator_id): + break + self._client.poll(timeout_ms=inner_timeout_ms(200)) + else: + continue - else: - self.join_future = None - exception = future.exception - if isinstance(exception, (Errors.UnknownMemberIdError, - Errors.RebalanceInProgressError, - Errors.IllegalGenerationError)): - continue - elif not future.retriable(): - raise exception # pylint: disable-msg=raising-bad-type - time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) - - def _rejoin_incomplete(self): - return self.join_future is not None + future = self._initiate_join_group() + self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + if future.is_done: + self._reset_join_group_future() + else: + raise Errors.KafkaTimeoutError() + + if future.succeeded(): + self.rejoining = False + self.rejoin_needed = False + self._on_join_complete(self._generation.generation_id, + self._generation.member_id, + self._generation.protocol, + future.value) + else: + exception = future.exception + if isinstance(exception, (Errors.UnknownMemberIdError, + Errors.RebalanceInProgressError, + Errors.IllegalGenerationError)): + continue + elif not future.retriable(): + raise exception # pylint: disable-msg=raising-bad-type + time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) def _send_join_group_request(self): """Join the group and return the assignment for the next generation. @@ -751,23 +758,31 @@ def request_rejoin(self): self.rejoin_needed = True def _start_heartbeat_thread(self): - if self._heartbeat_thread is None: - log.info('Starting new heartbeat thread') - self._heartbeat_thread = HeartbeatThread(weakref.proxy(self)) - self._heartbeat_thread.daemon = True - self._heartbeat_thread.start() + with self._lock: + if self._heartbeat_thread is None: + log.info('Starting new heartbeat thread') + self._heartbeat_thread = HeartbeatThread(weakref.proxy(self)) + self._heartbeat_thread.daemon = True + self._heartbeat_thread.start() + + def _disable_heartbeat_thread(self): + with self._lock: + if self._heartbeat_thread is not None: + self._heartbeat_thread.disable() def _close_heartbeat_thread(self): - if hasattr(self, '_heartbeat_thread') and self._heartbeat_thread is not None: - log.info('Stopping heartbeat thread') - try: - self._heartbeat_thread.close() - except ReferenceError: - pass - self._heartbeat_thread = None + with self._lock: + if self._heartbeat_thread is not None: + log.info('Stopping heartbeat thread') + try: + self._heartbeat_thread.close() + except ReferenceError: + pass + self._heartbeat_thread = None def __del__(self): - self._close_heartbeat_thread() + if hasattr(self, '_heartbeat_thread'): + self._close_heartbeat_thread() def close(self): """Close the coordinator, leave the current group, @@ -926,12 +941,15 @@ def __init__(self, coordinator): def enable(self): with self.coordinator._lock: + log.debug('Enabling heartbeat thread') self.enabled = True self.coordinator.heartbeat.reset_timeouts() self.coordinator._lock.notify() def disable(self): - self.enabled = False + with self.coordinator._lock: + log.debug('Disabling heartbeat thread') + self.enabled = False def close(self): if self.closed: From 7827889d5279c1334b6f3449babb80506b2101a5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 10:40:38 -0700 Subject: [PATCH 029/205] Set default pytest log level to debug --- pytest.ini | 1 + 1 file changed, 1 insertion(+) diff --git a/pytest.ini b/pytest.ini index f54588733..71912d76f 100644 --- a/pytest.ini +++ b/pytest.ini @@ -1,2 +1,3 @@ [pytest] log_format = %(created)f %(filename)-23s %(threadName)s %(message)s +log_level = DEBUG From a5d46116bd5ba8c62749355b1d0e96f90c04d82e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 10:45:20 -0700 Subject: [PATCH 030/205] Disable zookeeper admin server to avoid port conflicts --- servers/2.4.0/resources/zookeeper.properties | 1 + servers/2.5.0/resources/zookeeper.properties | 1 + servers/2.6.0/resources/zookeeper.properties | 1 + servers/resources/default/zookeeper.properties | 1 + 4 files changed, 4 insertions(+) diff --git a/servers/2.4.0/resources/zookeeper.properties b/servers/2.4.0/resources/zookeeper.properties index e3fd09742..b146fac9e 100644 --- a/servers/2.4.0/resources/zookeeper.properties +++ b/servers/2.4.0/resources/zookeeper.properties @@ -19,3 +19,4 @@ clientPort={port} clientPortAddress={host} # disable the per-ip limit on the number of connections since this is a non-production config maxClientCnxns=0 +admin.enableServer=false diff --git a/servers/2.5.0/resources/zookeeper.properties b/servers/2.5.0/resources/zookeeper.properties index e3fd09742..b146fac9e 100644 --- a/servers/2.5.0/resources/zookeeper.properties +++ b/servers/2.5.0/resources/zookeeper.properties @@ -19,3 +19,4 @@ clientPort={port} clientPortAddress={host} # disable the per-ip limit on the number of connections since this is a non-production config maxClientCnxns=0 +admin.enableServer=false diff --git a/servers/2.6.0/resources/zookeeper.properties b/servers/2.6.0/resources/zookeeper.properties index e3fd09742..b146fac9e 100644 --- a/servers/2.6.0/resources/zookeeper.properties +++ b/servers/2.6.0/resources/zookeeper.properties @@ -19,3 +19,4 @@ clientPort={port} clientPortAddress={host} # disable the per-ip limit on the number of connections since this is a non-production config maxClientCnxns=0 +admin.enableServer=false diff --git a/servers/resources/default/zookeeper.properties b/servers/resources/default/zookeeper.properties index e3fd09742..b146fac9e 100644 --- a/servers/resources/default/zookeeper.properties +++ b/servers/resources/default/zookeeper.properties @@ -19,3 +19,4 @@ clientPort={port} clientPortAddress={host} # disable the per-ip limit on the number of connections since this is a non-production config maxClientCnxns=0 +admin.enableServer=false From 747a1c13366e68bc2bee667ca227ff49844654c0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 10:51:36 -0700 Subject: [PATCH 031/205] Debug log when skipping api_versions request with pre-configured api_version --- kafka/conn.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/conn.py b/kafka/conn.py index 7af7459da..d1eb4119f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -521,6 +521,7 @@ def _try_api_versions_check(self): if self.config['api_version'] is not None: self._api_version = self.config['api_version'] self._api_versions = BROKER_API_VERSIONS[self._api_version] + log.debug('%s: Using pre-configured api_version %s for ApiVersions', self, self._api_version) return True elif self._check_version_idx is None: request = ApiVersionsRequest[self._api_versions_idx]() From 5a09dabb885265865c495527f65ad849089cf8ce Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 11:03:50 -0700 Subject: [PATCH 032/205] Fix external kafka/zk fixtures for testing (#2533) --- test/conftest.py | 22 +++++++--- test/fixtures.py | 108 ++++++++++++++++------------------------------- test/service.py | 10 ++++- 3 files changed, 62 insertions(+), 78 deletions(-) diff --git a/test/conftest.py b/test/conftest.py index d54a91243..bf1fa6687 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -1,24 +1,36 @@ from __future__ import absolute_import +import os 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.fixtures import KafkaFixture, ZookeeperFixture @pytest.fixture(scope="module") def zookeeper(): """Return a Zookeeper fixture""" - zk_instance = ZookeeperFixture.instance() - yield zk_instance - zk_instance.close() + if "ZOOKEEPER_URI" in os.environ: + parse = urlparse(os.environ["ZOOKEEPER_URI"]) + (host, port) = (parse.hostname, parse.port) + yield ZookeeperFixture.instance(host=host, port=port, external=True) + else: + zk_instance = ZookeeperFixture.instance() + yield zk_instance + zk_instance.close() @pytest.fixture(scope="module") -def kafka_broker(kafka_broker_factory): +def kafka_broker(kafka_broker_factory, zookeeper): """Return a Kafka broker fixture""" - return kafka_broker_factory()[0] + if "KAFKA_URI" in os.environ: + parse = urlparse(os.environ["KAFKA_URI"]) + (host, port) = (parse.hostname, parse.port) + return KafkaFixture.instance(0, zookeeper, host=host, port=port, external=True) + else: + return kafka_broker_factory()[0] @pytest.fixture(scope="module") diff --git a/test/fixtures.py b/test/fixtures.py index c9f138ef5..9843d5a2b 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -10,8 +10,7 @@ import uuid import py -from kafka.vendor.six.moves import urllib, range -from kafka.vendor.six.moves.urllib.parse import urlparse # pylint: disable=E0611,F0401 +from kafka.vendor.six.moves import range from kafka import errors, KafkaAdminClient, KafkaClient, KafkaConsumer, KafkaProducer from kafka.errors import InvalidReplicationFactorError, KafkaTimeoutError @@ -74,43 +73,6 @@ def __init__(self): if not os.path.isdir(self.kafka_root): raise FileNotFoundError(self.kafka_root) - @classmethod - def download_official_distribution(cls, - kafka_version=None, - scala_version=None, - output_dir=None): - if not kafka_version: - kafka_version = cls.kafka_version - if not scala_version: - scala_version = cls.scala_version - if not output_dir: - output_dir = os.path.join(cls.project_root, 'servers', 'dist') - - distfile = 'kafka_%s-%s' % (scala_version, kafka_version,) - url_base = 'https://archive.apache.org/dist/kafka/%s/' % (kafka_version,) - output_file = os.path.join(output_dir, distfile + '.tgz') - - if os.path.isfile(output_file): - log.info("Found file already on disk: %s", output_file) - return output_file - - # New tarballs are .tgz, older ones are sometimes .tar.gz - try: - url = url_base + distfile + '.tgz' - log.info("Attempting to download %s", url) - response = urllib.request.urlopen(url) - except urllib.error.HTTPError: - log.exception("HTTP Error") - url = url_base + distfile + '.tar.gz' - log.info("Attempting to download %s", url) - response = urllib.request.urlopen(url) - - log.info("Saving distribution file to %s", output_file) - with open(output_file, 'w') as output_file_fd: - output_file_fd.write(response.read()) - - return output_file - @classmethod def test_resource(cls, filename): path = os.path.join(cls.project_root, "servers", cls.kafka_version, "resources", filename) @@ -169,23 +131,18 @@ def dump_logs(self): class ZookeeperFixture(Fixture): @classmethod - def instance(cls): - if "ZOOKEEPER_URI" in os.environ: - parse = urlparse(os.environ["ZOOKEEPER_URI"]) - (host, port) = (parse.hostname, parse.port) - fixture = ExternalService(host, port) - else: - (host, port) = ("127.0.0.1", None) - fixture = cls(host, port) - + def instance(cls, host=None, port=None, external=False): + if host is None: + host = "127.0.0.1" + fixture = cls(host, port, external=external) fixture.open() return fixture - def __init__(self, host, port, tmp_dir=None): + def __init__(self, host, port, external=False, tmp_dir=None): super(ZookeeperFixture, self).__init__() self.host = host self.port = port - + self.running = external self.tmp_dir = tmp_dir def kafka_run_class_env(self): @@ -198,6 +155,8 @@ def out(self, message): log.info("*** Zookeeper [%s:%s]: %s", self.host, self.port or '(auto)', message) def open(self): + if self.running: + return if self.tmp_dir is None: self.tmp_dir = py.path.local.mkdtemp() #pylint: disable=no-member self.tmp_dir.ensure(dir=True) @@ -262,34 +221,30 @@ class KafkaFixture(Fixture): @classmethod def instance(cls, broker_id, zookeeper, zk_chroot=None, - host=None, port=None, - transport='PLAINTEXT', replicas=1, partitions=2, + host=None, port=None, external=False, + transport='PLAINTEXT', replicas=1, partitions=4, sasl_mechanism=None, auto_create_topic=True, tmp_dir=None): if zk_chroot is None: zk_chroot = "kafka-python_" + str(uuid.uuid4()).replace("-", "_") - if "KAFKA_URI" in os.environ: - parse = urlparse(os.environ["KAFKA_URI"]) - (host, port) = (parse.hostname, parse.port) - fixture = ExternalService(host, port) - else: - if host is None: - host = "localhost" - fixture = KafkaFixture(host, port, broker_id, - zookeeper, zk_chroot, - transport=transport, - replicas=replicas, partitions=partitions, - sasl_mechanism=sasl_mechanism, - auto_create_topic=auto_create_topic, - tmp_dir=tmp_dir) - - fixture.open() + if host is None: + host = "localhost" + fixture = KafkaFixture(host, port, broker_id, + zookeeper, zk_chroot, + external=external, + transport=transport, + replicas=replicas, partitions=partitions, + sasl_mechanism=sasl_mechanism, + auto_create_topic=auto_create_topic, + tmp_dir=tmp_dir) + + fixture.open() return fixture def __init__(self, host, port, broker_id, zookeeper, zk_chroot, replicas=1, partitions=2, transport='PLAINTEXT', sasl_mechanism=None, auto_create_topic=True, - tmp_dir=None): + tmp_dir=None, external=False): super(KafkaFixture, self).__init__() self.host = host @@ -321,9 +276,16 @@ def __init__(self, host, port, broker_id, zookeeper, zk_chroot, self.partitions = partitions self.tmp_dir = tmp_dir - self.running = False + self.external = external + + if self.external: + self.child = ExternalService(self.host, self.port) + (self._client,) = self.get_clients(1, client_id='_internal_client') + self.running = True + else: + self._client = None + self.running = False - self._client = None self.sasl_config = '' self.jaas_config = '' @@ -416,6 +378,8 @@ def _create_zk_chroot(self): self.out("Kafka chroot created in Zookeeper!") def start(self): + if self.running: + return True # Configure Kafka child process properties = self.tmp_dir.join("kafka.properties") jaas_conf = self.tmp_dir.join("kafka_server_jaas.conf") @@ -515,6 +479,8 @@ def __del__(self): self.close() def stop(self): + if self.external: + return if not self.running: self.out("Instance already stopped") return diff --git a/test/service.py b/test/service.py index e4e89f8fe..a53fab8da 100644 --- a/test/service.py +++ b/test/service.py @@ -29,6 +29,11 @@ def open(self): def close(self): pass + def dump_logs(self): + pass + + def wait_for(self, pattern, timeout=30): + pass class SpawnedService(threading.Thread): def __init__(self, args=None, env=None): @@ -52,8 +57,8 @@ def __init__(self, args=None, env=None): log.debug(" {key}={value}".format(key=key, value=value)) def _spawn(self): - if self.alive: return - if self.child and self.child.poll() is None: return + if self.alive or (self.child and self.child.poll() is None): + return self.child = subprocess.Popen( self.args, @@ -76,6 +81,7 @@ def _despawn(self): else: self.child.kill() + # via threading.Thread def run(self): self._spawn() while True: From 17abc60f856efc49b7c278cc9fcc92bc3bf8d976 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 11:13:54 -0700 Subject: [PATCH 033/205] Limit test duration to 5mins w/ pytest-timeout --- Makefile | 2 +- pytest.ini | 1 + requirements-dev.txt | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 2df1c6696..b9f199ef0 100644 --- a/Makefile +++ b/Makefile @@ -21,7 +21,7 @@ lint: pylint --recursive=y --errors-only kafka test test: build-integration - pytest --durations=10 $(PYTESTS) + pytest $(PYTESTS) cov-local: build-integration pytest --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka \ diff --git a/pytest.ini b/pytest.ini index 71912d76f..f6138c932 100644 --- a/pytest.ini +++ b/pytest.ini @@ -1,3 +1,4 @@ [pytest] log_format = %(created)f %(filename)-23s %(threadName)s %(message)s log_level = DEBUG +addopts = --durations=10 --timeout=300 diff --git a/requirements-dev.txt b/requirements-dev.txt index 6cfb6d83b..3bc51fd78 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -10,6 +10,7 @@ pytest pytest-cov pytest-mock pytest-pylint +pytest-timeout python-snappy Sphinx sphinx-rtd-theme From c03dd33bbb27ba5f46871128219106ec9c3f7b05 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 11:19:40 -0700 Subject: [PATCH 034/205] Improve test_consumer_group::test_group logging before group stabilized (#2534) --- test/test_consumer_group.py | 67 ++++++++++++++++++------------------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index 7d22346d0..c175e142c 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -75,43 +75,42 @@ def consumer_thread(i): try: timeout = time.time() + 15 while True: - for c in range(num_consumers): - - # Verify all consumers have been created - if c not in consumers: - logging.info('%s not in consumers list yet...', c) - break - - # Verify all consumers have an assignment - elif not consumers[c].assignment(): - logging.info('Consumer %s does not have assignment yet...', c) - break + assert time.time() < timeout, "timeout waiting for assignments" + # Verify all consumers have been created + missing_consumers = set(consumers.keys()) - set(range(num_consumers)) + if missing_consumers: + logging.info('Waiting on consumer threads: %s', missing_consumers) + time.sleep(1) + continue + + unassigned_consumers = {c for c, consumer in six.iteritems(consumers) if not consumer.assignment()} + if unassigned_consumers: + logging.info('Waiting for consumer assignments: %s', unassigned_consumers) + time.sleep(1) + continue # If all consumers exist and have an assignment + logging.info('All consumers have assignment... checking for stable group') + # 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)]) + + # New generation assignment is not complete until + # coordinator.rejoining = False + rejoining = set([c for c, consumer in six.iteritems(consumers) if consumer._coordinator.rejoining]) + + if not rejoining and len(generations) == 1: + for c, consumer in six.iteritems(consumers): + logging.info("[%s] %s %s: %s", c, + consumer._coordinator._generation.generation_id, + consumer._coordinator._generation.member_id, + consumer.assignment()) + break else: - - logging.info('All consumers have assignment... checking for stable group') - # 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 list(consumers.values())]) - - # New generation assignment is not complete until - # coordinator.rejoining = False - rejoining = set([c for c, consumer in list(consumers.items()) if consumer._coordinator.rejoining]) - - if not rejoining and len(generations) == 1: - for c, consumer in list(consumers.items()): - logging.info("[%s] %s %s: %s", c, - consumer._coordinator._generation.generation_id, - consumer._coordinator._generation.member_id, - consumer.assignment()) - break - else: - logging.info('Rejoining: %s, generations: %s', rejoining, generations) - time.sleep(1) - assert time.time() < timeout, "timeout waiting for assignments" - time.sleep(1) + logging.info('Rejoining: %s, generations: %s', rejoining, generations) + time.sleep(1) + continue logging.info('Group stabilized; verifying assignment') group_assignment = set() From b1d2d2cf0deeed9dfbdd4090841e8eb84145ff09 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 11:51:25 -0700 Subject: [PATCH 035/205] Update pytest log_format: use logger instead of filename; add thread id --- pytest.ini | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pytest.ini b/pytest.ini index f6138c932..7fcb1f4a8 100644 --- a/pytest.ini +++ b/pytest.ini @@ -1,4 +1,4 @@ [pytest] -log_format = %(created)f %(filename)-23s %(threadName)s %(message)s +log_format = %(asctime)s.%(msecs)03d %(levelname)-8s %(thread)d:%(threadName)s %(name)-23s %(message)s log_level = DEBUG addopts = --durations=10 --timeout=300 From 99b04a51acdc2bd1c1e56d757399156156f7e324 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 11:51:51 -0700 Subject: [PATCH 036/205] Add heartbeat thread id to debug logs on start --- kafka/coordinator/base.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 021c6de68..dd40bf5d4 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -764,6 +764,7 @@ def _start_heartbeat_thread(self): self._heartbeat_thread = HeartbeatThread(weakref.proxy(self)) self._heartbeat_thread.daemon = True self._heartbeat_thread.start() + log.debug("Started heartbeat thread %s", self._heartbeat_thread.ident) def _disable_heartbeat_thread(self): with self._lock: From 4b3405d353077385ad86f4abae25b98d576201a1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 12:43:41 -0700 Subject: [PATCH 037/205] Use NullLogger in producer atexit cleanup --- kafka/producer/kafka.py | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 1b9b12817..707daf124 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -449,7 +449,7 @@ def _cleanup_factory(self): _self = weakref.proxy(self) def wrapper(): try: - _self.close(timeout=0) + _self.close(timeout=0, null_logger=True) except (ReferenceError, AttributeError): pass return wrapper @@ -472,22 +472,22 @@ def _unregister_cleanup(self): self._cleanup = None def __del__(self): - # Disable logger during destruction to avoid touching dangling references - class NullLogger(object): - def __getattr__(self, name): - return lambda *args: None + self.close(null_logger=True) - global log - log = NullLogger() - - self.close() - - def close(self, timeout=None): + def close(self, timeout=None, null_logger=False): """Close this producer. Arguments: timeout (float, optional): timeout in seconds to wait for completion. """ + if null_logger: + # Disable logger during destruction to avoid touching dangling references + class NullLogger(object): + def __getattr__(self, name): + return lambda *args: None + + global log + log = NullLogger() # drop our atexit handler now to avoid leaks self._unregister_cleanup() From 7e6f8b1bfb6a6f2c1b2346e1cef83a89b7f84c75 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 12:44:52 -0700 Subject: [PATCH 038/205] Debug log if check_version connection attempt fails --- kafka/client_async.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kafka/client_async.py b/kafka/client_async.py index c04130c82..78ff1c118 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -1052,6 +1052,8 @@ def check_version(self, node_id=None, timeout=None, **kwargs): if conn._api_version is not None: return conn._api_version + else: + log.debug('Failed to identify api_version after connection attempt to %s', conn) # Timeout else: From a25ffae928da60e5f6a774cb1a8612e5c58b51a0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 12:45:24 -0700 Subject: [PATCH 039/205] Include request_timeout_ms in request debug log --- kafka/conn.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index d1eb4119f..2f8c2491c 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -948,6 +948,7 @@ def send(self, request, blocking=True, request_timeout_ms=None): return self._send(request, blocking=blocking, request_timeout_ms=request_timeout_ms) def _send(self, request, blocking=True, request_timeout_ms=None): + request_timeout_ms = request_timeout_ms or self.config['request_timeout_ms'] future = Future() with self._lock: if not self._can_send_recv(): @@ -958,11 +959,10 @@ def _send(self, request, blocking=True, request_timeout_ms=None): correlation_id = self._protocol.send_request(request) - log.debug('%s Request %d: %s', self, correlation_id, request) + log.debug('%s Request %d (timeout_ms %s): %s', self, correlation_id, request_timeout_ms, request) if request.expect_response(): assert correlation_id not in self.in_flight_requests, 'Correlation ID already in-flight!' sent_time = time.time() - request_timeout_ms = request_timeout_ms or self.config['request_timeout_ms'] timeout_at = sent_time + (request_timeout_ms / 1000) self.in_flight_requests[correlation_id] = (future, sent_time, timeout_at) else: From f7c234d377149ddfd6c4d8a2d3ff78f3a0ba0ddb Mon Sep 17 00:00:00 2001 From: Heikki Nousiainen Date: Tue, 29 Sep 2020 21:45:27 +0300 Subject: [PATCH 040/205] Support connections through SOCKS5 proxies Implement support for SOCKS5 proxies. Implement a new proxy wrapper that handles SOCKS5 connection, authentication and requesting connections to the actual Kafka broker endpoints. The proxy can be configured via a new keyword argument `socks5_proxy` to consumers, producers or admin client. The value is URL with optional username and password. E.g. `socks5://user:secret@proxy.example.com:10800` The implementation is done in state machine that makes progress on repeated calls to connect_ex. The rationale with this bit strange design is to minimize amount of changes on the actual BrokerConnection object. --- kafka/admin/client.py | 3 +- kafka/client_async.py | 4 +- kafka/conn.py | 17 ++- kafka/consumer/group.py | 2 + kafka/producer/kafka.py | 2 + kafka/socks5_wrapper.py | 248 ++++++++++++++++++++++++++++++++++++++++ 6 files changed, 271 insertions(+), 5 deletions(-) create mode 100644 kafka/socks5_wrapper.py diff --git a/kafka/admin/client.py b/kafka/admin/client.py index c46bc7f3a..4578d8059 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -151,8 +151,8 @@ class KafkaAdminClient(object): sasl mechanism handshake. Default: one of bootstrap servers sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer token provider instance. Default: None + socks5_proxy (str): Socks5 proxy url. Default: None kafka_client (callable): Custom class / callable for creating KafkaClient instances - """ DEFAULT_CONFIG = { # client configs @@ -188,6 +188,7 @@ class KafkaAdminClient(object): 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, + 'socks5_proxy': None, # metrics configs 'metric_reporters': [], diff --git a/kafka/client_async.py b/kafka/client_async.py index 78ff1c118..835864944 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -173,6 +173,7 @@ class KafkaClient(object): sasl mechanism handshake. Default: one of bootstrap servers sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer token provider instance. Default: None + socks5_proxy (str): Socks5 proxy URL. Default: None """ DEFAULT_CONFIG = { @@ -213,7 +214,8 @@ class KafkaClient(object): 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, - 'sasl_oauth_token_provider': None + 'sasl_oauth_token_provider': None, + 'socks5_proxy': None, } def __init__(self, **configs): diff --git a/kafka/conn.py b/kafka/conn.py index 2f8c2491c..ded935838 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -36,6 +36,7 @@ from kafka.protocol.sasl_handshake import SaslHandshakeRequest from kafka.protocol.types import Int32 from kafka.sasl import get_sasl_mechanism +from kafka.socks5_wrapper import Socks5Wrapper from kafka.version import __version__ @@ -185,6 +186,7 @@ class BrokerConnection(object): sasl mechanism handshake. Default: one of bootstrap servers sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer token provider instance. Default: None + socks5_proxy (str): Socks5 proxy url. Default: None """ DEFAULT_CONFIG = { @@ -220,7 +222,8 @@ class BrokerConnection(object): 'sasl_kerberos_name': None, 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, - 'sasl_oauth_token_provider': None + 'sasl_oauth_token_provider': None, + 'socks5_proxy': None, } SECURITY_PROTOCOLS = ('PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL') VERSION_CHECKS = ( @@ -241,6 +244,7 @@ def __init__(self, host, port, afi, **configs): self._check_version_idx = None self._api_versions_idx = 2 self._throttle_time = None + self._socks5_proxy = None self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: @@ -362,7 +366,11 @@ def connect(self): assert self._sock is None self._sock_afi, self._sock_addr = next_lookup try: - self._sock = socket.socket(self._sock_afi, socket.SOCK_STREAM) + if self.config["socks5_proxy"] is not None: + self._socks5_proxy = Socks5Wrapper(self.config["socks5_proxy"], self.afi) + self._sock = self._socks5_proxy.socket(self._sock_afi, socket.SOCK_STREAM) + else: + self._sock = socket.socket(self._sock_afi, socket.SOCK_STREAM) except (socket.error, OSError) as e: self.close(e) return self.state @@ -382,7 +390,10 @@ def connect(self): # to check connection status ret = None try: - ret = self._sock.connect_ex(self._sock_addr) + if self._socks5_proxy: + ret = self._socks5_proxy.connect_ex(self._sock_addr) + else: + ret = self._sock.connect_ex(self._sock_addr) except socket.error as err: ret = err.errno diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index ce66c9606..751403f52 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -260,6 +260,7 @@ class KafkaConsumer(six.Iterator): sasl mechanism handshake. Default: one of bootstrap servers sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer token provider instance. Default: None + socks5_proxy (str): Socks5 proxy URL. Default: None kafka_client (callable): Custom class / callable for creating KafkaClient instances Note: @@ -325,6 +326,7 @@ class KafkaConsumer(six.Iterator): 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, + 'socks5_proxy': None, 'legacy_iterator': False, # enable to revert to < 1.4.7 iterator 'kafka_client': KafkaClient, } diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 707daf124..47727dd9e 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -299,6 +299,7 @@ class KafkaProducer(object): sasl mechanism handshake. Default: one of bootstrap servers sasl_oauth_token_provider (kafka.sasl.oauth.AbstractTokenProvider): OAuthBearer token provider instance. Default: None + socks5_proxy (str): Socks5 proxy URL. Default: None kafka_client (callable): Custom class / callable for creating KafkaClient instances Note: @@ -355,6 +356,7 @@ class KafkaProducer(object): 'sasl_kerberos_service_name': 'kafka', 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, + 'socks5_proxy': None, 'kafka_client': KafkaClient, } diff --git a/kafka/socks5_wrapper.py b/kafka/socks5_wrapper.py new file mode 100644 index 000000000..18bea7c8d --- /dev/null +++ b/kafka/socks5_wrapper.py @@ -0,0 +1,248 @@ +try: + from urllib.parse import urlparse +except ImportError: + from urlparse import urlparse + +import errno +import logging +import random +import socket +import struct + +log = logging.getLogger(__name__) + + +class ProxyConnectionStates: + DISCONNECTED = '' + CONNECTING = '' + NEGOTIATE_PROPOSE = '' + NEGOTIATING = '' + AUTHENTICATING = '' + REQUEST_SUBMIT = '' + REQUESTING = '' + READ_ADDRESS = '' + COMPLETE = '' + + +class Socks5Wrapper: + """Socks5 proxy wrapper + + Manages connection through socks5 proxy with support for username/password + authentication. + """ + + def __init__(self, proxy_url, afi): + self._buffer_in = b'' + self._buffer_out = b'' + self._proxy_url = urlparse(proxy_url) + self._sock = None + self._state = ProxyConnectionStates.DISCONNECTED + self._target_afi = socket.AF_UNSPEC + + proxy_addrs = self.dns_lookup(self._proxy_url.hostname, self._proxy_url.port, afi) + # TODO raise error on lookup failure + self._proxy_addr = random.choice(proxy_addrs) + + @classmethod + def is_inet_4_or_6(cls, gai): + """Given a getaddrinfo struct, return True iff ipv4 or ipv6""" + return gai[0] in (socket.AF_INET, socket.AF_INET6) + + @classmethod + def dns_lookup(cls, host, port, afi=socket.AF_UNSPEC): + """Returns a list of getaddrinfo structs, optionally filtered to an afi (ipv4 / ipv6)""" + # XXX: all DNS functions in Python are blocking. If we really + # want to be non-blocking here, we need to use a 3rd-party + # library like python-adns, or move resolution onto its + # own thread. This will be subject to the default libc + # name resolution timeout (5s on most Linux boxes) + try: + return list(filter(cls.is_inet_4_or_6, + socket.getaddrinfo(host, port, afi, + socket.SOCK_STREAM))) + except socket.gaierror as ex: + log.warning("DNS lookup failed for proxy %s:%d, %r", host, port, ex) + return [] + + def socket(self, family, sock_type): + """Open and record a socket. + + Returns the actual underlying socket + object to ensure e.g. selects and ssl wrapping works as expected. + """ + self._target_afi = family # Store the address family of the target + afi, _, _, _, _ = self._proxy_addr + self._sock = socket.socket(afi, sock_type) + return self._sock + + def _flush_buf(self): + """Send out all data that is stored in the outgoing buffer. + + It is expected that the caller handles error handling, including non-blocking + as well as connection failure exceptions. + """ + while self._buffer_out: + sent_bytes = self._sock.send(self._buffer_out) + self._buffer_out = self._buffer_out[sent_bytes:] + + def _peek_buf(self, datalen): + """Ensure local inbound buffer has enough data, and return that data without + consuming the local buffer + + It's expected that the caller handles e.g. blocking exceptions""" + while True: + bytes_remaining = datalen - len(self._buffer_in) + if bytes_remaining <= 0: + break + data = self._sock.recv(bytes_remaining) + if not data: + break + self._buffer_in = self._buffer_in + data + + return self._buffer_in[:datalen] + + def _read_buf(self, datalen): + """Read and consume bytes from socket connection + + It's expected that the caller handles e.g. blocking exceptions""" + buf = self._peek_buf(datalen) + if buf: + self._buffer_in = self._buffer_in[len(buf):] + return buf + + def connect_ex(self, addr): + """Runs a state machine through connection to authentication to + proxy connection request. + + The somewhat strange setup is to facilitate non-intrusive use from + BrokerConnection state machine. + + This function is called with a socket in non-blocking mode. Both + send and receive calls can return in EWOULDBLOCK/EAGAIN which we + specifically avoid handling here. These are handled in main + BrokerConnection connection loop, which then would retry calls + to this function.""" + + if self._state == ProxyConnectionStates.DISCONNECTED: + self._state = ProxyConnectionStates.CONNECTING + + if self._state == ProxyConnectionStates.CONNECTING: + _, _, _, _, sockaddr = self._proxy_addr + ret = self._sock.connect_ex(sockaddr) + if not ret or ret == errno.EISCONN: + self._state = ProxyConnectionStates.NEGOTIATE_PROPOSE + else: + return ret + + if self._state == ProxyConnectionStates.NEGOTIATE_PROPOSE: + if self._proxy_url.username and self._proxy_url.password: + # Propose username/password + self._buffer_out = b"\x05\x01\x02" + else: + # Propose no auth + self._buffer_out = b"\x05\x01\x00" + self._state = ProxyConnectionStates.NEGOTIATING + + if self._state == ProxyConnectionStates.NEGOTIATING: + self._flush_buf() + buf = self._read_buf(2) + if buf[0:1] != b"\x05": + log.error("Unrecognized SOCKS version") + self._state = ProxyConnectionStates.DISCONNECTED + self._sock.close() + return errno.ECONNREFUSED + + if buf[1:2] == b"\x00": + # No authentication required + self._state = ProxyConnectionStates.REQUEST_SUBMIT + elif buf[1:2] == b"\x02": + # Username/password authentication selected + userlen = len(self._proxy_url.username) + passlen = len(self._proxy_url.password) + self._buffer_out = struct.pack( + "!bb{}sb{}s".format(userlen, passlen), + 1, # version + userlen, + self._proxy_url.username.encode(), + passlen, + self._proxy_url.password.encode(), + ) + self._state = ProxyConnectionStates.AUTHENTICATING + else: + log.error("Unrecognized SOCKS authentication method") + self._state = ProxyConnectionStates.DISCONNECTED + self._sock.close() + return errno.ECONNREFUSED + + if self._state == ProxyConnectionStates.AUTHENTICATING: + self._flush_buf() + buf = self._read_buf(2) + if buf == b"\x01\x00": + # Authentication succesful + self._state = ProxyConnectionStates.REQUEST_SUBMIT + else: + log.error("Socks5 proxy authentication failure") + self._state = ProxyConnectionStates.DISCONNECTED + self._sock.close() + return errno.ECONNREFUSED + + if self._state == ProxyConnectionStates.REQUEST_SUBMIT: + if self._target_afi == socket.AF_INET: + addr_type = 1 + addr_len = 4 + elif self._target_afi == socket.AF_INET6: + addr_type = 4 + addr_len = 16 + else: + log.error("Unknown address family, %r", self._target_afi) + self._state = ProxyConnectionStates.DISCONNECTED + self._sock.close() + return errno.ECONNREFUSED + + self._buffer_out = struct.pack( + "!bbbb{}sh".format(addr_len), + 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 + ) + self._state = ProxyConnectionStates.REQUESTING + + if self._state == ProxyConnectionStates.REQUESTING: + self._flush_buf() + buf = self._read_buf(2) + if buf[0:2] == b"\x05\x00": + self._state = ProxyConnectionStates.READ_ADDRESS + else: + log.error("Proxy request failed: %r", buf[1:2]) + self._state = ProxyConnectionStates.DISCONNECTED + self._sock.close() + return errno.ECONNREFUSED + + if self._state == ProxyConnectionStates.READ_ADDRESS: + # we don't really care about the remote endpoint address, but need to clear the stream + buf = self._peek_buf(2) + if buf[0:2] == b"\x00\x01": + _ = self._read_buf(2 + 4 + 2) # ipv4 address + port + elif buf[0:2] == b"\x00\x05": + _ = self._read_buf(2 + 16 + 2) # ipv6 address + port + else: + log.error("Unrecognized remote address type %r", buf[1:2]) + self._state = ProxyConnectionStates.DISCONNECTED + self._sock.close() + return errno.ECONNREFUSED + self._state = ProxyConnectionStates.COMPLETE + + if self._state == ProxyConnectionStates.COMPLETE: + return 0 + + # not reached; + # Send and recv will raise socket error on EWOULDBLOCK/EAGAIN that is assumed to be handled by + # the caller. The caller re-enters this state machine from retry logic with timer or via select & family + log.error("Internal error, state %r not handled correctly", self._state) + self._state = ProxyConnectionStates.DISCONNECTED + if self._sock: + self._sock.close() + return errno.ECONNREFUSED From 837df1eadf8d9f221c403f588cc4557197b0c6c5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 13:40:49 -0700 Subject: [PATCH 041/205] AdminClient: support delete_records (#2535) Co-authored-by: Ruslan Co-authored-by: Arsen Kitov --- kafka/admin/client.py | 119 +++++++++++++++++++++++++++++++-- kafka/protocol/admin.py | 32 +++++++++ test/conftest.py | 4 +- test/test_admin_integration.py | 73 +++++++++++++++++++- 4 files changed, 220 insertions(+), 8 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 4578d8059..6c2596cc8 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -15,14 +15,13 @@ from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment, ConsumerProtocol import kafka.errors as Errors from kafka.errors import ( - IncompatibleBrokerVersion, KafkaConfigurationError, NotControllerError, + IncompatibleBrokerVersion, KafkaConfigurationError, NotControllerError, UnknownTopicOrPartitionError, UnrecognizedBrokerVersion, IllegalArgumentError) from kafka.metrics import MetricConfig, Metrics from kafka.protocol.admin import ( CreateTopicsRequest, DeleteTopicsRequest, DescribeConfigsRequest, AlterConfigsRequest, CreatePartitionsRequest, ListGroupsRequest, DescribeGroupsRequest, DescribeAclsRequest, CreateAclsRequest, DeleteAclsRequest, - DeleteGroupsRequest, DescribeLogDirsRequest -) + DeleteGroupsRequest, DeleteRecordsRequest, DescribeLogDirsRequest) from kafka.protocol.commit import OffsetFetchRequest from kafka.protocol.find_coordinator import FindCoordinatorRequest from kafka.protocol.metadata import MetadataRequest @@ -1116,8 +1115,118 @@ def create_partitions(self, topic_partitions, timeout_ms=None, validate_only=Fal .format(version)) return self._send_request_to_controller(request) - # delete records protocol not yet implemented - # Note: send the request to the partition leaders + def _get_leader_for_partitions(self, partitions, timeout_ms=None): + """Finds ID of the leader node for every given topic partition. + + Will raise UnknownTopicOrPartitionError if for some partition no leader can be found. + + :param partitions: ``[TopicPartition]``: partitions for which to find leaders. + :param timeout_ms: ``float``: Timeout in milliseconds, if None (default), will be read from + config. + + :return: Dictionary with ``{leader_id -> {partitions}}`` + """ + timeout_ms = self._validate_timeout(timeout_ms) + + partitions = set(partitions) + topics = set(tp.topic for tp in partitions) + + response = self._get_cluster_metadata(topics=topics).to_object() + + leader2partitions = defaultdict(list) + valid_partitions = set() + for topic in response.get("topics", ()): + for partition in topic.get("partitions", ()): + t2p = TopicPartition(topic=topic["topic"], partition=partition["partition"]) + if t2p in partitions: + leader2partitions[partition["leader"]].append(t2p) + valid_partitions.add(t2p) + + if len(partitions) != len(valid_partitions): + unknown = set(partitions) - valid_partitions + raise UnknownTopicOrPartitionError( + "The following partitions are not known: %s" + % ", ".join(str(x) for x in unknown) + ) + + return leader2partitions + + def delete_records(self, records_to_delete, timeout_ms=None, partition_leader_id=None): + """Delete records whose offset is smaller than the given offset of the corresponding partition. + + :param records_to_delete: ``{TopicPartition: int}``: The earliest available offsets for the + given partitions. + :param timeout_ms: ``float``: Timeout in milliseconds, if None (default), will be read from + config. + :param partition_leader_id: ``str``: If specified, all deletion requests will be sent to + this node. No check is performed verifying that this is indeed the leader for all + listed partitions: use with caution. + + :return: Dictionary {topicPartition -> metadata}, where metadata is returned by the broker. + See DeleteRecordsResponse for possible fields. error_code for all partitions is + guaranteed to be zero, otherwise an exception is raised. + """ + 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 + # topics), we can send all of those in a single request. + # For that we store {leader -> {partitions for leader}}, and do 1 request per leader + if partition_leader_id is None: + leader2partitions = self._get_leader_for_partitions( + set(records_to_delete), timeout_ms + ) + else: + leader2partitions = {partition_leader_id: set(records_to_delete)} + + for leader, partitions in leader2partitions.items(): + topic2partitions = defaultdict(list) + for partition in partitions: + topic2partitions[partition.topic].append(partition) + + request = DeleteRecordsRequest[version]( + topics=[ + (topic, [(tp.partition, records_to_delete[tp]) for tp in partitions]) + for topic, partitions in topic2partitions.items() + ], + timeout_ms=timeout_ms + ) + future = self._send_request_to_node(leader, request) + self._wait_for_futures([future]) + + responses.append(future.value.to_object()) + + partition2result = {} + partition2error = {} + for response in responses: + for topic in response["topics"]: + for partition in topic["partitions"]: + tp = TopicPartition(topic["name"], partition["partition_index"]) + partition2result[tp] = partition + if partition["error_code"] != 0: + partition2error[tp] = partition["error_code"] + + if partition2error: + if len(partition2error) == 1: + key, error = next(iter(partition2error.items())) + raise Errors.for_code(error)( + "Error deleting records from topic %s partition %s" % (key.topic, key.partition) + ) + else: + raise Errors.BrokerResponseError( + "The following errors occured when trying to delete records: " + + ", ".join( + "%s(partition=%d): %s" % + (partition.topic, partition.partition, Errors.for_code(error).__name__) + for partition, error in partition2error.items() + ) + ) + + return partition2result # create delegation token protocol not yet implemented # Note: send the request to the least_loaded_node() diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index 058325cb1..63604e576 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -179,6 +179,38 @@ class DeleteTopicsRequest_v3(Request): ] +class DeleteRecordsResponse_v0(Response): + API_KEY = 21 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('topics', Array( + ('name', String('utf-8')), + ('partitions', Array( + ('partition_index', Int32), + ('low_watermark', Int64), + ('error_code', Int16))))), + ) + + +class DeleteRecordsRequest_v0(Request): + API_KEY = 21 + API_VERSION = 0 + RESPONSE_TYPE = DeleteRecordsResponse_v0 + SCHEMA = Schema( + ('topics', Array( + ('name', String('utf-8')), + ('partitions', Array( + ('partition_index', Int32), + ('offset', Int64))))), + ('timeout_ms', Int32) + ) + + +DeleteRecordsResponse = [DeleteRecordsResponse_v0] +DeleteRecordsRequest = [DeleteRecordsRequest_v0] + + class ListGroupsResponse_v0(Response): API_KEY = 16 API_VERSION = 0 diff --git a/test/conftest.py b/test/conftest.py index bf1fa6687..ddd491517 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -73,11 +73,11 @@ def kafka_consumer_factory(kafka_broker, topic, request): """Return a KafkaConsumer factory fixture""" _consumer = [None] - def factory(**kafka_consumer_params): + def factory(topics=(topic,), **kafka_consumer_params): params = {} if kafka_consumer_params is None else kafka_consumer_params.copy() params.setdefault('client_id', 'consumer_%s' % (request.node.name,)) params.setdefault('auto_offset_reset', 'earliest') - _consumer[0] = next(kafka_broker.get_consumers(cnt=1, topics=[topic], **params)) + _consumer[0] = next(kafka_broker.get_consumers(cnt=1, topics=list(topics), **params)) return _consumer[0] yield factory diff --git a/test/test_admin_integration.py b/test/test_admin_integration.py index 2f6b76598..83b6ccaf2 100644 --- a/test/test_admin_integration.py +++ b/test/test_admin_integration.py @@ -1,3 +1,4 @@ +from kafka.structs import TopicPartition import pytest from logging import info @@ -7,7 +8,9 @@ from kafka.admin import ( ACLFilter, ACLOperation, ACLPermissionType, ResourcePattern, ResourceType, ACL, ConfigResource, ConfigResourceType) -from kafka.errors import (NoError, GroupCoordinatorNotAvailableError, NonEmptyGroupError, GroupIdNotFoundError) +from kafka.errors import ( + BrokerResponseError, KafkaError, NoError, GroupCoordinatorNotAvailableError, NonEmptyGroupError, + GroupIdNotFoundError, OffsetOutOfRangeError, UnknownTopicOrPartitionError) @pytest.mark.skipif(env_kafka_version() < (0, 11), reason="ACL features require broker >=0.11") @@ -315,3 +318,71 @@ def test_delete_consumergroups_with_errors(kafka_admin_client, kafka_consumer_fa assert group1 not in consumergroups assert group2 in consumergroups assert group3 not in consumergroups + +@pytest.fixture(name="topic2") +def _topic2(kafka_broker, request): + """Same as `topic` fixture, but a different name if you need to topics.""" + topic_name = '%s_%s' % (request.node.name, random_string(10)) + kafka_broker.create_topics([topic_name]) + return topic_name + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason="Delete records requires broker >=0.11.0") +def test_delete_records(kafka_admin_client, kafka_consumer_factory, send_messages, topic, topic2): + t0p0 = TopicPartition(topic, 0) + t0p1 = TopicPartition(topic, 1) + t0p2 = TopicPartition(topic, 2) + t1p0 = TopicPartition(topic2, 0) + t1p1 = TopicPartition(topic2, 1) + t1p2 = TopicPartition(topic2, 2) + + partitions = (t0p0, t0p1, t0p2, t1p0, t1p1, t1p2) + + for p in partitions: + send_messages(range(0, 100), partition=p.partition, topic=p.topic) + + consumer1 = kafka_consumer_factory(group_id=None, topics=()) + consumer1.assign(partitions) + for _ in range(600): + next(consumer1) + + result = kafka_admin_client.delete_records({t0p0: -1, t0p1: 50, t1p0: 40, t1p2: 30}, timeout_ms=1000) + assert result[t0p0] == {"low_watermark": 100, "error_code": 0, "partition_index": t0p0.partition} + assert result[t0p1] == {"low_watermark": 50, "error_code": 0, "partition_index": t0p1.partition} + assert result[t1p0] == {"low_watermark": 40, "error_code": 0, "partition_index": t1p0.partition} + assert result[t1p2] == {"low_watermark": 30, "error_code": 0, "partition_index": t1p2.partition} + + consumer2 = kafka_consumer_factory(group_id=None, topics=()) + consumer2.assign(partitions) + all_messages = consumer2.poll(max_records=600, timeout_ms=2000) + assert sum(len(x) for x in all_messages.values()) == 600 - 100 - 50 - 40 - 30 + assert not consumer2.poll(max_records=1, timeout_ms=1000) # ensure there are no delayed messages + + assert not all_messages.get(t0p0, []) + assert [r.offset for r in all_messages[t0p1]] == list(range(50, 100)) + assert [r.offset for r in all_messages[t0p2]] == list(range(100)) + + assert [r.offset for r in all_messages[t1p0]] == list(range(40, 100)) + assert [r.offset for r in all_messages[t1p1]] == list(range(100)) + assert [r.offset for r in all_messages[t1p2]] == list(range(30, 100)) + + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason="Delete records requires broker >=0.11.0") +def test_delete_records_with_errors(kafka_admin_client, topic, send_messages): + sleep(1) # sometimes the topic is not created yet...? + p0 = TopicPartition(topic, 0) + p1 = TopicPartition(topic, 1) + p2 = TopicPartition(topic, 2) + # verify that topic has been created + send_messages(range(0, 1), partition=p2.partition, topic=p2.topic) + + with pytest.raises(UnknownTopicOrPartitionError): + kafka_admin_client.delete_records({TopicPartition(topic, 9999): -1}) + with pytest.raises(UnknownTopicOrPartitionError): + kafka_admin_client.delete_records({TopicPartition("doesntexist", 0): -1}) + with pytest.raises(OffsetOutOfRangeError): + kafka_admin_client.delete_records({p0: 1000}) + with pytest.raises(BrokerResponseError): + kafka_admin_client.delete_records({p0: 1000, p1: 1000}) + + + From a1b32995c954d81a6f66e8947cebdcfd28248a52 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 13:48:30 -0700 Subject: [PATCH 042/205] Admin - Implement perform leader election (#2536) Co-authored-by: Ygal Blum --- kafka/admin/client.py | 99 +++++++++++++++++++++++++++++++++++------ kafka/protocol/admin.py | 81 +++++++++++++++++++++++++++++++++ 2 files changed, 167 insertions(+), 13 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 6c2596cc8..171304da0 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -21,7 +21,7 @@ from kafka.protocol.admin import ( CreateTopicsRequest, DeleteTopicsRequest, DescribeConfigsRequest, AlterConfigsRequest, CreatePartitionsRequest, ListGroupsRequest, DescribeGroupsRequest, DescribeAclsRequest, CreateAclsRequest, DeleteAclsRequest, - DeleteGroupsRequest, DeleteRecordsRequest, DescribeLogDirsRequest) + DeleteGroupsRequest, DeleteRecordsRequest, DescribeLogDirsRequest, ElectLeadersRequest, ElectionType) from kafka.protocol.commit import OffsetFetchRequest from kafka.protocol.find_coordinator import FindCoordinatorRequest from kafka.protocol.metadata import MetadataRequest @@ -393,27 +393,55 @@ def _send_request_to_controller(self, request): # So this is a little brittle in that it assumes all responses have # one of these attributes and that they always unpack into # (topic, error_code) tuples. - topic_error_tuples = (response.topic_errors if hasattr(response, 'topic_errors') - else response.topic_error_codes) - # 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): + topic_error_tuples = getattr(response, 'topic_errors', getattr(response, 'topic_error_codes', None)) + if topic_error_tuples is not None: + success = self._parse_topic_request_response(topic_error_tuples, request, response, tries) + else: + # Leader Election request has a two layer error response (topic and partition) + success = self._parse_topic_partition_request_response(request, response, tries) + + if success: + return response + 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): + error_type = Errors.for_code(error_code) + if tries and error_type is NotControllerError: + # No need to inspect the rest of the errors for + # non-retriable errors because NotControllerError should + # either be thrown for all errors or no errors. + self._refresh_controller_id() + return False + elif error_type is not Errors.NoError: + raise error_type( + "Request '{}' failed with response '{}'." + .format(request, response)) + 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): error_type = Errors.for_code(error_code) if tries and error_type is NotControllerError: # No need to inspect the rest of the errors for # non-retriable errors because NotControllerError should # either be thrown for all errors or no errors. self._refresh_controller_id() - break - elif error_type is not Errors.NoError: + return False + elif error_type not in [Errors.NoError, Errors.ElectionNotNeeded]: raise error_type( "Request '{}' failed with response '{}'." .format(request, response)) - else: - return response - raise RuntimeError("This should never happen, please file a bug with full stacktrace if encountered") + return True @staticmethod def _convert_new_topic_request(new_topic): @@ -1651,6 +1679,51 @@ def _delete_consumer_groups_send_request(self, group_ids, group_coordinator_id): .format(version)) return self._send_request_to_node(group_coordinator_id, request) + @staticmethod + def _convert_topic_partitions(topic_partitions): + return [ + ( + topic, + partition_ids + ) + for topic, partition_ids in topic_partitions.items() + ] + + def _get_all_topic_partitions(self): + return [ + ( + topic, + [partition_info.partition for partition_info in self._client.cluster._partitions[topic].values()] + ) + for topic in self._client.cluster.topics() + ] + + def _get_topic_partitions(self, topic_partitions): + if topic_partitions is None: + return self._get_all_topic_partitions() + return self._convert_topic_partitions(topic_partitions) + + def perform_leader_election(self, election_type, topic_partitions=None, timeout_ms=None): + """Perform leader election on the topic partitions. + + :param election_type: Type of election to attempt. 0 for Perferred, 1 for Unclean + :param topic_partitions: A map of topic name strings to partition ids list. + By default, will run on all topic partitions + :param timeout_ms: Milliseconds to wait for the leader election process to complete + before the broker returns. + + :return: Appropriate version of ElectLeadersResponse class. + """ + version = self._client.api_version(ElectLeadersRequest, max_version=1) + timeout_ms = self._validate_timeout(timeout_ms) + request = ElectLeadersRequest[version]( + election_type=ElectionType(election_type), + topic_partitions=self._get_topic_partitions(topic_partitions), + timeout=timeout_ms, + ) + # 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. diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index 63604e576..4ac3c18c8 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -1,5 +1,12 @@ 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 kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String, Float64, CompactString, CompactArray, TaggedFields @@ -1031,3 +1038,77 @@ class ListPartitionReassignmentsRequest_v0(Request): ListPartitionReassignmentsRequest = [ListPartitionReassignmentsRequest_v0] ListPartitionReassignmentsResponse = [ListPartitionReassignmentsResponse_v0] + + +class ElectLeadersResponse_v0(Response): + API_KEY = 43 + API_VERSION = 1 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ('replication_election_results', Array( + ('topic', String('utf-8')), + ('partition_result', Array( + ('partition_id', Int32), + ('error_code', Int16), + ('error_message', String('utf-8')) + )) + )) + ) + + +class ElectLeadersRequest_v0(Request): + API_KEY = 43 + API_VERSION = 1 + RESPONSE_TYPE = ElectLeadersResponse_v0 + SCHEMA = Schema( + ('election_type', Int8), + ('topic_partitions', Array( + ('topic', String('utf-8')), + ('partition_ids', Array(Int32)) + )), + ('timeout', Int32), + ) + + +class ElectLeadersResponse_v1(Response): + API_KEY = 43 + API_VERSION = 1 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ('replication_election_results', Array( + ('topic', String('utf-8')), + ('partition_result', Array( + ('partition_id', Int32), + ('error_code', Int16), + ('error_message', String('utf-8')) + )) + )) + ) + + +class ElectLeadersRequest_v1(Request): + API_KEY = 43 + API_VERSION = 1 + RESPONSE_TYPE = ElectLeadersResponse_v1 + SCHEMA = Schema( + ('election_type', Int8), + ('topic_partitions', Array( + ('topic', String('utf-8')), + ('partition_ids', Array(Int32)) + )), + ('timeout', Int32), + ) + + +class ElectionType(IntEnum): + """ Leader election type + """ + + PREFERRED = 0, + UNCLEAN = 1 + + +ElectLeadersRequest = [ElectLeadersRequest_v0, ElectLeadersRequest_v1] +ElectLeadersResponse = [ElectLeadersResponse_v0, ElectLeadersResponse_v1] From de17b9f9b688257b6b674866e40061f99867005f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 14:49:16 -0700 Subject: [PATCH 043/205] Signal close to metrics expire_loop --- kafka/metrics/metrics.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/kafka/metrics/metrics.py b/kafka/metrics/metrics.py index 2c53488ff..41a37db58 100644 --- a/kafka/metrics/metrics.py +++ b/kafka/metrics/metrics.py @@ -55,10 +55,11 @@ def __init__(self, default_config=None, reporters=None, self._reporters = reporters or [] for reporter in self._reporters: reporter.init([]) + self._closed = False if enable_expiration: def expire_loop(): - while True: + while not self._closed: # delay 30 seconds time.sleep(30) self.ExpireSensorTask.run(self) @@ -259,3 +260,4 @@ def close(self): reporter.close() self._metrics.clear() + self._closed = True From 5360d79dcc07a058acf04dbf4957bcd8cadaf62f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 15:20:39 -0700 Subject: [PATCH 044/205] Limit producer close timeout to 1sec in __del__; use context managers to close in test_producer --- kafka/producer/kafka.py | 2 +- test/test_producer.py | 181 ++++++++++++++++++++++------------------ 2 files changed, 102 insertions(+), 81 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 47727dd9e..d5620075a 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -474,7 +474,7 @@ def _unregister_cleanup(self): self._cleanup = None def __del__(self): - self.close(null_logger=True) + self.close(timeout=1, null_logger=True) def close(self, timeout=None, null_logger=False): """Close this producer. diff --git a/test/test_producer.py b/test/test_producer.py index 7263130d1..ea2be89a0 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -1,3 +1,4 @@ +from contextlib import contextmanager import gc import platform import time @@ -22,6 +23,24 @@ def test_buffer_pool(): assert buf2.read() == b'' +@contextmanager +def producer_factory(**kwargs): + producer = KafkaProducer(**kwargs) + try: + yield producer + finally: + producer.close(timeout=0) + + +@contextmanager +def consumer_factory(**kwargs): + consumer = KafkaConsumer(**kwargs) + try: + yield consumer + finally: + consumer.close() + + @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") @pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd']) def test_end_to_end(kafka_broker, compression): @@ -35,37 +54,39 @@ def test_end_to_end(kafka_broker, compression): pytest.skip('zstd requires kafka 2.1.0 or newer') connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) - producer = KafkaProducer(bootstrap_servers=connect_str, - retries=5, - max_block_ms=30000, - compression_type=compression, - value_serializer=str.encode) - consumer = KafkaConsumer(bootstrap_servers=connect_str, - group_id=None, - consumer_timeout_ms=30000, - auto_offset_reset='earliest', - value_deserializer=bytes.decode) - - topic = random_string(5) - - messages = 100 - futures = [] - for i in range(messages): - futures.append(producer.send(topic, 'msg %d' % i)) - ret = [f.get(timeout=30) for f in futures] - assert len(ret) == messages - producer.close() - - consumer.subscribe([topic]) - msgs = set() - for i in range(messages): - try: - msgs.add(next(consumer).value) - except StopIteration: - break - - assert msgs == set(['msg %d' % (i,) for i in range(messages)]) - consumer.close() + producer_args = { + 'bootstrap_servers': connect_str, + 'retries': 5, + 'max_block_ms': 30000, + 'compression_type': compression, + 'value_serializer': str.encode, + } + consumer_args = { + 'bootstrap_servers': connect_str, + 'group_id': None, + 'consumer_timeout_ms': 30000, + 'auto_offset_reset': 'earliest', + 'value_deserializer': bytes.decode, + } + with producer_factory(**producer_args) as producer, consumer_factory(**consumer_args) as consumer: + topic = random_string(5) + + messages = 100 + futures = [] + for i in range(messages): + futures.append(producer.send(topic, 'msg %d' % i)) + ret = [f.get(timeout=30) for f in futures] + assert len(ret) == messages + + consumer.subscribe([topic]) + msgs = set() + for i in range(messages): + try: + msgs.add(next(consumer).value) + except StopIteration: + break + + assert msgs == set(['msg %d' % (i,) for i in range(messages)]) @pytest.mark.skipif(platform.python_implementation() != 'CPython', @@ -86,52 +107,52 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression): if compression == 'zstd' and env_kafka_version() < (2, 1, 0): pytest.skip('zstd requires 2.1.0 or more') connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) - producer = KafkaProducer(bootstrap_servers=connect_str, - retries=5, - max_block_ms=30000, - compression_type=compression) - magic = producer._max_usable_produce_magic() - - # record headers are supported in 0.11.0 - if env_kafka_version() < (0, 11, 0): - headers = None - else: - headers = [("Header Key", b"Header Value")] - - topic = random_string(5) - future = producer.send( - topic, - value=b"Simple value", key=b"Simple key", headers=headers, timestamp_ms=9999999, - partition=0) - record = future.get(timeout=5) - assert record is not None - assert record.topic == topic - assert record.partition == 0 - assert record.topic_partition == TopicPartition(topic, 0) - assert record.offset == 0 - if magic >= 1: - assert record.timestamp == 9999999 - else: - assert record.timestamp == -1 # NO_TIMESTAMP - - if magic >= 2: - assert record.checksum is None - elif magic == 1: - assert record.checksum == 1370034956 - else: - assert record.checksum == 3296137851 - - assert record.serialized_key_size == 10 - assert record.serialized_value_size == 12 - if headers: - assert record.serialized_header_size == 22 - - if magic == 0: - pytest.skip('generated timestamp case is skipped for broker 0.9 and below') - send_time = time.time() * 1000 - future = producer.send( - topic, - value=b"Simple value", key=b"Simple key", timestamp_ms=None, - partition=0) - record = future.get(timeout=5) - assert abs(record.timestamp - send_time) <= 1000 # Allow 1s deviation + with producer_factory(bootstrap_servers=connect_str, + retries=5, + max_block_ms=30000, + compression_type=compression) as producer: + magic = producer._max_usable_produce_magic() + + # record headers are supported in 0.11.0 + if env_kafka_version() < (0, 11, 0): + headers = None + else: + headers = [("Header Key", b"Header Value")] + + topic = random_string(5) + future = producer.send( + topic, + value=b"Simple value", key=b"Simple key", headers=headers, timestamp_ms=9999999, + partition=0) + record = future.get(timeout=5) + assert record is not None + assert record.topic == topic + assert record.partition == 0 + assert record.topic_partition == TopicPartition(topic, 0) + assert record.offset == 0 + if magic >= 1: + assert record.timestamp == 9999999 + else: + assert record.timestamp == -1 # NO_TIMESTAMP + + if magic >= 2: + assert record.checksum is None + elif magic == 1: + assert record.checksum == 1370034956 + else: + assert record.checksum == 3296137851 + + assert record.serialized_key_size == 10 + assert record.serialized_value_size == 12 + if headers: + assert record.serialized_header_size == 22 + + if magic == 0: + pytest.skip('generated timestamp case is skipped for broker 0.9 and below') + send_time = time.time() * 1000 + future = producer.send( + topic, + value=b"Simple value", key=b"Simple key", timestamp_ms=None, + partition=0) + record = future.get(timeout=5) + assert abs(record.timestamp - send_time) <= 1000 # Allow 1s deviation From ccd44ce3dd72c6ff9a2b4297c4398190e0ae4b41 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 15:25:50 -0700 Subject: [PATCH 045/205] Only refresh metadata if connection fails all dns records (#2532) --- kafka/client_async.py | 3 +-- kafka/conn.py | 4 ++++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 835864944..d792bb584 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -238,7 +238,6 @@ def __init__(self, **configs): self._api_versions = None self._connecting = set() self._sending = set() - self._refresh_on_disconnects = True # Not currently used, but data is collected internally self._last_bootstrap = 0 @@ -384,7 +383,7 @@ def _conn_state_change(self, node_id, sock, conn): elif self.cluster.is_bootstrap(node_id): self._bootstrap_fails += 1 - elif self._refresh_on_disconnects and not self._closed and not idle_disconnect: + elif conn.connect_failed() and not self._closed and not idle_disconnect: log.warning("Node %s connection failed -- refreshing metadata", node_id) self.cluster.request_update() diff --git a/kafka/conn.py b/kafka/conn.py index ded935838..b276d3d62 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -861,6 +861,10 @@ def disconnected(self): """Return True iff socket is closed""" return self.state is ConnectionStates.DISCONNECTED + def connect_failed(self): + """Return True iff connection attempt failed after attempting all dns records""" + return self.disconnected() and self.last_attempt >= 0 and len(self._gai) == 0 + def _reset_reconnect_backoff(self): self._failures = 0 self._reconnect_backoff = self.config['reconnect_backoff_ms'] / 1000.0 From d5c43484fc7800e6ed0d695ebc9dcacad529b4e3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 15:39:56 -0700 Subject: [PATCH 046/205] KafkaProducer: Flush pending records before close() (#2537) --- kafka/producer/kafka.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index d5620075a..b97983a78 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -506,6 +506,7 @@ def __getattr__(self, name): assert timeout >= 0 log.info("Closing the Kafka producer with %s secs timeout.", timeout) + self.flush(timeout) invoked_from_callback = bool(threading.current_thread() is self._sender) if timeout > 0: if invoked_from_callback: From c3c20cbee5fe1d938f81d10899068ee94065bcec Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 16:19:36 -0700 Subject: [PATCH 047/205] Fix OverflowError when connection_max_idle_ms is 0 or inf (#2538) --- kafka/client_async.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index d792bb584..7121ce7a7 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -30,7 +30,7 @@ from kafka.util import Dict, WeakMethod # 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 +from kafka.vendor import socketpair # noqa: F401 from kafka.version import __version__ if six.PY2: @@ -1184,7 +1184,7 @@ def is_expired(self, conn_id): def next_check_ms(self): now = time.time() - if not self.lru_connections: + if not self.lru_connections or self.next_idle_close_check_time == float('inf'): return float('inf') elif self.next_idle_close_check_time <= now: return 0 From febfdacb289776f27010ac90539947c2edd67492 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 16:24:04 -0700 Subject: [PATCH 048/205] Delay group coordinator until after bootstrap (#2539) --- kafka/coordinator/base.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index dd40bf5d4..0edd50616 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -252,12 +252,16 @@ def ensure_coordinator_ready(self, timeout_ms=None): # so we will just pick a node at random and treat # it as the "coordinator" if self.config['api_version'] < (0, 8, 2): - self.coordinator_id = self._client.least_loaded_node() - if self.coordinator_id is not None: + maybe_coordinator_id = self._client.least_loaded_node() + if maybe_coordinator_id is None or self._client.cluster.is_bootstrap(maybe_coordinator_id): + future = Future().failure(Errors.NoBrokersAvailable()) + else: + self.coordinator_id = maybe_coordinator_id self._client.maybe_connect(self.coordinator_id) - continue + continue + else: + future = self.lookup_coordinator() - future = self.lookup_coordinator() self._client.poll(future=future, timeout_ms=inner_timeout_ms()) if not future.is_done: @@ -677,7 +681,7 @@ def _send_group_coordinator_request(self): Future: resolves to the node id of the coordinator """ node_id = self._client.least_loaded_node() - if node_id is None: + if node_id is None or self._client.cluster.is_bootstrap(node_id): return Future().failure(Errors.NoBrokersAvailable()) elif not self._client.ready(node_id, metadata_priority=False): From 0117ace5e72946122a8640f35b77a35331e94086 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 16:24:24 -0700 Subject: [PATCH 049/205] Log warning when attempting to list offsets for unknown topic/partition (#2540) --- kafka/consumer/fetcher.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 2179e19fc..e5ae64c91 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -538,9 +538,13 @@ def _send_list_offsets_requests(self, timestamps): for partition, timestamp in six.iteritems(timestamps): node_id = self._client.cluster.leader_for_partition(partition) if node_id is None: + if partition.topic not in self._client.cluster.topics(): + log.warning("Could not lookup offsets for partition %s since no metadata is available for topic. " + "Wait for metadata refresh and try again", partition) + else: + log.warning("Could not lookup offsets for partition %s since no metadata is available for it. " + "Wait for metadata refresh and try again", partition) self._client.add_topic(partition.topic) - log.debug("Partition %s is unknown for fetching offset," - " wait for metadata refresh", partition) return Future().failure(Errors.StaleMetadata(partition)) elif node_id == -1: log.debug("Leader for partition %s unavailable for fetching " From ed4ecee8f369b2cf82a45aeb3b5b1d0d3c95263e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 16:38:18 -0700 Subject: [PATCH 050/205] Bump default python to 3.13 in CI tests (#2541) --- .github/workflows/python-package.yml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index ec4b8ec1e..96df685f4 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -34,14 +34,11 @@ jobs: - "3.5.2" - "3.9.0" python: - - "3.12" + - "3.13" include: #- python: "pypy3.9" # kafka: "2.6.0" # experimental: true - #- python: "~3.13.0-0" - # kafka: "2.6.0" - # experimental: true - python: "3.8" kafka: "3.9.0" - python: "3.9" @@ -50,6 +47,8 @@ jobs: kafka: "3.9.0" - python: "3.11" kafka: "3.9.0" + - python: "3.12" + kafka: "3.9.0" steps: - uses: actions/checkout@v4 From b60a2664a0f1098e73c5142920e9d4169a05b6e6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 17:36:09 -0700 Subject: [PATCH 051/205] Raise immediate error on producer.send after close (#2542) --- kafka/producer/kafka.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index b97983a78..b8ace0fc1 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -594,6 +594,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest KafkaTimeoutError: if unable to fetch topic metadata, or unable to obtain memory buffer prior to configured max_block_ms """ + assert not self._closed, 'KafkaProducer already closed!' assert value is not None or self.config['api_version'] >= (0, 8, 1), ( 'Null messages require kafka >= 0.8.1') assert not (value is None and key is None), 'Need at least one: key or value' From 1bd6573b30ddf896b6110be227906c67d1ee5205 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 18:56:34 -0700 Subject: [PATCH 052/205] Remove legacy/v1 consumer message iterator (#2543) --- kafka/consumer/group.py | 85 ++--------------------------------------- 1 file changed, 4 insertions(+), 81 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 751403f52..071371b98 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -327,7 +327,6 @@ class KafkaConsumer(six.Iterator): 'sasl_kerberos_domain_name': None, 'sasl_oauth_token_provider': None, 'socks5_proxy': None, - 'legacy_iterator': False, # enable to revert to < 1.4.7 iterator 'kafka_client': KafkaClient, } DEFAULT_SESSION_TIMEOUT_MS_0_9 = 30000 @@ -845,8 +844,7 @@ def seek(self, partition, offset): assert partition in self._subscription.assigned_partitions(), 'Unassigned partition' log.debug("Seeking to offset %s for partition %s", offset, partition) self._subscription.assignment[partition].seek(offset) - if not self.config['legacy_iterator']: - self._iterator = None + self._iterator = None def seek_to_beginning(self, *partitions): """Seek to the oldest available offset for partitions. @@ -871,8 +869,7 @@ def seek_to_beginning(self, *partitions): for tp in partitions: log.debug("Seeking to beginning of partition %s", tp) self._subscription.need_offset_reset(tp, OffsetResetStrategy.EARLIEST) - if not self.config['legacy_iterator']: - self._iterator = None + self._iterator = None def seek_to_end(self, *partitions): """Seek to the most recent available offset for partitions. @@ -897,8 +894,7 @@ def seek_to_end(self, *partitions): for tp in partitions: log.debug("Seeking to end of partition %s", tp) self._subscription.need_offset_reset(tp, OffsetResetStrategy.LATEST) - if not self.config['legacy_iterator']: - self._iterator = None + self._iterator = None def subscribe(self, topics=(), pattern=None, listener=None): """Subscribe to a list of topics, or a topic regex pattern. @@ -974,8 +970,7 @@ def unsubscribe(self): self._client.cluster.need_all_topic_metadata = False self._client.set_topics([]) log.debug("Unsubscribed all topics or patterns and assigned partitions") - if not self.config['legacy_iterator']: - self._iterator = None + self._iterator = None def metrics(self, raw=False): """Get metrics on consumer performance. @@ -1157,73 +1152,12 @@ def _message_generator_v2(self): self._subscription.assignment[tp].position = OffsetAndMetadata(record.offset + 1, '', -1) yield record - def _message_generator(self): - assert self.assignment() or self.subscription() is not None, 'No topic subscription or manual partition assignment' - - def inner_poll_ms(): - return max(0, min((1000 * (self._consumer_timeout - time.time())), self.config['retry_backoff_ms'])) - - while time.time() < self._consumer_timeout: - - if not self._coordinator.poll(timeout_ms=inner_poll_ms()): - continue - - # Fetch offsets for any subscribed partitions that we arent tracking yet - if not self._subscription.has_all_fetch_positions(): - partitions = self._subscription.missing_fetch_positions() - self._update_fetch_positions(partitions) - - self._client.poll(timeout_ms=inner_poll_ms()) - - # after the long poll, we should check whether the group needs to rebalance - # prior to returning data so that the group can stabilize faster - if self._coordinator.need_rejoin(): - continue - - # We need to make sure we at least keep up with scheduled tasks, - # like heartbeats, auto-commits, and metadata refreshes - timeout_at = self._next_timeout() - - # Short-circuit the fetch iterator if we are already timed out - # to avoid any unintentional interaction with fetcher setup - if time.time() > timeout_at: - continue - - for msg in self._fetcher: - yield msg - if time.time() > timeout_at: - log.debug("internal iterator timeout - breaking for poll") - break - self._client.poll(timeout_ms=0) - - # An else block on a for loop only executes if there was no break - # so this should only be called on a StopIteration from the fetcher - # We assume that it is safe to init_fetches when fetcher is done - # i.e., there are no more records stored internally - else: - self._fetcher.send_fetches() - - def _next_timeout(self): - timeout = min(self._consumer_timeout, - self._client.cluster.ttl() / 1000.0 + time.time(), - self._coordinator.time_to_next_poll() + time.time()) - return timeout - def __iter__(self): # pylint: disable=non-iterator-returned return self def __next__(self): if self._closed: raise StopIteration('KafkaConsumer closed') - # Now that the heartbeat thread runs in the background - # there should be no reason to maintain a separate iterator - # but we'll keep it available for a few releases just in case - if self.config['legacy_iterator']: - return self.next_v1() - else: - return self.next_v2() - - def next_v2(self): self._set_consumer_timeout() while time.time() < self._consumer_timeout: if not self._iterator: @@ -1234,17 +1168,6 @@ def next_v2(self): self._iterator = None raise StopIteration() - def next_v1(self): - if not self._iterator: - self._iterator = self._message_generator() - - self._set_consumer_timeout() - try: - return next(self._iterator) - except StopIteration: - self._iterator = None - raise - def _set_consumer_timeout(self): # consumer_timeout_ms can be used to stop iteration early if self.config['consumer_timeout_ms'] >= 0: From 3493380c80a75a843073b8b674c85bc2f8220b09 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 14 Mar 2025 19:17:46 -0700 Subject: [PATCH 053/205] Add optional timeout_ms kwarg to remaining consumer/coordinator methods (#2544) --- kafka/consumer/fetcher.py | 26 +++++++---- kafka/consumer/group.py | 84 +++++++++++++++++++---------------- kafka/coordinator/base.py | 5 ++- kafka/coordinator/consumer.py | 45 ++++++++++++------- kafka/util.py | 7 ++- test/test_fetcher.py | 4 +- 6 files changed, 101 insertions(+), 70 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index e5ae64c91..7527a1f39 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -135,17 +135,21 @@ def send_fetches(self): self._clean_done_fetch_futures() return futures - def reset_offsets_if_needed(self, partitions): + def reset_offsets_if_needed(self, partitions, timeout_ms=None): """Lookup and set offsets for any partitions which are awaiting an explicit reset. Arguments: partitions (set of TopicPartitions): the partitions to reset + + Raises: + KafkaTimeoutError if timeout_ms provided """ + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout resetting offsets') for tp in partitions: # TODO: If there are several offsets to reset, we could submit offset requests in parallel if self._subscriptions.is_assigned(tp) and self._subscriptions.is_offset_reset_needed(tp): - self._reset_offset(tp) + self._reset_offset(tp, timeout_ms=inner_timeout_ms()) def _clean_done_fetch_futures(self): while True: @@ -160,7 +164,7 @@ def in_flight_fetches(self): self._clean_done_fetch_futures() return bool(self._fetch_futures) - def update_fetch_positions(self, partitions): + def update_fetch_positions(self, partitions, timeout_ms=None): """Update the fetch positions for the provided partitions. Arguments: @@ -169,7 +173,9 @@ def update_fetch_positions(self, partitions): Raises: NoOffsetForPartitionError: if no offset is stored for a given partition and no reset policy is available + KafkaTimeoutError if timeout_ms provided. """ + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout updating fetch positions') # reset the fetch position to the committed position for tp in partitions: if not self._subscriptions.is_assigned(tp): @@ -182,12 +188,12 @@ def update_fetch_positions(self, partitions): continue if self._subscriptions.is_offset_reset_needed(tp): - self._reset_offset(tp) + self._reset_offset(tp, timeout_ms=inner_timeout_ms()) elif self._subscriptions.assignment[tp].committed is None: # there's no committed position, so we need to reset with the # default strategy self._subscriptions.need_offset_reset(tp) - self._reset_offset(tp) + self._reset_offset(tp, timeout_ms=inner_timeout_ms()) else: committed = self._subscriptions.assignment[tp].committed.offset log.debug("Resetting offset for partition %s to the committed" @@ -216,7 +222,7 @@ def beginning_or_end_offset(self, partitions, timestamp, timeout_ms): offsets[tp] = offsets[tp].offset return offsets - def _reset_offset(self, partition): + def _reset_offset(self, partition, timeout_ms=None): """Reset offsets for the given partition using the offset reset strategy. Arguments: @@ -224,6 +230,7 @@ def _reset_offset(self, partition): Raises: NoOffsetForPartitionError: if no offset reset strategy is defined + KafkaTimeoutError if timeout_ms provided """ timestamp = self._subscriptions.assignment[partition].reset_strategy if timestamp is OffsetResetStrategy.EARLIEST: @@ -235,7 +242,7 @@ def _reset_offset(self, partition): log.debug("Resetting offset for partition %s to %s offset.", partition, strategy) - offsets = self._retrieve_offsets({partition: timestamp}) + offsets = self._retrieve_offsets({partition: timestamp}, timeout_ms=timeout_ms) if partition in offsets: offset = offsets[partition].offset @@ -263,11 +270,14 @@ def _retrieve_offsets(self, timestamps, timeout_ms=None): retrieved offset, timestamp, and leader_epoch. If offset does not exist for the provided timestamp, that partition will be missing from this mapping. + + Raises: + KafkaTimeoutError if timeout_ms provided """ if not timestamps: return {} - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to find coordinator') + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout fetching offsets') timestamps = copy.copy(timestamps) while True: if not timestamps: diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 071371b98..3fccf4755 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -5,7 +5,7 @@ import socket import time -from kafka.errors import KafkaConfigurationError, UnsupportedVersionError +from kafka.errors import KafkaConfigurationError, KafkaTimeoutError, UnsupportedVersionError from kafka.vendor import six @@ -18,6 +18,7 @@ from kafka.metrics import MetricConfig, Metrics from kafka.protocol.list_offsets import OffsetResetStrategy from kafka.structs import OffsetAndMetadata, TopicPartition +from kafka.util import timeout_ms_fn from kafka.version import __version__ log = logging.getLogger(__name__) @@ -521,7 +522,7 @@ def commit_async(self, offsets=None, callback=None): offsets, callback=callback) return future - def commit(self, offsets=None): + def commit(self, offsets=None, timeout_ms=None): """Commit offsets to kafka, blocking until success or error. This commits offsets only to Kafka. The offsets committed using this API @@ -545,9 +546,9 @@ def commit(self, offsets=None): assert self.config['group_id'] is not None, 'Requires group_id' if offsets is None: offsets = self._subscription.all_consumed_offsets() - self._coordinator.commit_offsets_sync(offsets) + self._coordinator.commit_offsets_sync(offsets, timeout_ms=timeout_ms) - def committed(self, partition, metadata=False): + def committed(self, partition, metadata=False, timeout_ms=None): """Get the last committed offset for the given partition. This offset will be used as the position for the consumer @@ -564,6 +565,9 @@ def committed(self, partition, metadata=False): Returns: The last committed offset (int or OffsetAndMetadata), or None if there was no prior commit. + + Raises: + KafkaTimeoutError if timeout_ms provided """ assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1' assert self.config['group_id'] is not None, 'Requires group_id' @@ -572,10 +576,10 @@ def committed(self, partition, metadata=False): if self._subscription.is_assigned(partition): committed = self._subscription.assignment[partition].committed if committed is None: - self._coordinator.refresh_committed_offsets_if_needed() + self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=timeout_ms) committed = self._subscription.assignment[partition].committed else: - commit_map = self._coordinator.fetch_committed_offsets([partition]) + commit_map = self._coordinator.fetch_committed_offsets([partition], timeout_ms=timeout_ms) if partition in commit_map: committed = commit_map[partition] else: @@ -670,17 +674,13 @@ def poll(self, timeout_ms=0, max_records=None, update_offsets=True): assert not self._closed, 'KafkaConsumer is closed' # Poll for new data until the timeout expires - start = time.time() - remaining = timeout_ms + inner_timeout_ms = timeout_ms_fn(timeout_ms, None) while not self._closed: - records = self._poll_once(remaining, max_records, update_offsets=update_offsets) + records = self._poll_once(inner_timeout_ms(), max_records, update_offsets=update_offsets) if records: return records - elapsed_ms = (time.time() - start) * 1000 - remaining = timeout_ms - elapsed_ms - - if remaining <= 0: + if inner_timeout_ms() <= 0: break return {} @@ -695,14 +695,14 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): Returns: dict: Map of topic to list of records (may be empty). """ - begin = time.time() - if not self._coordinator.poll(timeout_ms=timeout_ms): + inner_timeout_ms = timeout_ms_fn(timeout_ms, None) + if not self._coordinator.poll(timeout_ms=inner_timeout_ms()): return {} # Fetch positions if we have partitions we're subscribed to that we # don't know the offset for if not self._subscription.has_all_fetch_positions(): - self._update_fetch_positions(self._subscription.missing_fetch_positions()) + self._update_fetch_positions(self._subscription.missing_fetch_positions(), timeout_ms=inner_timeout_ms()) # If data is available already, e.g. from a previous network client # poll() call to commit, then just return it immediately @@ -723,9 +723,7 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): if len(futures): self._client.poll(timeout_ms=0) - timeout_ms -= (time.time() - begin) * 1000 - timeout_ms = max(0, min(timeout_ms, self._coordinator.time_to_next_poll() * 1000)) - self._client.poll(timeout_ms=timeout_ms) + self._client.poll(timeout_ms=inner_timeout_ms(self._coordinator.time_to_next_poll() * 1000)) # after the long poll, we should check whether the group needs to rebalance # prior to returning data so that the group can stabilize faster if self._coordinator.need_rejoin(): @@ -734,7 +732,7 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): records, _ = self._fetcher.fetched_records(max_records, update_offsets=update_offsets) return records - def position(self, partition): + def position(self, partition, timeout_ms=None): """Get the offset of the next record that will be fetched Arguments: @@ -748,7 +746,7 @@ def position(self, partition): assert self._subscription.is_assigned(partition), 'Partition is not assigned' position = self._subscription.assignment[partition].position if position is None: - self._update_fetch_positions([partition]) + self._update_fetch_positions([partition], timeout_ms=timeout_ms) position = self._subscription.assignment[partition].position return position.offset if position else None @@ -1103,7 +1101,7 @@ def _use_consumer_group(self): return False return True - def _update_fetch_positions(self, partitions): + def _update_fetch_positions(self, partitions, timeout_ms=None): """Set the fetch position to the committed position (if there is one) or reset it using the offset reset policy the user has configured. @@ -1111,27 +1109,35 @@ def _update_fetch_positions(self, partitions): partitions (List[TopicPartition]): The partitions that need updating fetch positions. + Returns True if fetch positions updated, False if timeout + Raises: NoOffsetForPartitionError: If no offset is stored for a given partition and no offset reset policy is defined. """ - # Lookup any positions for partitions which are awaiting reset (which may be the - # case if the user called :meth:`seek_to_beginning` or :meth:`seek_to_end`. We do - # this check first to avoid an unnecessary lookup of committed offsets (which - # typically occurs when the user is manually assigning partitions and managing - # their own offsets). - self._fetcher.reset_offsets_if_needed(partitions) - - if not self._subscription.has_all_fetch_positions(): - # if we still don't have offsets for all partitions, then we should either seek - # to the last committed position or reset using the auto reset policy - if (self.config['api_version'] >= (0, 8, 1) and - self.config['group_id'] is not None): - # first refresh commits for all assigned partitions - self._coordinator.refresh_committed_offsets_if_needed() - - # Then, do any offset lookups in case some positions are not known - self._fetcher.update_fetch_positions(partitions) + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout updating fetch positions') + try: + # Lookup any positions for partitions which are awaiting reset (which may be the + # case if the user called :meth:`seek_to_beginning` or :meth:`seek_to_end`. We do + # this check first to avoid an unnecessary lookup of committed offsets (which + # typically occurs when the user is manually assigning partitions and managing + # their own offsets). + self._fetcher.reset_offsets_if_needed(partitions, timeout_ms=inner_timeout_ms()) + + if not self._subscription.has_all_fetch_positions(): + # if we still don't have offsets for all partitions, then we should either seek + # to the last committed position or reset using the auto reset policy + if (self.config['api_version'] >= (0, 8, 1) and + self.config['group_id'] is not None): + # first refresh commits for all assigned partitions + self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=inner_timeout_ms()) + + # Then, do any offset lookups in case some positions are not known + self._fetcher.update_fetch_positions(partitions, timeout_ms=inner_timeout_ms()) + return True + + except KafkaTimeoutError: + return False def _message_generator_v2(self): timeout_ms = 1000 * max(0, self._consumer_timeout - time.time()) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 0edd50616..c5e56c538 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -163,7 +163,7 @@ def group_protocols(self): pass @abc.abstractmethod - def _on_join_prepare(self, generation, member_id): + def _on_join_prepare(self, generation, member_id, timeout_ms=None): """Invoked prior to each group join or rejoin. This is typically used to perform any cleanup from the previous @@ -415,7 +415,8 @@ def join_group(self, timeout_ms=None): # while another rebalance is still in progress. if not self.rejoining: self._on_join_prepare(self._generation.generation_id, - self._generation.member_id) + self._generation.member_id, + timeout_ms=inner_timeout_ms()) self.rejoining = True # fence off the heartbeat thread explicitly so that it cannot diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 92c84024c..5b4752bf8 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -349,9 +349,9 @@ def _perform_assignment(self, leader_id, assignment_strategy, members): group_assignment[member_id] = assignment return group_assignment - def _on_join_prepare(self, generation, member_id): + def _on_join_prepare(self, generation, member_id, timeout_ms=None): # commit offsets prior to rebalance if auto-commit enabled - self._maybe_auto_commit_offsets_sync() + self._maybe_auto_commit_offsets_sync(timeout_ms=timeout_ms) # execute the user's callback before rebalance log.info("Revoking previously assigned partitions %s for group %s", @@ -392,17 +392,17 @@ def need_rejoin(self): return super(ConsumerCoordinator, self).need_rejoin() - def refresh_committed_offsets_if_needed(self): + def refresh_committed_offsets_if_needed(self, timeout_ms=None): """Fetch committed offsets for assigned partitions.""" if self._subscription.needs_fetch_committed_offsets: - offsets = self.fetch_committed_offsets(self._subscription.assigned_partitions()) + offsets = self.fetch_committed_offsets(self._subscription.assigned_partitions(), timeout_ms=timeout_ms) for partition, offset in six.iteritems(offsets): # verify assignment is still active if self._subscription.is_assigned(partition): self._subscription.assignment[partition].committed = offset self._subscription.needs_fetch_committed_offsets = False - def fetch_committed_offsets(self, partitions): + def fetch_committed_offsets(self, partitions, timeout_ms=None): """Fetch the current committed offsets for specified partitions Arguments: @@ -410,16 +410,23 @@ def fetch_committed_offsets(self, partitions): Returns: dict: {TopicPartition: OffsetAndMetadata} + + Raises: + KafkaTimeoutError if timeout_ms provided """ if not partitions: return {} + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.fetch_committed_offsets') while True: - self.ensure_coordinator_ready() + self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) # contact coordinator to fetch committed offsets future = self._send_offset_fetch_request(partitions) - self._client.poll(future=future) + self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + + if not future.is_done: + raise Errors.KafkaTimeoutError() if future.succeeded(): return future.value @@ -427,9 +434,9 @@ def fetch_committed_offsets(self, partitions): if not future.retriable(): raise future.exception # pylint: disable-msg=raising-bad-type - time.sleep(self.config['retry_backoff_ms'] / 1000) + time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) - def close(self, autocommit=True): + def close(self, autocommit=True, timeout_ms=None): """Close the coordinator, leave the current group, and reset local generation / member_id. @@ -440,7 +447,7 @@ def close(self, autocommit=True): """ try: if autocommit: - self._maybe_auto_commit_offsets_sync() + self._maybe_auto_commit_offsets_sync(timeout_ms=timeout_ms) finally: super(ConsumerCoordinator, self).close() @@ -498,7 +505,7 @@ def _do_commit_offsets_async(self, offsets, callback=None): future.add_both(lambda res: self.completed_offset_commits.appendleft((callback, offsets, res))) return future - def commit_offsets_sync(self, offsets): + def commit_offsets_sync(self, offsets, timeout_ms=None): """Commit specific offsets synchronously. This method will retry until the commit completes successfully or an @@ -517,11 +524,15 @@ def commit_offsets_sync(self, offsets): if not offsets: return + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.poll') while True: - self.ensure_coordinator_ready() + self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) future = self._send_offset_commit_request(offsets) - self._client.poll(future=future) + self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + + if not future.is_done: + raise Errors.KafkaTimeoutError() if future.succeeded(): return future.value @@ -529,12 +540,12 @@ def commit_offsets_sync(self, offsets): if not future.retriable(): raise future.exception # pylint: disable-msg=raising-bad-type - time.sleep(self.config['retry_backoff_ms'] / 1000) + time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) - def _maybe_auto_commit_offsets_sync(self): + def _maybe_auto_commit_offsets_sync(self, timeout_ms=None): if self.config['enable_auto_commit']: try: - self.commit_offsets_sync(self._subscription.all_consumed_offsets()) + self.commit_offsets_sync(self._subscription.all_consumed_offsets(), timeout_ms=timeout_ms) # The three main group membership errors are known and should not # require a stacktrace -- just a warning @@ -814,7 +825,7 @@ def _handle_offset_fetch_response(self, future, response): leader_epoch, metadata, error_code = partition_data[2:] else: metadata, error_code = partition_data[2:] - leader_epoch = -1 + leader_epoch = -1 # noqa: F841 tp = TopicPartition(topic, partition) error_type = Errors.for_code(error_code) if error_type is not Errors.NoError: diff --git a/kafka/util.py b/kafka/util.py index 6d061193a..d067a063d 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -21,7 +21,7 @@ def crc32(data): crc -= TO_SIGNED return crc else: - from binascii import crc32 + from binascii import crc32 # noqa: F401 def timeout_ms_fn(timeout_ms, error_message): @@ -32,7 +32,10 @@ def inner_timeout_ms(fallback=None): return fallback elapsed = (time.time() - begin) * 1000 if elapsed >= timeout_ms: - raise KafkaTimeoutError(error_message) + if error_message is not None: + raise KafkaTimeoutError(error_message) + else: + return 0 ret = max(0, timeout_ms - elapsed) if fallback is not None: return min(ret, fallback) diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 479f6e22b..3bf334e06 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -130,10 +130,10 @@ def test_update_fetch_positions(fetcher, topic, mocker): fetcher._subscriptions.need_offset_reset(partition) fetcher._subscriptions.assignment[partition].awaiting_reset = False fetcher.update_fetch_positions([partition]) - fetcher._reset_offset.assert_called_with(partition) + fetcher._reset_offset.assert_called_with(partition, timeout_ms=None) assert fetcher._subscriptions.assignment[partition].awaiting_reset is True fetcher.update_fetch_positions([partition]) - fetcher._reset_offset.assert_called_with(partition) + fetcher._reset_offset.assert_called_with(partition, timeout_ms=None) # partition needs reset, has committed offset fetcher._reset_offset.reset_mock() From a96bc9cc99bb88cfad9120fa5173fcdad6868688 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 15 Mar 2025 18:10:56 -0700 Subject: [PATCH 054/205] Call default_offset_commit_callback after _maybe_auto_commit_offsets_async (#2546) --- kafka/coordinator/consumer.py | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 5b4752bf8..4c1b38644 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -54,7 +54,7 @@ def __init__(self, client, subscription, metrics, **configs): auto_commit_interval_ms (int): milliseconds between automatic offset commits, if enable_auto_commit is True. Default: 5000. default_offset_commit_callback (callable): called as - callback(offsets, exception) response will be either an Exception + callback(offsets, response) response will be either an Exception or None. This callback can be used to trigger custom actions when a commit request completes. assignors (list): List of objects to use to distribute partition @@ -453,8 +453,8 @@ def close(self, autocommit=True, timeout_ms=None): def _invoke_completed_offset_commit_callbacks(self): while self.completed_offset_commits: - callback, offsets, exception = self.completed_offset_commits.popleft() - callback(offsets, exception) + callback, offsets, res_or_exc = self.completed_offset_commits.popleft() + callback(offsets, res_or_exc) def commit_offsets_async(self, offsets, callback=None): """Commit specific offsets asynchronously. @@ -859,20 +859,19 @@ def _handle_offset_fetch_response(self, future, response): " %s", self.group_id, tp) future.success(offsets) - def _default_offset_commit_callback(self, offsets, exception): - if exception is not None: - log.error("Offset commit failed: %s", exception) - - def _commit_offsets_async_on_complete(self, offsets, exception): - if exception is not None: + def _default_offset_commit_callback(self, offsets, res_or_exc): + if isinstance(res_or_exc, Exception): log.warning("Auto offset commit failed for group %s: %s", - self.group_id, exception) - if getattr(exception, 'retriable', False): - self.next_auto_commit_deadline = min(time.time() + self.config['retry_backoff_ms'] / 1000, self.next_auto_commit_deadline) + self.group_id, res_or_exc) else: log.debug("Completed autocommit of offsets %s for group %s", offsets, self.group_id) + def _commit_offsets_async_on_complete(self, offsets, res_or_exc): + if isinstance(res_or_exc, Exception) and getattr(res_or_exc, 'retriable', False): + self.next_auto_commit_deadline = min(time.time() + self.config['retry_backoff_ms'] / 1000, self.next_auto_commit_deadline) + self.config['default_offset_commit_callback'](offsets, res_or_exc) + def _maybe_auto_commit_offsets_async(self): if self.config['enable_auto_commit']: if self.coordinator_unknown(): From 4b89741e912cd1f991c51ceda88a45672fd5902e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 15 Mar 2025 18:44:04 -0700 Subject: [PATCH 055/205] Release 2.1.0 --- CHANGES.md | 87 +++++++++++++++++++++++++++++++++++++ docs/changelog.rst | 98 ++++++++++++++++++++++++++++++++++++++++++ docs/compatibility.rst | 4 +- kafka/version.py | 2 +- 4 files changed, 188 insertions(+), 3 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index ee28a84e7..8a0be7283 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,90 @@ +# 2.1.0 (Mar 14, 2025) + +Support Kafka Broker 2.1 API Baseline +* Add baseline leader_epoch support for ListOffsets v4 / FetchRequest v10 (#2511) +* Support OffsetFetch v5 / OffsetCommit v6 (2.1 baseline) (#2505) +* Support 2.1 baseline consumer group apis (#2503) +* Support FindCoordinatorRequest v2 in consumer and admin client (#2502) +* Support ListOffsets v3 in consumer (#2501) +* Support Fetch Request/Response v6 in consumer (#2500) +* Add support for Metadata Request/Response v7 (#2497) +* Implement Incremental Fetch Sessions / KIP-227 (#2508) +* Implement client-side connection throttling / KIP-219 (#2510) +* Add KafkaClient.api_version(operation) for best available from api_versions (#2495) + +Consumer +* Timeout coordinator poll / ensure_coordinator_ready / ensure_active_group (#2526) +* Add optional timeout_ms kwarg to remaining consumer/coordinator methods (#2544) +* Check for coordinator.poll failure in KafkaConsumer +* Only mark coordinator dead if connection_delay > 0 (#2530) +* Delay group coordinator until after bootstrap (#2539) +* KAFKA-4160: Ensure rebalance listener not called with coordinator lock (#1438) +* Call default_offset_commit_callback after `_maybe_auto_commit_offsets_async` (#2546) +* Remove legacy/v1 consumer message iterator (#2543) +* Log warning when attempting to list offsets for unknown topic/partition (#2540) +* Add heartbeat thread id to debug logs on start +* Add inner_timeout_ms handler to fetcher; add fallback (#2529) + +Producer +* KafkaProducer: Flush pending records before close() (#2537) +* Raise immediate error on producer.send after close (#2542) +* Limit producer close timeout to 1sec in __del__; use context managers to close in test_producer +* Use NullLogger in producer atexit cleanup +* Attempt to fix metadata race condition when partitioning in producer.send (#2523) +* Remove unused partial KIP-467 implementation (ProduceResponse batch error details) (#2524) + +AdminClient +* Implement perform leader election (#2536) +* Support delete_records (#2535) + +Networking +* Call ApiVersionsRequest during connection, prior to Sasl Handshake (#2493) +* Fake api_versions for old brokers, rename to ApiVersionsRequest, and handle error decoding (#2494) +* Debug log when skipping api_versions request with pre-configured api_version +* Only refresh metadata if connection fails all dns records (#2532) +* Support connections through SOCKS5 proxies (#2531) +* Fix OverflowError when connection_max_idle_ms is 0 or inf (#2538) +* socket.setblocking for eventlet/gevent compatibility +* Support custom per-request timeouts (#2498) +* Include request_timeout_ms in request debug log +* Support client.poll with future and timeout_ms +* mask unused afi var +* Debug log if check_version connection attempt fails + +SASL Modules +* Refactor Sasl authentication with SaslMechanism abstract base class; support SaslAuthenticate (#2515) +* Add SSPI (Kerberos for Windows) authentication mechanism (#2521) +* Support AWS_MSK_IAM authentication (#2519) +* Cleanup sasl mechanism configuration checks; fix gssapi bugs; add sasl_kerberos_name config (#2520) +* Move kafka.oauth.AbstractTokenProvider -> kafka.sasl.oauth.AbstractTokenProvider (#2525) + +Testing +* Bump default python to 3.13 in CI tests (#2541) +* Update pytest log_format: use logger instead of filename; add thread id +* Improve test_consumer_group::test_group logging before group stabilized (#2534) +* Limit test duration to 5mins w/ pytest-timeout +* Fix external kafka/zk fixtures for testing (#2533) +* Disable zookeeper admin server to avoid port conflicts +* Set default pytest log level to debug +* test_group: shorter timeout, more logging, more sleep +* Cache servers/dist in github actions workflow (#2527) +* Remove tox.ini; update testing docs +* Use thread-specific client_id in test_group +* Fix subprocess log warning; specify timeout_ms kwarg in consumer.poll tests +* Only set KAFKA_JVM_PERFORMANCE_OPTS in makefile if unset; add note re: 2.0-2.3 broker testing +* Add kafka command to test.fixtures; raise FileNotFoundError if version not installed + +Documentation +* Improve ClusterMetadata docs re: node_id/broker_id str/int types +* Document api_version_auto_timeout_ms default; override in group tests + +Fixes +* Signal close to metrics expire_loop +* Add kafka.util timeout_ms_fn +* fixup TopicAuthorizationFailedError construction +* Fix lint issues via ruff check (#2522) +* Make the "mock" dependency optional (only used in Python < 3.3). (#2518) + # 2.0.6 (Mar 4, 2025) Networking diff --git a/docs/changelog.rst b/docs/changelog.rst index 3216ad8ff..9441c07ef 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,104 @@ Changelog ========= +2.1.0 (Mar 14, 2025) +#################### + +Support Kafka Broker 2.1 API Baseline +------------------------------------- +* Add baseline leader_epoch support for ListOffsets v4 / FetchRequest v10 (#2511) +* Support OffsetFetch v5 / OffsetCommit v6 (2.1 baseline) (#2505) +* Support 2.1 baseline consumer group apis (#2503) +* Support FindCoordinatorRequest v2 in consumer and admin client (#2502) +* Support ListOffsets v3 in consumer (#2501) +* Support Fetch Request/Response v6 in consumer (#2500) +* Add support for Metadata Request/Response v7 (#2497) +* Implement Incremental Fetch Sessions / KIP-227 (#2508) +* Implement client-side connection throttling / KIP-219 (#2510) +* Add KafkaClient.api_version(operation) for best available from api_versions (#2495) + +Consumer +-------- +* Timeout coordinator poll / ensure_coordinator_ready / ensure_active_group (#2526) +* Add optional timeout_ms kwarg to remaining consumer/coordinator methods (#2544) +* Check for coordinator.poll failure in KafkaConsumer +* Only mark coordinator dead if connection_delay > 0 (#2530) +* Delay group coordinator until after bootstrap (#2539) +* KAFKA-4160: Ensure rebalance listener not called with coordinator lock (#1438) +* Call default_offset_commit_callback after `_maybe_auto_commit_offsets_async` (#2546) +* Remove legacy/v1 consumer message iterator (#2543) +* Log warning when attempting to list offsets for unknown topic/partition (#2540) +* Add heartbeat thread id to debug logs on start +* Add inner_timeout_ms handler to fetcher; add fallback (#2529) + +Producer +-------- +* KafkaProducer: Flush pending records before close() (#2537) +* Raise immediate error on producer.send after close (#2542) +* Limit producer close timeout to 1sec in __del__; use context managers to close in test_producer +* Use NullLogger in producer atexit cleanup +* Attempt to fix metadata race condition when partitioning in producer.send (#2523) +* Remove unused partial KIP-467 implementation (ProduceResponse batch error details) (#2524) + +AdminClient +----------- +* Implement perform leader election (#2536) +* Support delete_records (#2535) + +Networking +---------- +* Call ApiVersionsRequest during connection, prior to Sasl Handshake (#2493) +* Fake api_versions for old brokers, rename to ApiVersionsRequest, and handle error decoding (#2494) +* Debug log when skipping api_versions request with pre-configured api_version +* Only refresh metadata if connection fails all dns records (#2532) +* Support connections through SOCKS5 proxies (#2531) +* Fix OverflowError when connection_max_idle_ms is 0 or inf (#2538) +* socket.setblocking for eventlet/gevent compatibility +* Support custom per-request timeouts (#2498) +* Include request_timeout_ms in request debug log +* Support client.poll with future and timeout_ms +* mask unused afi var +* Debug log if check_version connection attempt fails + +SASL Modules +------------ +* Refactor Sasl authentication with SaslMechanism abstract base class; support SaslAuthenticate (#2515) +* Add SSPI (Kerberos for Windows) authentication mechanism (#2521) +* Support AWS_MSK_IAM authentication (#2519) +* Cleanup sasl mechanism configuration checks; fix gssapi bugs; add sasl_kerberos_name config (#2520) +* Move kafka.oauth.AbstractTokenProvider -> kafka.sasl.oauth.AbstractTokenProvider (#2525) + +Testing +------- +* Bump default python to 3.13 in CI tests (#2541) +* Update pytest log_format: use logger instead of filename; add thread id +* Improve test_consumer_group::test_group logging before group stabilized (#2534) +* Limit test duration to 5mins w/ pytest-timeout +* Fix external kafka/zk fixtures for testing (#2533) +* Disable zookeeper admin server to avoid port conflicts +* Set default pytest log level to debug +* test_group: shorter timeout, more logging, more sleep +* Cache servers/dist in github actions workflow (#2527) +* Remove tox.ini; update testing docs +* Use thread-specific client_id in test_group +* Fix subprocess log warning; specify timeout_ms kwarg in consumer.poll tests +* Only set KAFKA_JVM_PERFORMANCE_OPTS in makefile if unset; add note re: 2.0-2.3 broker testing +* Add kafka command to test.fixtures; raise FileNotFoundError if version not installed + +Documentation +------------- +* Improve ClusterMetadata docs re: node_id/broker_id str/int types +* Document api_version_auto_timeout_ms default; override in group tests + +Fixes +----- +* Signal close to metrics expire_loop +* Add kafka.util timeout_ms_fn +* fixup TopicAuthorizationFailedError construction +* Fix lint issues via ruff check (#2522) +* Make the "mock" dependency optional (only used in Python < 3.3). (#2518) + + 2.0.6 (Mar 4, 2025) ################### diff --git a/docs/compatibility.rst b/docs/compatibility.rst index d9e2ba957..710542c40 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, authentication codecs, and -transactional producer/consumer support are not fully implemented. PRs welcome! +not all features are supported. Specifically, transactional producer/consumer +support is not fully implemented. PRs welcome! kafka-python is tested on python 2.7, and 3.8-3.12. diff --git a/kafka/version.py b/kafka/version.py index 83d888e17..a33997dd1 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.0.dev' +__version__ = '2.1.0' From cebfed210c7bca4a6c699df6748b008dbb7ce087 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 15 Mar 2025 19:34:24 -0700 Subject: [PATCH 056/205] Bump version for development / fix release date --- CHANGES.md | 2 +- docs/changelog.rst | 2 +- kafka/version.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 8a0be7283..3ebd8180b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,4 +1,4 @@ -# 2.1.0 (Mar 14, 2025) +# 2.1.0 (Mar 15, 2025) Support Kafka Broker 2.1 API Baseline * Add baseline leader_epoch support for ListOffsets v4 / FetchRequest v10 (#2511) diff --git a/docs/changelog.rst b/docs/changelog.rst index 9441c07ef..170efc2f1 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,7 +1,7 @@ Changelog ========= -2.1.0 (Mar 14, 2025) +2.1.0 (Mar 15, 2025) #################### Support Kafka Broker 2.1 API Baseline diff --git a/kafka/version.py b/kafka/version.py index a33997dd1..98622e8c0 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.0' +__version__ = '2.1.1.dev' From e2477763066698e86da80e405dd3c8c40afa2e8d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 16 Mar 2025 10:43:35 -0700 Subject: [PATCH 057/205] Add lock synchronization to Future success/failure (#2549) --- kafka/future.py | 37 ++++++++++++++++++++++++------------- 1 file changed, 24 insertions(+), 13 deletions(-) diff --git a/kafka/future.py b/kafka/future.py index d0f3c6658..2af061ee7 100644 --- a/kafka/future.py +++ b/kafka/future.py @@ -2,6 +2,7 @@ import functools import logging +import threading log = logging.getLogger(__name__) @@ -15,6 +16,7 @@ def __init__(self): self.exception = None self._callbacks = [] self._errbacks = [] + self._lock = threading.Lock() def succeeded(self): return self.is_done and not bool(self.exception) @@ -30,37 +32,46 @@ def retriable(self): def success(self, value): assert not self.is_done, 'Future is already complete' - self.value = value - self.is_done = True + with self._lock: + self.value = value + self.is_done = True if self._callbacks: self._call_backs('callback', self._callbacks, self.value) return self def failure(self, e): assert not self.is_done, 'Future is already complete' - self.exception = e if type(e) is not type else e() - assert isinstance(self.exception, BaseException), ( + exception = e if type(e) is not type else e() + assert isinstance(exception, BaseException), ( 'future failed without an exception') - self.is_done = True + with self._lock: + self.exception = exception + self.is_done = True self._call_backs('errback', self._errbacks, self.exception) return self def add_callback(self, f, *args, **kwargs): if args or kwargs: f = functools.partial(f, *args, **kwargs) - if self.is_done and not self.exception: - self._call_backs('callback', [f], self.value) - else: - self._callbacks.append(f) + with self._lock: + if not self.is_done: + self._callbacks.append(f) + elif self.succeeded(): + self._lock.release() + self._call_backs('callback', [f], self.value) + self._lock.acquire() return self def add_errback(self, f, *args, **kwargs): if args or kwargs: f = functools.partial(f, *args, **kwargs) - if self.is_done and self.exception: - self._call_backs('errback', [f], self.exception) - else: - self._errbacks.append(f) + with self._lock: + if not self.is_done: + self._errbacks.append(f) + elif self.failed(): + self._lock.release() + self._call_backs('errback', [f], self.exception) + self._lock.acquire() return self def add_both(self, f, *args, **kwargs): From da4329ec951ad7c73e3871bdb61e2b02ecd3d953 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 16 Mar 2025 10:43:53 -0700 Subject: [PATCH 058/205] Improve connection error handling when try_api_versions_check fails all attempts (#2548) --- kafka/conn.py | 6 +++++- test/test_conn.py | 30 ++++++++++++++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/kafka/conn.py b/kafka/conn.py index b276d3d62..c94154885 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -531,6 +531,9 @@ def _try_api_versions_check(self): if self._api_versions_future is None: if self.config['api_version'] is not None: self._api_version = self.config['api_version'] + # api_version will be normalized by KafkaClient, so this should not happen + if self._api_version not in BROKER_API_VERSIONS: + raise Errors.UnrecognizedBrokerVersion('api_version %s not found in kafka.protocol.broker_api_versions' % (self._api_version,)) self._api_versions = BROKER_API_VERSIONS[self._api_version] log.debug('%s: Using pre-configured api_version %s for ApiVersions', self, self._api_version) return True @@ -553,7 +556,8 @@ def _try_api_versions_check(self): self.state = ConnectionStates.API_VERSIONS_RECV self.config['state_change_callback'](self.node_id, self._sock, self) else: - raise 'Unable to determine broker version.' + self.close(Errors.KafkaConnectionError('Unable to determine broker version.')) + return False for r, f in self.recv(): f.success(r) diff --git a/test/test_conn.py b/test/test_conn.py index ea88fd04c..6af01498f 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -69,6 +69,36 @@ def test_connect(_socket, conn, states): assert conn.state is state +def test_api_versions_check(_socket): + conn = BrokerConnection('localhost', 9092, socket.AF_INET) + assert conn._api_versions_future is None + conn.connect() + assert conn._api_versions_future is not None + assert conn.connecting() is True + assert conn.state is ConnectionStates.API_VERSIONS_RECV + + assert conn._try_api_versions_check() is False + assert conn.connecting() is True + assert conn.state is ConnectionStates.API_VERSIONS_RECV + + conn._api_versions_future = None + conn._check_version_idx = 0 + assert conn._try_api_versions_check() is False + assert conn.connecting() is True + + conn._check_version_idx = len(conn.VERSION_CHECKS) + conn._api_versions_future = None + assert conn._try_api_versions_check() is False + assert conn.connecting() is False + assert conn.disconnected() is True + + +def test_api_versions_check_unrecognized(_socket): + conn = BrokerConnection('localhost', 9092, socket.AF_INET, api_version=(0, 0)) + with pytest.raises(Errors.UnrecognizedBrokerVersion): + conn.connect() + + def test_connect_timeout(_socket, conn): assert conn.state is ConnectionStates.DISCONNECTED From 406f16c3f68246a40381c23abf4a3f30c8f05b09 Mon Sep 17 00:00:00 2001 From: "Romain Geissler @ Amadeus" Date: Sun, 16 Mar 2025 18:44:26 +0100 Subject: [PATCH 059/205] Fix packaging of 2.1.0 in Fedora: testing requires "pytest-timeout". (#2550) --- pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyproject.toml b/pyproject.toml index ddd40a08e..87132defb 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -37,7 +37,7 @@ crc32c = ["crc32c"] lz4 = ["lz4"] snappy = ["python-snappy"] zstd = ["zstandard"] -testing = ["pytest", "mock; python_version < '3.3'", "pytest-mock"] +testing = ["pytest", "mock; python_version < '3.3'", "pytest-mock", "pytest-timeout"] [tool.setuptools] include-package-data = false From 2eab4eea9b671108b1628107ecbfa1ee25718e21 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 16 Mar 2025 11:25:30 -0700 Subject: [PATCH 060/205] Fix StickyPartitionAssignor encode --- kafka/coordinator/assignors/sticky/sticky_assignor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/coordinator/assignors/sticky/sticky_assignor.py b/kafka/coordinator/assignors/sticky/sticky_assignor.py index 6e79c597e..69f68f564 100644 --- a/kafka/coordinator/assignors/sticky/sticky_assignor.py +++ b/kafka/coordinator/assignors/sticky/sticky_assignor.py @@ -659,7 +659,7 @@ def _metadata(cls, topics, member_assignment_partitions, generation=-1): partitions_by_topic = defaultdict(list) for topic_partition in member_assignment_partitions: partitions_by_topic[topic_partition.topic].append(topic_partition.partition) - data = StickyAssignorUserDataV1(six.viewitems(partitions_by_topic), generation) + data = StickyAssignorUserDataV1(list(partitions_by_topic.items()), generation) user_data = data.encode() return ConsumerProtocolMemberMetadata(cls.version, list(topics), user_data) From 1ed5b16b183d611c4afdeb9eda7f5e3463b4e951 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 16 Mar 2025 10:47:08 -0700 Subject: [PATCH 061/205] Patch Release 2.1.1 --- CHANGES.md | 8 ++++++++ docs/changelog.rst | 11 +++++++++++ kafka/version.py | 2 +- 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 3ebd8180b..78eab7769 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,11 @@ +# 2.1.1 (Mar 16, 2025) + +Fixes +* Fix packaging of 2.1.0 in Fedora: testing requires "pytest-timeout". (#2550) +* Improve connection error handling when try_api_versions_check fails all attempts (#2548) +* Add lock synchronization to Future success/failure (#2549) +* Fix StickyPartitionAssignor encode + # 2.1.0 (Mar 15, 2025) Support Kafka Broker 2.1 API Baseline diff --git a/docs/changelog.rst b/docs/changelog.rst index 170efc2f1..a72ef9eae 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,17 @@ Changelog ========= +2.1.1 (Mar 16, 2025) +#################### + +Fixes +----- +* Fix packaging of 2.1.0 in Fedora: testing requires "pytest-timeout". (#2550) +* Improve connection error handling when try_api_versions_check fails all attempts (#2548) +* Add lock synchronization to Future success/failure (#2549) +* Fix StickyPartitionAssignor encode + + 2.1.0 (Mar 15, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 98622e8c0..55fa725bd 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.1.dev' +__version__ = '2.1.1' From ca5592f26d0d71dfe1b5b3c96d9a5f7f180a5b20 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 16 Mar 2025 11:33:39 -0700 Subject: [PATCH 062/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 55fa725bd..d78c819ee 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.1' +__version__ = '2.1.2.dev' From b40682421d694ff212ec85bc6f47d64f29d3270f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 16 Mar 2025 18:09:26 -0700 Subject: [PATCH 063/205] Use six.add_metaclass for py2/py3 compatible abc (#2551) --- kafka/consumer/subscription_state.py | 3 +-- kafka/metrics/compound_stat.py | 4 ++-- kafka/metrics/measurable_stat.py | 3 ++- kafka/metrics/metrics_reporter.py | 5 +++-- kafka/metrics/stat.py | 5 +++-- kafka/metrics/stats/sampled_stat.py | 4 ++-- kafka/protocol/abstract.py | 5 +++-- kafka/protocol/api.py | 7 ++++--- kafka/record/abc.py | 11 +++++++---- kafka/sasl/abc.py | 5 +++-- 10 files changed, 30 insertions(+), 22 deletions(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index b30922b3e..0cfcfd2d4 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -421,6 +421,7 @@ def is_fetchable(self): return not self.paused and self.has_valid_position +@six.add_metaclass(abc.ABCMeta) class ConsumerRebalanceListener(object): """ A callback interface that the user can implement to trigger custom actions @@ -462,8 +463,6 @@ class ConsumerRebalanceListener(object): taking over that partition has their on_partitions_assigned() callback called to load the state. """ - __metaclass__ = abc.ABCMeta - @abc.abstractmethod def on_partitions_revoked(self, revoked): """ diff --git a/kafka/metrics/compound_stat.py b/kafka/metrics/compound_stat.py index ac92480dc..f8b149c54 100644 --- a/kafka/metrics/compound_stat.py +++ b/kafka/metrics/compound_stat.py @@ -3,16 +3,16 @@ import abc from kafka.metrics.stat import AbstractStat +from kafka.vendor.six import add_metaclass +@add_metaclass(abc.ABCMeta) class AbstractCompoundStat(AbstractStat): """ A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the example for a histogram which has many associated percentiles. """ - __metaclass__ = abc.ABCMeta - def stats(self): """ Return list of NamedMeasurable diff --git a/kafka/metrics/measurable_stat.py b/kafka/metrics/measurable_stat.py index 4487adf6e..08222b144 100644 --- a/kafka/metrics/measurable_stat.py +++ b/kafka/metrics/measurable_stat.py @@ -4,8 +4,10 @@ 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): """ An AbstractMeasurableStat is an AbstractStat that is also @@ -13,4 +15,3 @@ class AbstractMeasurableStat(AbstractStat, AbstractMeasurable): This is the interface used for most of the simple statistics such as Avg, Max, Count, etc. """ - __metaclass__ = abc.ABCMeta diff --git a/kafka/metrics/metrics_reporter.py b/kafka/metrics/metrics_reporter.py index d8bd12b3b..8df2e9ea6 100644 --- a/kafka/metrics/metrics_reporter.py +++ b/kafka/metrics/metrics_reporter.py @@ -2,14 +2,15 @@ import abc +from kafka.vendor.six import add_metaclass + +@add_metaclass(abc.ABCMeta) class AbstractMetricsReporter(object): """ An abstract class to allow things to listen as new metrics are created so they can be reported. """ - __metaclass__ = abc.ABCMeta - @abc.abstractmethod def init(self, metrics): """ diff --git a/kafka/metrics/stat.py b/kafka/metrics/stat.py index 9fd2f01ec..8825d2783 100644 --- a/kafka/metrics/stat.py +++ b/kafka/metrics/stat.py @@ -2,14 +2,15 @@ import abc +from kafka.vendor.six import add_metaclass + +@add_metaclass(abc.ABCMeta) class AbstractStat(object): """ An AbstractStat is a quantity such as average, max, etc that is computed off the stream of updates to a sensor """ - __metaclass__ = abc.ABCMeta - @abc.abstractmethod def record(self, config, value, time_ms): """ diff --git a/kafka/metrics/stats/sampled_stat.py b/kafka/metrics/stats/sampled_stat.py index c41b14bbc..146687916 100644 --- a/kafka/metrics/stats/sampled_stat.py +++ b/kafka/metrics/stats/sampled_stat.py @@ -3,8 +3,10 @@ import abc from kafka.metrics.measurable_stat import AbstractMeasurableStat +from kafka.vendor.six import add_metaclass +@add_metaclass(abc.ABCMeta) class AbstractSampledStat(AbstractMeasurableStat): """ An AbstractSampledStat records a single scalar value measured over @@ -20,8 +22,6 @@ class AbstractSampledStat(AbstractMeasurableStat): Subclasses of this class define different statistics measured using this basic pattern. """ - __metaclass__ = abc.ABCMeta - def __init__(self, initial_value): self._initial_value = initial_value self._samples = [] diff --git a/kafka/protocol/abstract.py b/kafka/protocol/abstract.py index 2de65c4bb..7ce5fc18f 100644 --- a/kafka/protocol/abstract.py +++ b/kafka/protocol/abstract.py @@ -2,10 +2,11 @@ import abc +from kafka.vendor.six import add_metaclass -class AbstractType(object): - __metaclass__ = abc.ABCMeta +@add_metaclass(abc.ABCMeta) +class AbstractType(object): @abc.abstractmethod def encode(cls, value): # pylint: disable=no-self-argument pass diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index f12cb972b..714da4d1d 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -5,6 +5,8 @@ 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( @@ -49,9 +51,8 @@ class ResponseHeaderV2(Struct): ) +@add_metaclass(abc.ABCMeta) class Request(Struct): - __metaclass__ = abc.ABCMeta - FLEXIBLE_VERSION = False @abc.abstractproperty @@ -92,8 +93,8 @@ def parse_response_header(self, read_buffer): return ResponseHeader.decode(read_buffer) +@add_metaclass(abc.ABCMeta) class Response(Struct): - __metaclass__ = abc.ABCMeta @abc.abstractproperty def API_KEY(self): diff --git a/kafka/record/abc.py b/kafka/record/abc.py index 8509e23e5..73f91a039 100644 --- a/kafka/record/abc.py +++ b/kafka/record/abc.py @@ -1,9 +1,12 @@ from __future__ import absolute_import + import abc +from kafka.vendor.six import add_metaclass + +@add_metaclass(abc.ABCMeta) class ABCRecord(object): - __metaclass__ = abc.ABCMeta __slots__ = () @abc.abstractproperty @@ -44,8 +47,8 @@ def headers(self): """ +@add_metaclass(abc.ABCMeta) class ABCRecordBatchBuilder(object): - __metaclass__ = abc.ABCMeta __slots__ = () @abc.abstractmethod @@ -84,11 +87,11 @@ def build(self): """ +@add_metaclass(abc.ABCMeta) class ABCRecordBatch(object): """ For v2 encapsulates a RecordBatch, for v0/v1 a single (maybe compressed) message. """ - __metaclass__ = abc.ABCMeta __slots__ = () @abc.abstractmethod @@ -98,8 +101,8 @@ def __iter__(self): """ +@add_metaclass(abc.ABCMeta) class ABCRecords(object): - __metaclass__ = abc.ABCMeta __slots__ = () @abc.abstractmethod diff --git a/kafka/sasl/abc.py b/kafka/sasl/abc.py index 8977c7c23..0577888a9 100644 --- a/kafka/sasl/abc.py +++ b/kafka/sasl/abc.py @@ -2,10 +2,11 @@ import abc +from kafka.vendor.six import add_metaclass -class SaslMechanism(object): - __metaclass__ = abc.ABCMeta +@add_metaclass(abc.ABCMeta) +class SaslMechanism(object): @abc.abstractmethod def __init__(self, **config): pass From d4a6a05df9a21e390db656715a2d5cfda2d8f0e3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 15:36:02 -0700 Subject: [PATCH 064/205] Fix Fetcher retriable error handling (#2554) --- kafka/consumer/fetcher.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 7527a1f39..641a0f259 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -934,9 +934,9 @@ def _parse_fetched_data(self, completed_fetch): elif error_type is Errors.TopicAuthorizationFailedError: log.warning("Not authorized to read from topic %s.", tp.topic) raise Errors.TopicAuthorizationFailedError(set([tp.topic])) - elif error_type.is_retriable: + elif getattr(error_type, 'retriable', False): log.debug("Retriable error fetching partition %s: %s", tp, error_type()) - if error_type.invalid_metadata: + if getattr(error_type, 'invalid_metadata', False): self._client.cluster.request_update() else: raise error_type('Unexpected error while fetching data') From 44eb48dd2d2fe3f7eaf6fc5583dbcaff6ecffcf7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 15:39:04 -0700 Subject: [PATCH 065/205] Add magic property to ABCRecord and implement for LegacyRecord --- kafka/record/abc.py | 5 +++++ kafka/record/legacy_records.py | 21 +++++++++++++++------ 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/kafka/record/abc.py b/kafka/record/abc.py index 73f91a039..c0071e605 100644 --- a/kafka/record/abc.py +++ b/kafka/record/abc.py @@ -100,6 +100,11 @@ def __iter__(self): if needed. """ + @abc.abstractproperty + def magic(self): + """ Return magic value (0, 1, 2) for batch. + """ + @add_metaclass(abc.ABCMeta) class ABCRecords(object): diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index 2f8523fcb..10d883119 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -164,6 +164,10 @@ def timestamp_type(self): def compression_type(self): return self._attributes & self.CODEC_MASK + @property + def magic(self): + return self._magic + def validate_crc(self): crc = calc_crc32(self._buffer[self.MAGIC_OFFSET:]) return self._crc == crc @@ -272,21 +276,22 @@ def __iter__(self): key, value = self._read_key_value(msg_pos + key_offset) yield LegacyRecord( - offset, timestamp, timestamp_type, + self._magic, offset, timestamp, timestamp_type, key, value, crc) else: key, value = self._read_key_value(key_offset) yield LegacyRecord( - self._offset, self._timestamp, timestamp_type, + self._magic, self._offset, self._timestamp, timestamp_type, key, value, self._crc) class LegacyRecord(ABCRecord): - __slots__ = ("_offset", "_timestamp", "_timestamp_type", "_key", "_value", + __slots__ = ("_magic", "_offset", "_timestamp", "_timestamp_type", "_key", "_value", "_crc") - def __init__(self, offset, timestamp, timestamp_type, key, value, crc): + def __init__(self, magic, offset, timestamp, timestamp_type, key, value, crc): + self._magic = magic self._offset = offset self._timestamp = timestamp self._timestamp_type = timestamp_type @@ -294,6 +299,10 @@ def __init__(self, offset, timestamp, timestamp_type, key, value, crc): self._value = value self._crc = crc + @property + def magic(self): + return self._magic + @property def offset(self): return self._offset @@ -332,9 +341,9 @@ def checksum(self): def __repr__(self): return ( - "LegacyRecord(offset={!r}, timestamp={!r}, timestamp_type={!r}," + "LegacyRecord(magic={!r} offset={!r}, timestamp={!r}, timestamp_type={!r}," " key={!r}, value={!r}, crc={!r})".format( - self._offset, self._timestamp, self._timestamp_type, + self._magic, self._offset, self._timestamp, self._timestamp_type, self._key, self._value, self._crc) ) From 83fed59f6b7467449d35d7755f46aad619e0866e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 15:39:55 -0700 Subject: [PATCH 066/205] Add size_in_bytes to ABCRecordBatch and implement for Legacy and Default --- kafka/record/abc.py | 5 +++++ kafka/record/default_records.py | 19 ++++++++++++------- kafka/record/legacy_records.py | 4 ++++ 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/kafka/record/abc.py b/kafka/record/abc.py index c0071e605..2bebceb02 100644 --- a/kafka/record/abc.py +++ b/kafka/record/abc.py @@ -9,6 +9,11 @@ class ABCRecord(object): __slots__ = () + @abc.abstractproperty + def size_in_bytes(self): + """ Number of total bytes in record + """ + @abc.abstractproperty def offset(self): """ Absolute offset of record diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index 14732cb06..cccf40a7b 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -275,10 +275,10 @@ def _read_msg( if self.is_control_batch: return ControlRecord( - offset, timestamp, self.timestamp_type, key, value, headers) + length, offset, timestamp, self.timestamp_type, key, value, headers) else: return DefaultRecord( - offset, timestamp, self.timestamp_type, key, value, headers) + length, offset, timestamp, self.timestamp_type, key, value, headers) def __iter__(self): self._maybe_uncompress() @@ -314,10 +314,11 @@ def validate_crc(self): class DefaultRecord(ABCRecord): - __slots__ = ("_offset", "_timestamp", "_timestamp_type", "_key", "_value", + __slots__ = ("_size_in_bytes", "_offset", "_timestamp", "_timestamp_type", "_key", "_value", "_headers") - def __init__(self, offset, timestamp, timestamp_type, key, value, headers): + def __init__(self, size_in_bytes, offset, timestamp, timestamp_type, key, value, headers): + self._size_in_bytes = size_in_bytes self._offset = offset self._timestamp = timestamp self._timestamp_type = timestamp_type @@ -325,6 +326,10 @@ def __init__(self, offset, timestamp, timestamp_type, key, value, headers): self._value = value self._headers = headers + @property + def size_in_bytes(self): + return self._size_in_bytes + @property def offset(self): return self._offset @@ -371,7 +376,7 @@ def __repr__(self): class ControlRecord(DefaultRecord): - __slots__ = ("_offset", "_timestamp", "_timestamp_type", "_key", "_value", + __slots__ = ("_size_in_bytes", "_offset", "_timestamp", "_timestamp_type", "_key", "_value", "_headers", "_version", "_type") KEY_STRUCT = struct.Struct( @@ -379,8 +384,8 @@ class ControlRecord(DefaultRecord): "h" # Type => Int16 (0 indicates an abort marker, 1 indicates a commit) ) - def __init__(self, offset, timestamp, timestamp_type, key, value, headers): - super(ControlRecord, self).__init__(offset, timestamp, timestamp_type, key, value, headers) + def __init__(self, size_in_bytes, offset, timestamp, timestamp_type, key, value, headers): + super(ControlRecord, self).__init__(size_in_bytes, offset, timestamp, timestamp_type, key, value, headers) (self._version, self._type) = self.KEY_STRUCT.unpack(self._key) # see https://kafka.apache.org/documentation/#controlbatch diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index 10d883119..28f5187f8 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -339,6 +339,10 @@ def headers(self): def checksum(self): return self._crc + @property + def size_in_bytes(self): + return LegacyRecordBatchBuilder.estimate_size_in_bytes(self._magic, None, self._key, self._value) + def __repr__(self): return ( "LegacyRecord(magic={!r} offset={!r}, timestamp={!r}, timestamp_type={!r}," From df12982d3184f93bb900c50701786bc97613c5ba Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 16:20:38 -0700 Subject: [PATCH 067/205] remove fetcher message_generator / iterator interface --- kafka/consumer/fetcher.py | 68 --------------------------------------- test/test_fetcher.py | 18 ----------- 2 files changed, 86 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 641a0f259..eaf89bfb6 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -407,62 +407,6 @@ def _append(self, drained, part, max_records, update_offsets): part.discard() return 0 - def _message_generator(self): - """Iterate over fetched_records""" - while self._next_partition_records or self._completed_fetches: - - if not self._next_partition_records: - completion = self._completed_fetches.popleft() - self._next_partition_records = self._parse_fetched_data(completion) - continue - - # Send additional FetchRequests when the internal queue is low - # this should enable moderate pipelining - if len(self._completed_fetches) <= self.config['iterator_refetch_records']: - self.send_fetches() - - tp = self._next_partition_records.topic_partition - - # We can ignore any prior signal to drop pending record batches - # because we are starting from a fresh one where fetch_offset == position - # i.e., the user seek()'d to this position - self._subscriptions.assignment[tp].drop_pending_record_batch = False - - for msg in self._next_partition_records.take(): - - # Because we are in a generator, it is possible for - # subscription state to change between yield calls - # so we need to re-check on each loop - # this should catch assignment changes, pauses - # and resets via seek_to_beginning / seek_to_end - if not self._subscriptions.is_fetchable(tp): - log.debug("Not returning fetched records for partition %s" - " since it is no longer fetchable", tp) - self._next_partition_records = None - break - - # If there is a seek during message iteration, - # we should stop unpacking this record batch and - # wait for a new fetch response that aligns with the - # new seek position - elif self._subscriptions.assignment[tp].drop_pending_record_batch: - log.debug("Skipping remainder of record batch for partition %s", tp) - self._subscriptions.assignment[tp].drop_pending_record_batch = False - self._next_partition_records = None - break - - # Compressed messagesets may include earlier messages - elif msg.offset < self._subscriptions.assignment[tp].position.offset: - log.debug("Skipping message offset: %s (expecting %s)", - msg.offset, - self._subscriptions.assignment[tp].position.offset) - continue - - self._subscriptions.assignment[tp].position = OffsetAndMetadata(msg.offset + 1, '', -1) - yield msg - - self._next_partition_records = None - def _unpack_records(self, tp, records): try: batch = records.next_batch() @@ -514,18 +458,6 @@ def _unpack_records(self, tp, records): log.exception('StopIteration raised unpacking messageset') raise RuntimeError('StopIteration raised unpacking messageset') - def __iter__(self): # pylint: disable=non-iterator-returned - return self - - def __next__(self): - if not self._iterator: - self._iterator = self._message_generator() - try: - return next(self._iterator) - except StopIteration: - self._iterator = None - raise - def _deserialize(self, f, topic, bytes_): if not f: return bytes_ diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 3bf334e06..eedcf4f7a 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -463,24 +463,6 @@ def test__unpack_records(fetcher): assert records[2].offset == 2 -def test__message_generator(fetcher, topic, mocker): - fetcher.config['check_crcs'] = False - tp = TopicPartition(topic, 0) - msgs = [] - for i in range(10): - msgs.append((None, b"foo", None)) - completed_fetch = CompletedFetch( - tp, 0, 0, [0, 100, _build_record_batch(msgs)], - mocker.MagicMock() - ) - fetcher._completed_fetches.append(completed_fetch) - for i in range(10): - msg = next(fetcher) - assert isinstance(msg, ConsumerRecord) - assert msg.offset == i - assert msg.value == b'foo' - - def test__parse_fetched_data(fetcher, topic, mocker): fetcher.config['check_crcs'] = False tp = TopicPartition(topic, 0) From c763939e30ffdba4efae0cc727cc8609fec54464 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 18:46:33 -0700 Subject: [PATCH 068/205] Add record.validate_crc() for v0/v1 crc checks --- kafka/record/abc.py | 5 +++++ kafka/record/default_records.py | 3 +++ kafka/record/legacy_records.py | 20 +++++++++++++++----- 3 files changed, 23 insertions(+), 5 deletions(-) diff --git a/kafka/record/abc.py b/kafka/record/abc.py index 2bebceb02..df7178562 100644 --- a/kafka/record/abc.py +++ b/kafka/record/abc.py @@ -45,6 +45,11 @@ def checksum(self): be the checksum for v0 and v1 and None for v2 and above. """ + @abc.abstractmethod + def validate_crc(self): + """ Return True if v0/v1 record matches checksum. noop/True for v2 records + """ + @abc.abstractproperty def headers(self): """ If supported by version list of key-value tuples, or empty list if diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index cccf40a7b..fb2990b53 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -366,6 +366,9 @@ def headers(self): def checksum(self): return None + def validate_crc(self): + return True + def __repr__(self): return ( "DefaultRecord(offset={!r}, timestamp={!r}, timestamp_type={!r}," diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index 28f5187f8..920b4fcc6 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -236,6 +236,9 @@ def _read_key_value(self, pos): value = self._buffer[pos:pos + value_size].tobytes() return key, value + def _crc_bytes(self, msg_pos, length): + return self._buffer[msg_pos + self.MAGIC_OFFSET:msg_pos + self.LOG_OVERHEAD + length] + def __iter__(self): if self._magic == 1: key_offset = self.KEY_OFFSET_V1 @@ -259,7 +262,7 @@ def __iter__(self): absolute_base_offset = -1 for header, msg_pos in headers: - offset, _, crc, _, attrs, timestamp = header + offset, length, crc, _, attrs, timestamp = header # There should only ever be a single layer of compression assert not attrs & self.CODEC_MASK, ( 'MessageSet at offset %d appears double-compressed. This ' @@ -275,22 +278,24 @@ def __iter__(self): offset += absolute_base_offset key, value = self._read_key_value(msg_pos + key_offset) + crc_bytes = self._crc_bytes(msg_pos, length) yield LegacyRecord( self._magic, offset, timestamp, timestamp_type, - key, value, crc) + key, value, crc, crc_bytes) else: key, value = self._read_key_value(key_offset) + crc_bytes = self._crc_bytes(0, len(self._buffer) - self.LOG_OVERHEAD) yield LegacyRecord( self._magic, self._offset, self._timestamp, timestamp_type, - key, value, self._crc) + key, value, self._crc, crc_bytes) class LegacyRecord(ABCRecord): __slots__ = ("_magic", "_offset", "_timestamp", "_timestamp_type", "_key", "_value", - "_crc") + "_crc", "_crc_bytes") - def __init__(self, magic, offset, timestamp, timestamp_type, key, value, crc): + def __init__(self, magic, offset, timestamp, timestamp_type, key, value, crc, crc_bytes): self._magic = magic self._offset = offset self._timestamp = timestamp @@ -298,6 +303,7 @@ def __init__(self, magic, offset, timestamp, timestamp_type, key, value, crc): self._key = key self._value = value self._crc = crc + self._crc_bytes = crc_bytes @property def magic(self): @@ -339,6 +345,10 @@ def headers(self): def checksum(self): return self._crc + def validate_crc(self): + crc = calc_crc32(self._crc_bytes) + return self._crc == crc + @property def size_in_bytes(self): return LegacyRecordBatchBuilder.estimate_size_in_bytes(self._magic, None, self._key, self._value) From 585a111903d4dfd35b9baf2ca433ea206188abee Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 18:47:56 -0700 Subject: [PATCH 069/205] MemoryRecordsBuilder: support arbitrary offset, skipping offsets --- kafka/record/memory_records.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index fc2ef2d6b..72baea547 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -115,7 +115,7 @@ class MemoryRecordsBuilder(object): __slots__ = ("_builder", "_batch_size", "_buffer", "_next_offset", "_closed", "_bytes_written") - def __init__(self, magic, compression_type, batch_size): + def __init__(self, magic, compression_type, batch_size, offset=0): assert magic in [0, 1, 2], "Not supported magic" assert compression_type in [0, 1, 2, 3, 4], "Not valid compression type" if magic >= 2: @@ -130,10 +130,14 @@ def __init__(self, magic, compression_type, batch_size): self._batch_size = batch_size self._buffer = None - self._next_offset = 0 + self._next_offset = offset self._closed = False self._bytes_written = 0 + def skip(self, offsets_to_skip): + # Exposed for testing compacted records + self._next_offset += offsets_to_skip + def append(self, timestamp, key, value, headers=[]): """ Append a message to the buffer. From 8be25684415aafe78a6d5c834be34ddb2f4c3505 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 18:48:31 -0700 Subject: [PATCH 070/205] DefaultRecordsBatchBuilder: support empty batch --- kafka/record/default_records.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index fb2990b53..0d69d72a2 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -556,8 +556,8 @@ def write_header(self, use_compression_type=True): 0, # CRC will be set below, as we need a filled buffer for it self._get_attributes(use_compression_type), self._last_offset, - self._first_timestamp, - self._max_timestamp, + self._first_timestamp or 0, + self._max_timestamp or 0, self._producer_id, self._producer_epoch, self._base_sequence, From b75248e6f5d05c5e68601c0ece3f0d3fa675df6c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 18:51:20 -0700 Subject: [PATCH 071/205] Add FetchMetrics class; move topic_fetch_metrics inside aggregator --- kafka/consumer/fetcher.py | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index eaf89bfb6..2f9e442fe 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -843,7 +843,6 @@ def _parse_fetched_data(self, completed_fetch): record_too_large_partitions, self.config['max_partition_fetch_bytes']), record_too_large_partitions) - self._sensors.record_topic_fetch_metrics(tp.topic, num_bytes, records_count) elif error_type in (Errors.NotLeaderForPartitionError, Errors.ReplicaNotAvailableError, @@ -1128,6 +1127,14 @@ def to_forget(self): return list(partition_data.items()) +class FetchMetrics(object): + __slots__ = ('total_bytes', 'total_records') + + def __init__(self): + self.total_bytes = 0 + self.total_records = 0 + + class FetchResponseMetricAggregator(object): """ Since we parse the message data for each partition from each fetch @@ -1138,8 +1145,8 @@ class FetchResponseMetricAggregator(object): def __init__(self, sensors, partitions): self.sensors = sensors self.unrecorded_partitions = partitions - self.total_bytes = 0 - self.total_records = 0 + self.fetch_metrics = FetchMetrics() + self.topic_fetch_metrics = collections.defaultdict(FetchMetrics) def record(self, partition, num_bytes, num_records): """ @@ -1148,13 +1155,17 @@ def record(self, partition, num_bytes, num_records): have reported, we write the metric. """ self.unrecorded_partitions.remove(partition) - self.total_bytes += num_bytes - self.total_records += num_records + self.fetch_metrics.total_bytes += num_bytes + self.fetch_metrics.total_records += num_records + self.topic_fetch_metrics[partition.topic].total_bytes += num_bytes + self.topic_fetch_metrics[partition.topic].total_records += num_records # once all expected partitions from the fetch have reported in, record the metrics if not self.unrecorded_partitions: - self.sensors.bytes_fetched.record(self.total_bytes) - self.sensors.records_fetched.record(self.total_records) + 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): + self.sensors.record_topic_fetch_metrics(topic, metrics.total_bytes, metrics.total_records) class FetchManagerMetrics(object): From cab08df86f01b6dc4f09b051ab634d5aa0218d5b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 18:53:01 -0700 Subject: [PATCH 072/205] revert last_offset_from_record_batch --- kafka/consumer/fetcher.py | 11 ----------- kafka/consumer/subscription_state.py | 5 ----- 2 files changed, 16 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 2f9e442fe..05354cea9 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -417,7 +417,6 @@ def _unpack_records(self, tp, records): try: batch_offset = batch.base_offset + batch.last_offset_delta leader_epoch = batch.leader_epoch - self._subscriptions.assignment[tp].last_offset_from_record_batch = batch_offset # Control batches have a single record indicating whether a transaction # was aborted or committed. # When isolation_level is READ_COMMITTED (currently unsupported) @@ -643,16 +642,6 @@ def _create_fetch_requests(self): for partition in self._fetchable_partitions(): node_id = self._client.cluster.leader_for_partition(partition) - # advance position for any deleted compacted messages if required - if self._subscriptions.assignment[partition].last_offset_from_record_batch: - next_offset_from_batch_header = self._subscriptions.assignment[partition].last_offset_from_record_batch + 1 - if next_offset_from_batch_header > self._subscriptions.assignment[partition].position.offset: - log.debug( - "Advance position for partition %s from %s to %s (last record batch location plus one)" - " to correct for deleted compacted messages and/or transactional control records", - partition, self._subscriptions.assignment[partition].position.offset, next_offset_from_batch_header) - self._subscriptions.assignment[partition].position = OffsetAndMetadata(next_offset_from_batch_header, '', -1) - position = self._subscriptions.assignment[partition].position # fetch if there is a leader and no in-flight requests diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 0cfcfd2d4..abe37fb86 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -382,9 +382,6 @@ def __init__(self): self._position = None # OffsetAndMetadata exposed to the user self.highwater = None self.drop_pending_record_batch = False - # The last message offset hint available from a record batch with - # magic=2 which includes deleted compacted messages - self.last_offset_from_record_batch = None def _set_position(self, offset): assert self.has_valid_position, 'Valid position required' @@ -400,7 +397,6 @@ def await_reset(self, strategy): self.awaiting_reset = True self.reset_strategy = strategy self._position = None - self.last_offset_from_record_batch = None self.has_valid_position = False def seek(self, offset): @@ -409,7 +405,6 @@ def seek(self, offset): self.reset_strategy = None self.has_valid_position = True self.drop_pending_record_batch = True - self.last_offset_from_record_batch = None def pause(self): self.paused = True From 36f7a0cc9750b736a719a3ec0812ed70dee23f03 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 19:02:14 -0700 Subject: [PATCH 073/205] Debug log fetch records return; separate offsets update log --- kafka/consumer/fetcher.py | 28 ++++++++++++---------------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 05354cea9..c40106cfa 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -363,37 +363,33 @@ def _append(self, drained, part, max_records, update_offsets): return 0 tp = part.topic_partition - fetch_offset = part.fetch_offset if not self._subscriptions.is_assigned(tp): # this can happen when a rebalance happened before # fetched records are returned to the consumer's poll call log.debug("Not returning fetched records for partition %s" " since it is no longer assigned", tp) + elif not self._subscriptions.is_fetchable(tp): + # this can happen when a partition is paused before + # fetched records are returned to the consumer's poll call + log.debug("Not returning fetched records for assigned partition" + " %s since it is no longer fetchable", tp) + else: # note that the position should always be available # as long as the partition is still assigned position = self._subscriptions.assignment[tp].position - if not self._subscriptions.is_fetchable(tp): - # this can happen when a partition is paused before - # fetched records are returned to the consumer's poll call - log.debug("Not returning fetched records for assigned partition" - " %s since it is no longer fetchable", tp) - - elif fetch_offset == position.offset: - # we are ensured to have at least one record since we already checked for emptiness + if part.fetch_offset == position.offset: part_records = part.take(max_records) next_offset = part_records[-1].offset + 1 leader_epoch = part_records[-1].leader_epoch - log.log(0, "Returning fetched records at offset %d for assigned" - " partition %s and update position to %s (leader epoch %s)", position.offset, - tp, next_offset, leader_epoch) - - for record in part_records: - drained[tp].append(record) - + log.debug("Returning fetched records at offset %d for assigned" + " partition %s", position.offset, tp) + drained[tp].extend(part_records) if update_offsets: # TODO: save leader_epoch + log.debug("Updating fetch position for assigned partition %s to %s (leader epoch %s)", + tp, next_offset, leader_epoch) self._subscriptions.assignment[tp].position = OffsetAndMetadata(next_offset, '', -1) return len(part_records) From a0b96ecc18db98423cdf7cf2ecc1145a6f4d7ae9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 19:05:47 -0700 Subject: [PATCH 074/205] Lazy _unpack_records in PartitionRecords --- kafka/consumer/fetcher.py | 226 ++++++++++++++++++++------------------ test/test_fetcher.py | 78 +++++++------ 2 files changed, 163 insertions(+), 141 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index c40106cfa..840393a5d 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -2,6 +2,7 @@ import collections import copy +import itertools import logging import random import sys @@ -378,88 +379,35 @@ def _append(self, drained, part, max_records, update_offsets): # note that the position should always be available # as long as the partition is still assigned position = self._subscriptions.assignment[tp].position - if part.fetch_offset == position.offset: + if part.next_fetch_offset == position.offset: part_records = part.take(max_records) - next_offset = part_records[-1].offset + 1 - leader_epoch = part_records[-1].leader_epoch - log.debug("Returning fetched records at offset %d for assigned" " partition %s", position.offset, tp) drained[tp].extend(part_records) - if update_offsets: + # We want to increment subscription position if (1) we're using consumer.poll(), + # or (2) we didn't return any records (consumer iterator will update position + # when each message is yielded). There may be edge cases where we re-fetch records + # that we'll end up skipping, but for now we'll live with that. + highwater = self._subscriptions.assignment[tp].highwater + if highwater is not None: + self._sensors.records_fetch_lag.record(highwater - part.next_fetch_offset) + if update_offsets or not part_records: # TODO: save leader_epoch log.debug("Updating fetch position for assigned partition %s to %s (leader epoch %s)", - tp, next_offset, leader_epoch) - self._subscriptions.assignment[tp].position = OffsetAndMetadata(next_offset, '', -1) + tp, part.next_fetch_offset, part.leader_epoch) + self._subscriptions.assignment[tp].position = OffsetAndMetadata(part.next_fetch_offset, '', -1) return len(part_records) else: # these records aren't next in line based on the last consumed # position, ignore them they must be from an obsolete request log.debug("Ignoring fetched records for %s at offset %s since" - " the current position is %d", tp, part.fetch_offset, + " the current position is %d", tp, part.next_fetch_offset, position.offset) - part.discard() + part.drain() return 0 - def _unpack_records(self, tp, records): - try: - batch = records.next_batch() - while batch is not None: - - # Try DefaultsRecordBatch / message log format v2 - # base_offset, last_offset_delta, and control batches - try: - batch_offset = batch.base_offset + batch.last_offset_delta - leader_epoch = batch.leader_epoch - # Control batches have a single record indicating whether a transaction - # was aborted or committed. - # When isolation_level is READ_COMMITTED (currently unsupported) - # we should also skip all messages from aborted transactions - # For now we only support READ_UNCOMMITTED and so we ignore the - # abort/commit signal. - if batch.is_control_batch: - batch = records.next_batch() - continue - except AttributeError: - leader_epoch = -1 - pass - - for record in batch: - key_size = len(record.key) if record.key is not None else -1 - value_size = len(record.value) if record.value is not None else -1 - key = self._deserialize( - self.config['key_deserializer'], - tp.topic, record.key) - value = self._deserialize( - self.config['value_deserializer'], - tp.topic, record.value) - headers = record.headers - header_size = sum( - len(h_key.encode("utf-8")) + (len(h_val) if h_val is not None else 0) for h_key, h_val in - headers) if headers else -1 - yield ConsumerRecord( - tp.topic, tp.partition, leader_epoch, record.offset, record.timestamp, - record.timestamp_type, key, value, headers, record.checksum, - key_size, value_size, header_size) - - batch = records.next_batch() - - # If unpacking raises StopIteration, it is erroneously - # caught by the generator. We want all exceptions to be raised - # back to the user. See Issue 545 - except StopIteration: - log.exception('StopIteration raised unpacking messageset') - raise RuntimeError('StopIteration raised unpacking messageset') - - def _deserialize(self, f, topic, bytes_): - if not f: - return bytes_ - if isinstance(f, Deserializer): - return f.deserialize(topic, bytes_) - return f(bytes_) - def _send_list_offsets_requests(self, timestamps): """Fetch offsets for each partition in timestamps dict. This may send request to multiple nodes, based on who is Leader for partition. @@ -773,12 +721,9 @@ def _handle_fetch_error(self, node_id, exception): def _parse_fetched_data(self, completed_fetch): tp = completed_fetch.topic_partition fetch_offset = completed_fetch.fetched_offset - num_bytes = 0 - records_count = 0 - parsed_records = None - error_code, highwater = completed_fetch.partition_data[:2] error_type = Errors.for_code(error_code) + parsed_records = None try: if not self._subscriptions.is_fetchable(tp): @@ -807,13 +752,12 @@ def _parse_fetched_data(self, completed_fetch): log.debug("Adding fetched record for partition %s with" " offset %d to buffered record list", tp, position.offset) - unpacked = list(self._unpack_records(tp, records)) - parsed_records = self.PartitionRecords(fetch_offset, tp, unpacked) - if unpacked: - last_offset = unpacked[-1].offset - self._sensors.records_fetch_lag.record(highwater - last_offset) - num_bytes = records.valid_bytes() - records_count = len(unpacked) + parsed_records = self.PartitionRecords(fetch_offset, tp, records, + self.config['key_deserializer'], + self.config['value_deserializer'], + self.config['check_crcs'], + completed_fetch.metric_aggregator) + return parsed_records elif records.size_in_bytes() > 0: # we did not read a single message from a non-empty # buffer because that message's size is larger than @@ -858,52 +802,116 @@ def _parse_fetched_data(self, completed_fetch): raise error_type('Unexpected error while fetching data') finally: - completed_fetch.metric_aggregator.record(tp, num_bytes, records_count) + if parsed_records is None: + completed_fetch.metric_aggregator.record(tp, 0, 0) - return parsed_records + return None + + def close(self): + if self._next_partition_records is not None: + self._next_partition_records.drain() class PartitionRecords(object): - def __init__(self, fetch_offset, tp, messages): + def __init__(self, fetch_offset, tp, records, key_deserializer, value_deserializer, check_crcs, metric_aggregator): self.fetch_offset = fetch_offset self.topic_partition = tp - self.messages = messages + self.leader_epoch = -1 + self.next_fetch_offset = fetch_offset + self.bytes_read = 0 + self.records_read = 0 + self.metric_aggregator = metric_aggregator + self.check_crcs = check_crcs + self.record_iterator = itertools.dropwhile( + self._maybe_skip_record, + self._unpack_records(tp, records, key_deserializer, value_deserializer)) + + def _maybe_skip_record(self, record): # When fetching an offset that is in the middle of a # compressed batch, we will get all messages in the batch. # But we want to start 'take' at the fetch_offset # (or the next highest offset in case the message was compacted) - for i, msg in enumerate(messages): - if msg.offset < fetch_offset: - log.debug("Skipping message offset: %s (expecting %s)", - msg.offset, fetch_offset) - else: - self.message_idx = i - break - + if record.offset < self.fetch_offset: + log.debug("Skipping message offset: %s (expecting %s)", + record.offset, self.fetch_offset) + return True else: - self.message_idx = 0 - self.messages = None + return False - # For truthiness evaluation we need to define __len__ or __nonzero__ - def __len__(self): - if self.messages is None or self.message_idx >= len(self.messages): - return 0 - return len(self.messages) - self.message_idx + # For truthiness evaluation + def __bool__(self): + return self.record_iterator is not None - def discard(self): - self.messages = None + def drain(self): + if self.record_iterator is not None: + self.record_iterator = None + self.metric_aggregator.record(self.topic_partition, self.bytes_read, self.records_read) def take(self, n=None): - if not len(self): - return [] - if n is None or n > len(self): - n = len(self) - next_idx = self.message_idx + n - res = self.messages[self.message_idx:next_idx] - self.message_idx = next_idx - # fetch_offset should be incremented by 1 to parallel the - # subscription position (also incremented by 1) - self.fetch_offset = max(self.fetch_offset, res[-1].offset + 1) - return res + return list(itertools.islice(self.record_iterator, 0, n)) + + def _unpack_records(self, tp, records, key_deserializer, value_deserializer): + try: + batch = records.next_batch() + last_batch = None + while batch is not None: + last_batch = batch + + # Try DefaultsRecordBatch / message log format v2 + # base_offset, last_offset_delta, and control batches + if batch.magic == 2: + self.leader_epoch = batch.leader_epoch + # Control batches have a single record indicating whether a transaction + # was aborted or committed. + # When isolation_level is READ_COMMITTED (currently unsupported) + # we should also skip all messages from aborted transactions + # For now we only support READ_UNCOMMITTED and so we ignore the + # abort/commit signal. + if batch.is_control_batch: + self.next_fetch_offset = next(batch).offset + 1 + batch = records.next_batch() + continue + + for record in batch: + key_size = len(record.key) if record.key is not None else -1 + value_size = len(record.value) if record.value is not None else -1 + key = self._deserialize(key_deserializer, tp.topic, record.key) + value = self._deserialize(value_deserializer, tp.topic, record.value) + headers = record.headers + header_size = sum( + len(h_key.encode("utf-8")) + (len(h_val) if h_val is not None else 0) for h_key, h_val in + headers) if headers else -1 + self.records_read += 1 + self.bytes_read += record.size_in_bytes + self.next_fetch_offset = record.offset + 1 + yield ConsumerRecord( + tp.topic, tp.partition, self.leader_epoch, record.offset, record.timestamp, + record.timestamp_type, key, value, headers, record.checksum, + key_size, value_size, header_size) + + batch = records.next_batch() + else: + # Message format v2 preserves the last offset in a batch even if the last record is removed + # through compaction. By using the next offset computed from the last offset in the batch, + # we ensure that the offset of the next fetch will point to the next batch, which avoids + # unnecessary re-fetching of the same batch (in the worst case, the consumer could get stuck + # fetching the same batch repeatedly). + if last_batch and last_batch.magic == 2: + self.next_fetch_offset = last_batch.base_offset + last_batch.last_offset_delta + 1 + self.drain() + + # If unpacking raises StopIteration, it is erroneously + # caught by the generator. We want all exceptions to be raised + # back to the user. See Issue 545 + except StopIteration: + log.exception('StopIteration raised unpacking messageset') + raise RuntimeError('StopIteration raised unpacking messageset') + + def _deserialize(self, f, topic, bytes_): + if not f: + return bytes_ + if isinstance(f, Deserializer): + return f.deserialize(topic, bytes_) + return f(bytes_) class FetchSessionHandler(object): diff --git a/test/test_fetcher.py b/test/test_fetcher.py index eedcf4f7a..a22f78657 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -52,9 +52,9 @@ def fetcher(client, subscription_state, topic): return Fetcher(client, subscription_state, Metrics()) -def _build_record_batch(msgs, compression=0): +def _build_record_batch(msgs, compression=0, offset=0, magic=2): builder = MemoryRecordsBuilder( - magic=1, compression_type=0, batch_size=9999999) + magic=magic, compression_type=0, batch_size=9999999, offset=offset) for msg in msgs: key, value, timestamp = msg builder.append(key=key, value=value, timestamp=timestamp, headers=[]) @@ -443,8 +443,7 @@ def test__handle_fetch_error(fetcher, caplog, exception, log_level): assert caplog.records[0].levelname == logging.getLevelName(log_level) -def test__unpack_records(fetcher): - fetcher.config['check_crcs'] = False +def test__unpack_records(mocker): tp = TopicPartition('foo', 0) messages = [ (None, b"a", None), @@ -452,7 +451,8 @@ def test__unpack_records(fetcher): (None, b"c", None), ] memory_records = MemoryRecords(_build_record_batch(messages)) - records = list(fetcher._unpack_records(tp, memory_records)) + part_records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock()) + records = list(part_records.record_iterator) assert len(records) == 3 assert all(map(lambda x: isinstance(x, ConsumerRecord), records)) assert records[0].value == b'a' @@ -475,7 +475,8 @@ def test__parse_fetched_data(fetcher, topic, mocker): ) partition_record = fetcher._parse_fetched_data(completed_fetch) assert isinstance(partition_record, fetcher.PartitionRecords) - assert len(partition_record) == 10 + assert partition_record + assert len(partition_record.take()) == 10 def test__parse_fetched_data__paused(fetcher, topic, mocker): @@ -545,7 +546,7 @@ def test__parse_fetched_data__out_of_range(fetcher, topic, mocker): assert fetcher._subscriptions.assignment[tp].awaiting_reset is True -def test_partition_records_offset(): +def test_partition_records_offset(mocker): """Test that compressed messagesets are handle correctly when fetch offset is in the middle of the message list """ @@ -553,39 +554,45 @@ def test_partition_records_offset(): batch_end = 130 fetch_offset = 123 tp = TopicPartition('foo', 0) - messages = [ConsumerRecord(tp.topic, tp.partition, -1, i, - None, None, 'key', 'value', [], 'checksum', 0, 0, -1) - for i in range(batch_start, batch_end)] - records = Fetcher.PartitionRecords(fetch_offset, None, messages) - assert len(records) > 0 + messages = [(None, b'msg', None) for i in range(batch_start, batch_end)] + memory_records = MemoryRecords(_build_record_batch(messages, offset=batch_start)) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock()) + assert records + assert records.next_fetch_offset == fetch_offset msgs = records.take(1) assert msgs[0].offset == fetch_offset - assert records.fetch_offset == fetch_offset + 1 + assert records.next_fetch_offset == fetch_offset + 1 msgs = records.take(2) assert len(msgs) == 2 - assert len(records) > 0 - records.discard() - assert len(records) == 0 + assert records + assert records.next_fetch_offset == fetch_offset + 3 + records.drain() + assert not records -def test_partition_records_empty(): - records = Fetcher.PartitionRecords(0, None, []) - assert len(records) == 0 +def test_partition_records_empty(mocker): + tp = TopicPartition('foo', 0) + memory_records = MemoryRecords(_build_record_batch([])) + records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock()) + msgs = records.take() + assert len(msgs) == 0 + assert not records -def test_partition_records_no_fetch_offset(): +def test_partition_records_no_fetch_offset(mocker): batch_start = 0 batch_end = 100 fetch_offset = 123 tp = TopicPartition('foo', 0) - messages = [ConsumerRecord(tp.topic, tp.partition, -1, i, - None, None, 'key', 'value', None, 'checksum', 0, 0, -1) - for i in range(batch_start, batch_end)] - records = Fetcher.PartitionRecords(fetch_offset, None, messages) - assert len(records) == 0 + messages = [(None, b'msg', None) for i in range(batch_start, batch_end)] + memory_records = MemoryRecords(_build_record_batch(messages, offset=batch_start)) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock()) + msgs = records.take() + assert len(msgs) == 0 + assert not records -def test_partition_records_compacted_offset(): +def test_partition_records_compacted_offset(mocker): """Test that messagesets are handle correctly when the fetch offset points to a message that has been compacted """ @@ -593,10 +600,17 @@ def test_partition_records_compacted_offset(): batch_end = 100 fetch_offset = 42 tp = TopicPartition('foo', 0) - messages = [ConsumerRecord(tp.topic, tp.partition, -1, i, - None, None, 'key', 'value', None, 'checksum', 0, 0, -1) - for i in range(batch_start, batch_end) if i != fetch_offset] - records = Fetcher.PartitionRecords(fetch_offset, None, messages) - assert len(records) == batch_end - fetch_offset - 1 - msgs = records.take(1) + builder = MemoryRecordsBuilder( + magic=2, compression_type=0, batch_size=9999999) + + for i in range(batch_start, batch_end): + if i == fetch_offset: + builder.skip(1) + else: + builder.append(key=None, value=b'msg', timestamp=None, headers=[]) + builder.close() + memory_records = MemoryRecords(builder.buffer()) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock()) + msgs = records.take() + assert len(msgs) == batch_end - fetch_offset - 1 assert msgs[0].offset == fetch_offset + 1 From a441ef312bc3ccdf461149dc354e2ea966d22a60 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 19:06:54 -0700 Subject: [PATCH 075/205] Validate crcs in fetcher --- kafka/consumer/fetcher.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 840393a5d..90dfdbbbc 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -856,6 +856,11 @@ def _unpack_records(self, tp, records, key_deserializer, value_deserializer): while batch is not None: last_batch = batch + if self.check_crcs and not batch.validate_crc(): + raise Errors.CorruptRecordException( + "Record batch for partition %s at offset %s failed crc check" % ( + self.topic_partition, batch.base_offset)) + # Try DefaultsRecordBatch / message log format v2 # base_offset, last_offset_delta, and control batches if batch.magic == 2: @@ -872,6 +877,10 @@ def _unpack_records(self, tp, records, key_deserializer, value_deserializer): continue for record in batch: + if self.check_crcs and not record.validate_crc(): + raise Errors.CorruptRecordException( + "Record for partition %s at offset %s failed crc check" % ( + self.topic_partition, record.offset)) key_size = len(record.key) if record.key is not None else -1 value_size = len(record.value) if record.value is not None else -1 key = self._deserialize(key_deserializer, tp.topic, record.key) From 14643caba81e12c7dfe8663ab163679cedfbd5c5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 19:07:27 -0700 Subject: [PATCH 076/205] simplify consumer.poll send fetches logic --- kafka/consumer/group.py | 22 +++++++++------------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 3fccf4755..fa5b8ea8b 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -707,22 +707,18 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): # If data is available already, e.g. from a previous network client # poll() call to commit, then just return it immediately records, partial = self._fetcher.fetched_records(max_records, update_offsets=update_offsets) + # Before returning the fetched records, we can send off the + # next round of fetches and avoid block waiting for their + # responses to enable pipelining while the user is handling the + # fetched records. + if not partial: + futures = self._fetcher.send_fetches() + if len(futures): + self._client.poll(timeout_ms=0) + if records: - # Before returning the fetched records, we can send off the - # next round of fetches and avoid block waiting for their - # responses to enable pipelining while the user is handling the - # fetched records. - if not partial: - futures = self._fetcher.send_fetches() - if len(futures): - self._client.poll(timeout_ms=0) return records - # Send any new fetches (won't resend pending fetches) - futures = self._fetcher.send_fetches() - if len(futures): - self._client.poll(timeout_ms=0) - self._client.poll(timeout_ms=inner_timeout_ms(self._coordinator.time_to_next_poll() * 1000)) # after the long poll, we should check whether the group needs to rebalance # prior to returning data so that the group can stabilize faster From bea9d467e9d2ed92b8f64a1cbf91a5efa355c0e0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 22:55:45 -0700 Subject: [PATCH 077/205] Patch Release 2.1.2 --- CHANGES.md | 19 +++++++++++++++++++ docs/changelog.rst | 23 +++++++++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 43 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 78eab7769..e24c5c0aa 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,22 @@ +# 2.1.2 (Mar 17, 2025) + +Fixes +* Simplify consumer.poll send fetches logic +* Fix crc validation in consumer / fetcher +* Lazy `_unpack_records` in PartitionRecords to fix premature fetch offset advance in consumer.poll() (#2555) +* Debug log fetch records return; separate offsets update log +* Fix Fetcher retriable error handling (#2554) +* Use six.add_metaclass for py2/py3 compatible abc (#2551) + +Improvements +* Add FetchMetrics class; move topic_fetch_metrics inside aggregator +* DefaultRecordsBatchBuilder: support empty batch +* MemoryRecordsBuilder: support arbitrary offset, skipping offsets +* Add record.validate_crc() for v0/v1 crc checks +* Remove fetcher message_generator / iterator interface +* Add size_in_bytes to ABCRecordBatch and implement for Legacy and Default +* Add magic property to ABCRecord and implement for LegacyRecord + # 2.1.1 (Mar 16, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index a72ef9eae..4695c8dba 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,29 @@ Changelog ========= +2.1.2 (Mar 17, 2025) +#################### + +Fixes +----- +* Simplify consumer.poll send fetches logic +* Fix crc validation in consumer / fetcher +* Lazy `_unpack_records` in PartitionRecords to fix premature fetch offset advance in consumer.poll() (#2555) +* Debug log fetch records return; separate offsets update log +* Fix Fetcher retriable error handling (#2554) +* Use six.add_metaclass for py2/py3 compatible abc (#2551) + +Improvements +------------ +* Add FetchMetrics class; move topic_fetch_metrics inside aggregator +* DefaultRecordsBatchBuilder: support empty batch +* MemoryRecordsBuilder: support arbitrary offset, skipping offsets +* Add record.validate_crc() for v0/v1 crc checks +* Remove fetcher message_generator / iterator interface +* Add size_in_bytes to ABCRecordBatch and implement for Legacy and Default +* Add magic property to ABCRecord and implement for LegacyRecord + + 2.1.1 (Mar 16, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index d78c819ee..f81156126 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.2.dev' +__version__ = '2.1.2' From 39f2a6db7c84fe7cff3abe7446131f27547eb400 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 17 Mar 2025 23:01:51 -0700 Subject: [PATCH 078/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index f81156126..b0be43da3 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.2' +__version__ = '2.1.3.dev' From 9afaa4113ad897fed76805ec36eadfd1e1be825a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 18 Mar 2025 10:52:45 -0700 Subject: [PATCH 079/205] Return empty set from consumer.partitions_for_topic when topic not found (#2556) --- kafka/cluster.py | 1 + kafka/consumer/group.py | 2 +- kafka/coordinator/consumer.py | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index fd5abe852..9d2115859 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -112,6 +112,7 @@ def partitions_for_topic(self, topic): Returns: set: {partition (int), ...} + None if topic not found. """ if topic not in self._partitions: return None diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index fa5b8ea8b..71b295d49 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -634,7 +634,7 @@ def partitions_for_topic(self, topic): if partitions is None: self._fetch_all_topic_metadata() partitions = cluster.partitions_for_topic(topic) - return partitions + return partitions or set() def poll(self, timeout_ms=0, max_records=None, update_offsets=True): """Fetch data from assigned topics / partitions. diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 4c1b38644..3d180ca0c 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -203,8 +203,8 @@ def _auto_assign_all_partitions(self): def _build_metadata_snapshot(self, subscription, cluster): metadata_snapshot = {} for topic in subscription.group_subscription(): - partitions = cluster.partitions_for_topic(topic) or [] - metadata_snapshot[topic] = set(partitions) + partitions = cluster.partitions_for_topic(topic) + metadata_snapshot[topic] = partitions or set() return metadata_snapshot def _lookup_assignor(self, name): From e6fa9f3a93b05975f822a3c2730cb750c08cf23b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 19 Mar 2025 10:16:21 -0700 Subject: [PATCH 080/205] Add python 3.13 to compatibility list --- docs/compatibility.rst | 2 +- pyproject.toml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/compatibility.rst b/docs/compatibility.rst index 710542c40..772889334 100644 --- a/docs/compatibility.rst +++ b/docs/compatibility.rst @@ -16,6 +16,6 @@ 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! -kafka-python is tested on python 2.7, and 3.8-3.12. +kafka-python is tested on python 2.7, and 3.8-3.13. Builds and tests via Github Actions Workflows. See https://github.com/dpkp/kafka-python/actions diff --git a/pyproject.toml b/pyproject.toml index 87132defb..2a675c111 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -26,6 +26,7 @@ classifiers = [ "Programming Language :: Python :: 3.10", "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", + "Programming Language :: Python :: 3.13", "Programming Language :: Python :: Implementation :: CPython", "Programming Language :: Python :: Implementation :: PyPy", "Topic :: Software Development :: Libraries :: Python Modules", From 4995e9b91cbecd8cb940a5afcf157a90aac6610f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Mar 2025 09:48:48 -0700 Subject: [PATCH 081/205] KIP-511: Use ApiVersions v4 on initial connect w/ client_software_name + version (#2558) --- kafka/conn.py | 25 +++++++++++++++--- kafka/protocol/admin.py | 3 ++- kafka/protocol/api.py | 14 ++++++----- kafka/protocol/api_versions.py | 46 +++++++++++++++++++++++++++++++++- kafka/protocol/parser.py | 13 +++++----- 5 files changed, 83 insertions(+), 18 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index c94154885..f6af172f4 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -101,6 +101,10 @@ class BrokerConnection(object): server-side log entries that correspond to this client. Also submitted to GroupCoordinator for logging with respect to consumer group administration. Default: 'kafka-python-{version}' + client_software_name (str): Sent to kafka broker for KIP-511. + Default: 'kafka-python' + client_software_version (str): Sent to kafka broker for KIP-511. + Default: The kafka-python version (via kafka.version). reconnect_backoff_ms (int): The amount of time in milliseconds to wait before attempting to reconnect to a given host. Default: 50. @@ -191,6 +195,8 @@ class BrokerConnection(object): DEFAULT_CONFIG = { 'client_id': 'kafka-python-' + __version__, + 'client_software_name': 'kafka-python', + 'client_software_version': __version__, 'node_id': 0, 'request_timeout_ms': 30000, 'reconnect_backoff_ms': 50, @@ -242,7 +248,7 @@ def __init__(self, host, port, afi, **configs): self._api_versions = None self._api_version = None self._check_version_idx = None - self._api_versions_idx = 2 + self._api_versions_idx = 4 # version of ApiVersionsRequest to try on first connect self._throttle_time = None self._socks5_proxy = None @@ -538,7 +544,14 @@ def _try_api_versions_check(self): log.debug('%s: Using pre-configured api_version %s for ApiVersions', self, self._api_version) return True elif self._check_version_idx is None: - request = ApiVersionsRequest[self._api_versions_idx]() + version = self._api_versions_idx + if version >= 3: + request = ApiVersionsRequest[version]( + client_software_name=self.config['client_software_name'], + client_software_version=self.config['client_software_version'], + _tagged_fields={}) + else: + request = ApiVersionsRequest[version]() future = Future() response = self._send(request, blocking=True, request_timeout_ms=(self.config['api_version_auto_timeout_ms'] * 0.8)) response.add_callback(self._handle_api_versions_response, future) @@ -573,11 +586,15 @@ def _try_api_versions_check(self): def _handle_api_versions_response(self, future, response): error_type = Errors.for_code(response.error_code) - # if error_type i UNSUPPORTED_VERSION: retry w/ latest version from response if error_type is not Errors.NoError: future.failure(error_type()) if error_type is Errors.UnsupportedVersionError: self._api_versions_idx -= 1 + for api_key, min_version, max_version, *rest in response.api_versions: + # If broker provides a lower max_version, skip to that + if api_key == response.API_KEY: + self._api_versions_idx = min(self._api_versions_idx, max_version) + break if self._api_versions_idx >= 0: self._api_versions_future = None self.state = ConnectionStates.API_VERSIONS_SEND @@ -587,7 +604,7 @@ def _handle_api_versions_response(self, future, response): return self._api_versions = dict([ (api_key, (min_version, max_version)) - for api_key, min_version, max_version in response.api_versions + for api_key, min_version, max_version, *rest in response.api_versions ]) self._api_version = self._infer_broker_version_from_api_versions(self._api_versions) log.info('Broker version identified as %s', '.'.join(map(str, self._api_version))) diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py index 4ac3c18c8..255166801 100644 --- a/kafka/protocol/admin.py +++ b/kafka/protocol/admin.py @@ -737,7 +737,6 @@ class DescribeConfigsRequest_v2(Request): class DescribeLogDirsResponse_v0(Response): API_KEY = 35 API_VERSION = 0 - FLEXIBLE_VERSION = True SCHEMA = Schema( ('throttle_time_ms', Int32), ('log_dirs', Array( @@ -970,6 +969,7 @@ class AlterPartitionReassignmentsResponse_v0(Response): )), ("tags", TaggedFields) ) + FLEXIBLE_VERSION = True class AlterPartitionReassignmentsRequest_v0(Request): @@ -1017,6 +1017,7 @@ class ListPartitionReassignmentsResponse_v0(Response): )), ("tags", TaggedFields) ) + FLEXIBLE_VERSION = True class ListPartitionReassignmentsRequest_v0(Request): diff --git a/kafka/protocol/api.py b/kafka/protocol/api.py index 714da4d1d..9cd5767c1 100644 --- a/kafka/protocol/api.py +++ b/kafka/protocol/api.py @@ -82,19 +82,15 @@ def expect_response(self): def to_object(self): return _to_object(self.SCHEMA, self) - def build_request_header(self, correlation_id, client_id): + def build_header(self, correlation_id, client_id): if self.FLEXIBLE_VERSION: return RequestHeaderV2(self, correlation_id=correlation_id, client_id=client_id) return RequestHeader(self, correlation_id=correlation_id, client_id=client_id) - def parse_response_header(self, read_buffer): - if self.FLEXIBLE_VERSION: - return ResponseHeaderV2.decode(read_buffer) - return ResponseHeader.decode(read_buffer) - @add_metaclass(abc.ABCMeta) class Response(Struct): + FLEXIBLE_VERSION = False @abc.abstractproperty def API_KEY(self): @@ -114,6 +110,12 @@ def SCHEMA(self): def to_object(self): return _to_object(self.SCHEMA, self) + @classmethod + def parse_header(cls, read_buffer): + if cls.FLEXIBLE_VERSION: + return ResponseHeaderV2.decode(read_buffer) + return ResponseHeader.decode(read_buffer) + def _to_object(schema, data): obj = {} diff --git a/kafka/protocol/api_versions.py b/kafka/protocol/api_versions.py index 7e2e61251..e7cedd954 100644 --- a/kafka/protocol/api_versions.py +++ b/kafka/protocol/api_versions.py @@ -3,7 +3,7 @@ from io import BytesIO from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Int16, Int32, Schema +from kafka.protocol.types import Array, CompactArray, CompactString, Int16, Int32, Schema, TaggedFields class BaseApiVersionsResponse(Response): @@ -61,6 +61,28 @@ class ApiVersionsResponse_v2(BaseApiVersionsResponse): SCHEMA = ApiVersionsResponse_v1.SCHEMA +class ApiVersionsResponse_v3(BaseApiVersionsResponse): + API_KEY = 18 + API_VERSION = 3 + SCHEMA = Schema( + ('error_code', Int16), + ('api_versions', CompactArray( + ('api_key', Int16), + ('min_version', Int16), + ('max_version', Int16), + ('_tagged_fields', TaggedFields))), + ('throttle_time_ms', Int32), + ('_tagged_fields', TaggedFields) + ) + # Note: ApiVersions Response does not send FLEXIBLE_VERSION header! + + +class ApiVersionsResponse_v4(BaseApiVersionsResponse): + API_KEY = 18 + API_VERSION = 4 + SCHEMA = ApiVersionsResponse_v3.SCHEMA + + class ApiVersionsRequest_v0(Request): API_KEY = 18 API_VERSION = 0 @@ -82,9 +104,31 @@ class ApiVersionsRequest_v2(Request): SCHEMA = ApiVersionsRequest_v1.SCHEMA +class ApiVersionsRequest_v3(Request): + API_KEY = 18 + API_VERSION = 3 + RESPONSE_TYPE = ApiVersionsResponse_v3 + SCHEMA = Schema( + ('client_software_name', CompactString('utf-8')), + ('client_software_version', CompactString('utf-8')), + ('_tagged_fields', TaggedFields) + ) + FLEXIBLE_VERSION = True + + +class ApiVersionsRequest_v4(Request): + API_KEY = 18 + API_VERSION = 4 + RESPONSE_TYPE = ApiVersionsResponse_v4 + SCHEMA = ApiVersionsRequest_v3.SCHEMA + FLEXIBLE_VERSION = True + + ApiVersionsRequest = [ ApiVersionsRequest_v0, ApiVersionsRequest_v1, ApiVersionsRequest_v2, + ApiVersionsRequest_v3, ApiVersionsRequest_v4, ] ApiVersionsResponse = [ ApiVersionsResponse_v0, ApiVersionsResponse_v1, ApiVersionsResponse_v2, + ApiVersionsResponse_v3, ApiVersionsResponse_v4, ] diff --git a/kafka/protocol/parser.py b/kafka/protocol/parser.py index e7799fce6..4bc427330 100644 --- a/kafka/protocol/parser.py +++ b/kafka/protocol/parser.py @@ -59,7 +59,7 @@ def send_request(self, request, correlation_id=None): if correlation_id is None: correlation_id = self._next_correlation_id() - header = request.build_request_header(correlation_id=correlation_id, client_id=self._client_id) + header = request.build_header(correlation_id=correlation_id, client_id=self._client_id) message = b''.join([header.encode(), request.encode()]) size = Int32.encode(len(message)) data = size + message @@ -136,13 +136,14 @@ def _process_response(self, read_buffer): if not self.in_flight_requests: raise Errors.CorrelationIdError('No in-flight-request found for server response') (correlation_id, request) = self.in_flight_requests.popleft() - response_header = request.parse_response_header(read_buffer) + response_type = request.RESPONSE_TYPE + response_header = response_type.parse_header(read_buffer) recv_correlation_id = response_header.correlation_id log.debug('Received correlation id: %d', recv_correlation_id) # 0.8.2 quirk if (recv_correlation_id == 0 and correlation_id != 0 and - request.RESPONSE_TYPE is FindCoordinatorResponse[0] and + response_type is FindCoordinatorResponse[0] and (self._api_version == (0, 8, 2) or self._api_version is None)): log.warning('Kafka 0.8.2 quirk -- GroupCoordinatorResponse' ' Correlation ID does not match request. This' @@ -156,15 +157,15 @@ def _process_response(self, read_buffer): % (correlation_id, recv_correlation_id)) # decode response - log.debug('Processing response %s', request.RESPONSE_TYPE.__name__) + log.debug('Processing response %s', response_type.__name__) try: - response = request.RESPONSE_TYPE.decode(read_buffer) + response = response_type.decode(read_buffer) except ValueError: read_buffer.seek(0) buf = read_buffer.read() log.error('Response %d [ResponseType: %s Request: %s]:' ' Unable to decode %d-byte buffer: %r', - correlation_id, request.RESPONSE_TYPE, + correlation_id, response_type, request, len(buf), buf) raise Errors.KafkaProtocolError('Unable to decode response') From 3bd280a0ac71b8377f874b1fe7c45ee40935e263 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Mar 2025 13:09:38 -0700 Subject: [PATCH 082/205] Support KRaft / 4.0 brokers in tests (#2559) --- .github/workflows/python-package.yml | 1 + Makefile | 1 + kafka/protocol/broker_api_versions.py | 2 + servers/4.0.0/resources/kafka.properties | 161 +++++++++++++++++++++++ test/conftest.py | 26 ++-- test/fixtures.py | 103 +++++++++++---- test/test_sasl_integration.py | 2 +- 7 files changed, 260 insertions(+), 36 deletions(-) create mode 100644 servers/4.0.0/resources/kafka.properties diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 96df685f4..c9055f95a 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -33,6 +33,7 @@ jobs: - "3.0.2" - "3.5.2" - "3.9.0" + - "4.0.0" python: - "3.13" include: diff --git a/Makefile b/Makefile index b9f199ef0..c0128e7e2 100644 --- a/Makefile +++ b/Makefile @@ -68,6 +68,7 @@ kafka_scala_0_9_0_1=2.11 kafka_scala_0_10_0_0=2.11 kafka_scala_0_10_0_1=2.11 kafka_scala_0_10_1_0=2.11 +kafka_scala_4_0_0=2.13 scala_version=$(if $(SCALA_VERSION),$(SCALA_VERSION),$(if $(kafka_scala_$(subst .,_,$(1))),$(kafka_scala_$(subst .,_,$(1))),2.12)) kafka_artifact_name=kafka_$(call scala_version,$(1))-$(1).$(if $(filter 0.8.0,$(1)),tar.gz,tgz) diff --git a/kafka/protocol/broker_api_versions.py b/kafka/protocol/broker_api_versions.py index 299ab547a..af142d07c 100644 --- a/kafka/protocol/broker_api_versions.py +++ b/kafka/protocol/broker_api_versions.py @@ -63,4 +63,6 @@ (3, 9): {0: (0, 11), 1: (0, 17), 2: (0, 9), 3: (0, 12), 4: (0, 7), 5: (0, 4), 6: (0, 8), 7: (0, 3), 8: (0, 9), 9: (0, 9), 10: (0, 6), 11: (0, 9), 12: (0, 4), 13: (0, 5), 14: (0, 5), 15: (0, 5), 16: (0, 5), 17: (0, 1), 18: (0, 4), 19: (0, 7), 20: (0, 6), 21: (0, 2), 22: (0, 5), 23: (0, 4), 24: (0, 5), 25: (0, 4), 26: (0, 4), 27: (0, 1), 28: (0, 4), 29: (0, 3), 30: (0, 3), 31: (0, 3), 32: (0, 4), 33: (0, 2), 34: (0, 2), 35: (0, 4), 36: (0, 2), 37: (0, 3), 38: (0, 3), 39: (0, 2), 40: (0, 2), 41: (0, 3), 42: (0, 2), 43: (0, 2), 44: (0, 1), 45: (0, 0), 46: (0, 0), 47: (0, 0), 48: (0, 1), 49: (0, 1), 50: (0, 0), 51: (0, 0), 56: (0, 3), 57: (0, 1), 58: (0, 0), 60: (0, 1), 61: (0, 0), 65: (0, 0), 66: (0, 1), 67: (0, 0), 68: (0, 0), 69: (0, 0)}, + (4, 0): {0: (0, 12), 1: (4, 17), 2: (1, 10), 3: (0, 13), 8: (2, 9), 9: (1, 9), 10: (0, 6), 11: (2, 9), 12: (0, 4), 13: (0, 5), 14: (0, 5), 15: (0, 6), 16: (0, 5), 17: (0, 1), 18: (0, 4), 19: (2, 7), 20: (1, 6), 21: (0, 2), 22: (0, 5), 23: (2, 4), 24: (0, 5), 25: (0, 4), 26: (0, 5), 27: (1, 1), 28: (0, 5), 29: (1, 3), 30: (1, 3), 31: (1, 3), 32: (1, 4), 33: (0, 2), 34: (1, 2), 35: (1, 4), 36: (0, 2), 37: (0, 3), 38: (1, 3), 39: (1, 2), 40: (1, 2), 41: (1, 3), 42: (0, 2), 43: (0, 2), 44: (0, 1), 45: (0, 0), 46: (0, 0), 47: (0, 0), 48: (0, 1), 49: (0, 1), 50: (0, 0), 51: (0, 0), 55: (0, 2), 57: (0, 2), 60: (0, 2), 61: (0, 0), 64: (0, 0), 65: (0, 0), 66: (0, 1), 68: (0, 1), 69: (0, 1), 74: (0, 0), 75: (0, 0), 80: (0, 0), 81: (0, 0)}, + } diff --git a/servers/4.0.0/resources/kafka.properties b/servers/4.0.0/resources/kafka.properties new file mode 100644 index 000000000..3dba393ba --- /dev/null +++ b/servers/4.0.0/resources/kafka.properties @@ -0,0 +1,161 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +############################# Server Basics ############################# + +# The role of this server. Setting this puts us in KRaft mode +process.roles=broker,controller + +# The node id associated with this instance's roles +node.id={broker_id} + +# List of controller endpoints used connect to the controller cluster +controller.quorum.bootstrap.servers={controller_bootstrap_host}:{controller_port} + +############################# Socket Server Settings ############################# + +# The address the socket server listens on. +# Combined nodes (i.e. those with `process.roles=broker,controller`) must list the controller listener here at a minimum. +# If the broker listener is not defined, the default listener will use a host name that is equal to the value of java.net.InetAddress.getCanonicalHostName(), +# with PLAINTEXT listener name, and port 9092. +# FORMAT: +# listeners = listener_name://host_name:port +# EXAMPLE: +# listeners = PLAINTEXT://your.host.name:9092 +#listeners=PLAINTEXT://:9092,CONTROLLER://:9093 +listeners={transport}://{host}:{port},CONTROLLER://{host}:{controller_port} + +# Name of listener used for communication between brokers. +inter.broker.listener.name={transport} + +{sasl_config} + +authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer +allow.everyone.if.no.acl.found=true + +# Listener name, hostname and port the broker or the controller will advertise to clients. +# If not set, it uses the value for "listeners". +advertised.listeners={transport}://{host}:{port},CONTROLLER://{host}:{controller_port} + +# A comma-separated list of the names of the listeners used by the controller. +# If no explicit mapping set in `listener.security.protocol.map`, default will be using PLAINTEXT protocol +# This is required if running in KRaft mode. +controller.listener.names=CONTROLLER + +# Maps listener names to security protocols, the default is for them to be the same. See the config documentation for more details +listener.security.protocol.map=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT,SSL:SSL,SASL_PLAINTEXT:SASL_PLAINTEXT,SASL_SSL:SASL_SSL + +# The number of threads that the server uses for receiving requests from the network and sending responses to the network +num.network.threads=3 + +# The number of threads that the server uses for processing requests, which may include disk I/O +num.io.threads=8 + +# The send buffer (SO_SNDBUF) used by the socket server +socket.send.buffer.bytes=102400 + +# The receive buffer (SO_RCVBUF) used by the socket server +socket.receive.buffer.bytes=102400 + +# The maximum size of a request that the socket server will accept (protection against OOM) +socket.request.max.bytes=104857600 + + +############################# Log Basics ############################# + +# A comma separated list of directories under which to store log files +log.dirs={tmp_dir}/kraft-combined-logs + +# The default number of log partitions per topic. More partitions allow greater +# parallelism for consumption, but this will also result in more files across +# the brokers. +num.partitions={partitions} +default.replication.factor={replicas} + +## Short Replica Lag -- Drops failed brokers out of ISR +replica.lag.time.max.ms=1000 +replica.socket.timeout.ms=1000 + +# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. +# This value is recommended to be increased for installations with data dirs located in RAID array. +num.recovery.threads.per.data.dir=1 + +############################# Internal Topic Settings ############################# +# The replication factor for the group metadata internal topics "__consumer_offsets", "__share_group_state" and "__transaction_state" +# For anything other than development testing, a value greater than 1 is recommended to ensure availability such as 3. +offsets.topic.replication.factor=1 +share.coordinator.state.topic.replication.factor=1 +share.coordinator.state.topic.min.isr=1 +transaction.state.log.replication.factor=1 +transaction.state.log.min.isr=1 + +############################# Log Flush Policy ############################# + +# Messages are immediately written to the filesystem but by default we only fsync() to sync +# the OS cache lazily. The following configurations control the flush of data to disk. +# There are a few important trade-offs here: +# 1. Durability: Unflushed data may be lost if you are not using replication. +# 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush. +# 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to excessive seeks. +# The settings below allow one to configure the flush policy to flush data after a period of time or +# every N messages (or both). This can be done globally and overridden on a per-topic basis. + +# The number of messages to accept before forcing a flush of data to disk +#log.flush.interval.messages=10000 + +# The maximum amount of time a message can sit in a log before we force a flush +#log.flush.interval.ms=1000 + +############################# Log Retention Policy ############################# + +# The following configurations control the disposal of log segments. The policy can +# be set to delete segments after a period of time, or after a given size has accumulated. +# A segment will be deleted whenever *either* of these criteria are met. Deletion always happens +# from the end of the log. + +# The minimum age of a log file to be eligible for deletion due to age +log.retention.hours=168 + +# A size-based retention policy for logs. Segments are pruned from the log unless the remaining +# segments drop below log.retention.bytes. Functions independently of log.retention.hours. +#log.retention.bytes=1073741824 + +# The maximum size of a log segment file. When this size is reached a new log segment will be created. +log.segment.bytes=1073741824 + +# The interval at which log segments are checked to see if they can be deleted according +# to the retention policies +log.retention.check.interval.ms=300000 + +# By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires. +# If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction. +log.cleaner.enable=false + +# tune down offset topics to reduce setup time in tests +offsets.commit.timeout.ms=500 +offsets.topic.num.partitions=2 +offsets.topic.replication.factor=1 + +# Allow shorter session timeouts for tests +group.min.session.timeout.ms=1000 + +############################# Group Coordinator Settings ############################# + +# The following configuration specifies the time, in milliseconds, that the GroupCoordinator will delay the initial consumer rebalance. +# The rebalance will be further delayed by the value of group.initial.rebalance.delay.ms as new members join the group, up to a maximum of max.poll.interval.ms. +# The default value for this is 3 seconds. +# We override this to 0 here as it makes for a better out-of-the-box experience for development and testing. +# However, in production environments the default value of 3 seconds is more suitable as this will help to avoid unnecessary, and potentially expensive, rebalances during application startup. +group.initial.rebalance.delay.ms=0 diff --git a/test/conftest.py b/test/conftest.py index ddd491517..4c4c503e7 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -9,6 +9,7 @@ from test.testutil import env_kafka_version, random_string from test.fixtures import KafkaFixture, ZookeeperFixture + @pytest.fixture(scope="module") def zookeeper(): """Return a Zookeeper fixture""" @@ -23,18 +24,18 @@ def zookeeper(): @pytest.fixture(scope="module") -def kafka_broker(kafka_broker_factory, zookeeper): +def kafka_broker(kafka_broker_factory): """Return a Kafka broker fixture""" if "KAFKA_URI" in os.environ: parse = urlparse(os.environ["KAFKA_URI"]) (host, port) = (parse.hostname, parse.port) - return KafkaFixture.instance(0, zookeeper, host=host, port=port, external=True) + return KafkaFixture.instance(0, host=host, port=port, external=True) else: - return kafka_broker_factory()[0] + return kafka_broker_factory() @pytest.fixture(scope="module") -def kafka_broker_factory(zookeeper): +def kafka_broker_factory(): """Return a Kafka broker fixture factory""" assert env_kafka_version(), 'KAFKA_VERSION must be specified to run integration tests' @@ -42,16 +43,20 @@ def kafka_broker_factory(zookeeper): def factory(**broker_params): params = {} if broker_params is None else broker_params.copy() params.setdefault('partitions', 4) - num_brokers = params.pop('num_brokers', 1) - brokers = tuple(KafkaFixture.instance(x, zookeeper, **params) - for x in range(num_brokers)) - _brokers.extend(brokers) - return brokers + node_id = params.pop('node_id', 0) + broker = KafkaFixture.instance(node_id, **params) + _brokers.append(broker) + return broker yield factory + zks = set() for broker in _brokers: + zks.add(broker.zookeeper) broker.close() + for zk in zks: + if zk: + zk.close() @pytest.fixture @@ -108,11 +113,13 @@ def factory(**kafka_producer_params): if _producer[0]: _producer[0].close() + @pytest.fixture def kafka_admin_client(kafka_admin_client_factory): """Return a KafkaAdminClient fixture""" yield kafka_admin_client_factory() + @pytest.fixture def kafka_admin_client_factory(kafka_broker): """Return a KafkaAdminClient factory fixture""" @@ -128,6 +135,7 @@ def factory(**kafka_admin_client_params): if _admin_client[0]: _admin_client[0].close() + @pytest.fixture def topic(kafka_broker, request): """Return a topic fixture""" diff --git a/test/fixtures.py b/test/fixtures.py index 9843d5a2b..dc41cc8e3 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -1,6 +1,7 @@ from __future__ import absolute_import, division import atexit +import base64 import logging import os import os.path @@ -11,6 +12,7 @@ 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 @@ -220,17 +222,25 @@ class KafkaFixture(Fixture): broker_password = 'alice-secret' @classmethod - def instance(cls, broker_id, zookeeper, zk_chroot=None, - host=None, port=None, external=False, + def instance(cls, broker_id, zookeeper=None, zk_chroot=None, + host="localhost", port=None, external=False, transport='PLAINTEXT', replicas=1, partitions=4, sasl_mechanism=None, auto_create_topic=True, tmp_dir=None): - if zk_chroot is None: - zk_chroot = "kafka-python_" + str(uuid.uuid4()).replace("-", "_") - if host is None: - host = "localhost" + # Kafka requries zookeeper prior to 4.0 release + if env_kafka_version() < (4, 0): + if zookeeper is None: + if "ZOOKEEPER_URI" in os.environ: + parse = urlparse(os.environ["ZOOKEEPER_URI"]) + (host, port) = (parse.hostname, parse.port) + zookeeper = ZookeeperFixture.instance(host=host, port=port, external=True) + elif not external: + zookeeper = ZookeeperFixture.instance() + if zk_chroot is None: + zk_chroot = "kafka-python_" + str(uuid.uuid4()).replace("-", "_") + fixture = KafkaFixture(host, port, broker_id, - zookeeper, zk_chroot, + zookeeper=zookeeper, zk_chroot=zk_chroot, external=external, transport=transport, replicas=replicas, partitions=partitions, @@ -241,15 +251,23 @@ def instance(cls, broker_id, zookeeper, zk_chroot=None, fixture.open() return fixture - def __init__(self, host, port, broker_id, zookeeper, zk_chroot, + def __init__(self, host, port, broker_id, zookeeper=None, zk_chroot=None, replicas=1, partitions=2, transport='PLAINTEXT', sasl_mechanism=None, auto_create_topic=True, tmp_dir=None, external=False): super(KafkaFixture, self).__init__() self.host = host - self.port = port + self.controller_bootstrap_host = host + if port is None: + self.auto_port = True + self.port = get_open_port() + else: + self.auto_port = False + self.port = port + self.controller_port = self.port + 1 + self.cluster_id = self._gen_cluster_id() self.broker_id = broker_id self.auto_create_topic = auto_create_topic self.transport = transport.upper() @@ -262,15 +280,19 @@ def __init__(self, host, port, broker_id, zookeeper, zk_chroot, # TODO: checking for port connection would be better than scanning logs # until then, we need the pattern to work across all supported broker versions # The logging format changed slightly in 1.0.0 - self.start_pattern = r"\[Kafka ?Server (id=)?%d\],? started" % (broker_id,) - # Need to wait until the broker has fetched user configs from zookeeper in case we use scram as sasl mechanism - self.scram_pattern = r"Removing Produce quota for user %s" % (self.broker_user) + if env_kafka_version() < (4, 0): + self.start_pattern = r"\[Kafka ?Server (id=)?%d\],? started" % (broker_id,) + # Need to wait until the broker has fetched user configs from zookeeper in case we use scram as sasl mechanism + self.scram_pattern = r"Removing Produce quota for user %s" % (self.broker_user) + else: + self.start_pattern = r"\[KafkaRaftServer nodeId=%d\] Kafka Server started" % (broker_id,) + self.scram_pattern = r"Replayed UserScramCredentialRecord creating new entry for %s" % (self.broker_user,) self.zookeeper = zookeeper self.zk_chroot = zk_chroot # Add the attributes below for the template binding - self.zk_host = self.zookeeper.host - self.zk_port = self.zookeeper.port + self.zk_host = self.zookeeper.host if self.zookeeper else None + self.zk_port = self.zookeeper.port if self.zookeeper else None self.replicas = replicas self.partitions = partitions @@ -289,6 +311,9 @@ def __init__(self, host, port, broker_id, zookeeper, zk_chroot, self.sasl_config = '' self.jaas_config = '' + def _gen_cluster_id(self): + return base64.b64encode(uuid.uuid4().bytes).decode('utf-8').rstrip('=') + def _sasl_config(self): if not self.sasl_enabled: return '' @@ -400,12 +425,11 @@ def start(self): backoff = 1 end_at = time.time() + max_timeout tries = 1 - auto_port = (self.port is None) while time.time() < end_at: # We have had problems with port conflicts on travis # so we will try a different port on each retry # unless the fixture was passed a specific port - if auto_port: + if self.auto_port: self.port = get_open_port() self.out('Attempting to start on port %d (try #%d)' % (self.port, tries)) self.render_template(properties_template, properties, vars(self)) @@ -451,6 +475,9 @@ def open(self): self.tmp_dir.ensure(dir=True) self.tmp_dir.ensure('logs', dir=True) self.tmp_dir.ensure('data', dir=True) + properties = self.tmp_dir.join('kafka.properties') + properties_template = self.test_resource('kafka.properties') + self.render_template(properties_template, properties, vars(self)) self.out("Running local instance...") log.info(" host = %s", self.host) @@ -458,19 +485,26 @@ def open(self): log.info(" transport = %s", self.transport) log.info(" sasl_mechanism = %s", self.sasl_mechanism) log.info(" broker_id = %s", self.broker_id) - log.info(" zk_host = %s", self.zookeeper.host) - log.info(" zk_port = %s", self.zookeeper.port) + log.info(" zk_host = %s", self.zk_host) + log.info(" zk_port = %s", self.zk_port) log.info(" zk_chroot = %s", self.zk_chroot) log.info(" replicas = %s", self.replicas) log.info(" partitions = %s", self.partitions) log.info(" tmp_dir = %s", self.tmp_dir.strpath) - self._create_zk_chroot() + if self.zookeeper: + if self.zk_chroot: + self._create_zk_chroot() + # add user to zookeeper for the first server + if self.sasl_enabled and self.sasl_mechanism.startswith("SCRAM-SHA") and self.broker_id == 0: + self._add_scram_user() + + else: + # running in KRaft mode + self._format_log_dirs() + self.sasl_config = self._sasl_config() self.jaas_config = self._jaas_config() - # add user to zookeeper for the first server - if self.sasl_enabled and self.sasl_mechanism.startswith("SCRAM-SHA") and self.broker_id == 0: - self._add_scram_user() self.start() atexit.register(self.close) @@ -502,6 +536,21 @@ def dump_logs(self): super(KafkaFixture, self).dump_logs() self.zookeeper.dump_logs() + def _format_log_dirs(self): + self.out("Formatting log dirs for kraft bootstrapping") + args = self.run_script('kafka-storage.sh', 'format', '--standalone', '-t', self.cluster_id, '-c', self.tmp_dir.join("kafka.properties")) + if self.sasl_enabled and self.sasl_mechanism.startswith("SCRAM-SHA"): + args.extend(['--add-scram', '{}=[name={},password={}]'.format(self.sasl_mechanism, self.broker_user, self.broker_password)]) + env = self.kafka_run_class_env() + proc = subprocess.Popen(args, env=env, stdout=subprocess.PIPE, stderr=subprocess.PIPE) + stdout, stderr = proc.communicate() + if proc.returncode != 0: + self.out("Failed to format log dirs for kraft bootstrap!") + self.out(stdout) + self.out(stderr) + raise RuntimeError("Failed to format log dirs!") + return True + def _send_request(self, request, timeout=None): def _failure(error): raise error @@ -541,8 +590,9 @@ def _create_topic(self, topic_name, num_partitions=None, replication_factor=None # Try different methods to create a topic, from the fastest to the slowest if self.auto_create_topic and num_partitions == self.partitions and replication_factor == self.replicas: self._create_topic_via_metadata(topic_name, timeout_ms) - elif env_kafka_version() >= (0, 10, 1, 0): + elif env_kafka_version() >= (0, 10, 1, 0) and env_kafka_version() < (4, 0): try: + # 4.0 brokers dropped support for CreateTopicsRequest v0 (TODO: pick from api_versions) self._create_topic_via_admin_api(topic_name, num_partitions, replication_factor, timeout_ms) except InvalidReplicationFactorError: # wait and try again @@ -686,8 +736,8 @@ def get_api_versions(): def run_brokers(): logging.basicConfig(level=logging.ERROR) - zk = ZookeeperFixture.instance() - k = KafkaFixture.instance(0, zk) + k = KafkaFixture.instance(0) + zk = k.zookeeper print("Kafka", k.kafka_version, "running on port:", k.port) try: @@ -696,7 +746,8 @@ def run_brokers(): except KeyboardInterrupt: print("Bye!") k.close() - zk.close() + if zk: + zk.close() if __name__ == '__main__': diff --git a/test/test_sasl_integration.py b/test/test_sasl_integration.py index 0f404da20..69323fb92 100644 --- a/test/test_sasl_integration.py +++ b/test/test_sasl_integration.py @@ -25,7 +25,7 @@ ] ) def sasl_kafka(request, kafka_broker_factory): - sasl_kafka = kafka_broker_factory(transport="SASL_PLAINTEXT", sasl_mechanism=request.param)[0] + sasl_kafka = kafka_broker_factory(transport="SASL_PLAINTEXT", sasl_mechanism=request.param) yield sasl_kafka sasl_kafka.child.dump_logs() From 301828a5cca095e60665c3af2ec0fa54b4ca20f9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Mar 2025 13:11:33 -0700 Subject: [PATCH 083/205] Update kafka broker compatibility docs --- docs/compatibility.rst | 4 ++-- docs/index.rst | 2 +- docs/tests.rst | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/compatibility.rst b/docs/compatibility.rst index 772889334..353273114 100644 --- a/docs/compatibility.rst +++ b/docs/compatibility.rst @@ -1,12 +1,12 @@ Compatibility ------------- -.. image:: https://img.shields.io/badge/kafka-3.9--0.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-4.0--0.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python -kafka-python is compatible with (and tested against) broker versions 3.9 +kafka-python is compatible with (and tested against) broker versions 4.0 through 0.8.0 . kafka-python is not compatible with the 0.8.2-beta release. Because the kafka server protocol is backwards compatible, kafka-python is diff --git a/docs/index.rst b/docs/index.rst index 5dd4f183a..471a234f0 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -1,7 +1,7 @@ kafka-python ############ -.. image:: https://img.shields.io/badge/kafka-3.9--0.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-4.0--0.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/en/master/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python diff --git a/docs/tests.rst b/docs/tests.rst index 988afca65..c8adb2d76 100644 --- a/docs/tests.rst +++ b/docs/tests.rst @@ -45,7 +45,7 @@ Integration tests .. code:: bash - KAFKA_VERSION=3.9.0 make test + KAFKA_VERSION=4.0.0 make test Integration tests start Kafka and Zookeeper fixtures. Make will download From 3f3314cba1cfd5b80fab375843e14b1c47b80264 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Mar 2025 13:19:05 -0700 Subject: [PATCH 084/205] Use get_open_port for KRaft controller port --- test/fixtures.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/fixtures.py b/test/fixtures.py index dc41cc8e3..3adb87a97 100644 --- a/test/fixtures.py +++ b/test/fixtures.py @@ -265,7 +265,7 @@ def __init__(self, host, port, broker_id, zookeeper=None, zk_chroot=None, else: self.auto_port = False self.port = port - self.controller_port = self.port + 1 + self.controller_port = get_open_port() self.cluster_id = self._gen_cluster_id() self.broker_id = broker_id From cd4830afada51418e6d3fe4e998f08d7bdac69c1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 21 Mar 2025 13:22:44 -0700 Subject: [PATCH 085/205] Test older pythons against 4.0 broker --- .github/workflows/python-package.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index c9055f95a..df790120a 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -41,15 +41,15 @@ jobs: # kafka: "2.6.0" # experimental: true - python: "3.8" - kafka: "3.9.0" + kafka: "4.0.0" - python: "3.9" - kafka: "3.9.0" + kafka: "4.0.0" - python: "3.10" - kafka: "3.9.0" + kafka: "4.0.0" - python: "3.11" - kafka: "3.9.0" + kafka: "4.0.0" - python: "3.12" - kafka: "3.9.0" + kafka: "4.0.0" steps: - uses: actions/checkout@v4 From d214321705efde740491847a4f827c537cfdcfbc Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 09:55:08 -0700 Subject: [PATCH 086/205] Move ensure_valid_topic_name to kafka.util; use in client and producer (#2561) --- kafka/client_async.py | 8 +++++- kafka/consumer/subscription_state.py | 25 ++----------------- kafka/producer/kafka.py | 5 ++++ kafka/util.py | 24 ++++++++++++++++++ ...est_subscription_state.py => test_util.py} | 5 ++-- 5 files changed, 40 insertions(+), 27 deletions(-) rename test/{test_subscription_state.py => test_util.py} (83%) diff --git a/kafka/client_async.py b/kafka/client_async.py index 7121ce7a7..4de05b33e 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -27,7 +27,7 @@ from kafka.metrics.stats.rate import TimeUnit from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.metadata import MetadataRequest -from kafka.util import Dict, WeakMethod +from kafka.util import Dict, 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 @@ -909,7 +909,13 @@ def add_topic(self, topic): Returns: Future: resolves after metadata request/response + + Raises: + TypeError: if topic is not a string + ValueError: if topic is invalid: must be chars (a-zA-Z0-9._-), and less than 250 length """ + ensure_valid_topic_name(topic) + if topic in self._topics: return Future().success(set(self._topics)) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index abe37fb86..2b2bcb477 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -9,6 +9,7 @@ from kafka.errors import IllegalStateError from kafka.protocol.list_offsets import OffsetResetStrategy from kafka.structs import OffsetAndMetadata +from kafka.util import ensure_valid_topic_name log = logging.getLogger(__name__) @@ -43,10 +44,6 @@ class SubscriptionState(object): " (2) subscribe to topics matching a regex pattern," " (3) assign itself specific topic-partitions.") - # Taken from: https://github.com/apache/kafka/blob/39eb31feaeebfb184d98cc5d94da9148c2319d81/clients/src/main/java/org/apache/kafka/common/internals/Topic.java#L29 - _MAX_NAME_LENGTH = 249 - _TOPIC_LEGAL_CHARS = re.compile('^[a-zA-Z0-9._-]+$') - def __init__(self, offset_reset_strategy='earliest'): """Initialize a SubscriptionState instance @@ -123,24 +120,6 @@ def subscribe(self, topics=(), pattern=None, listener=None): raise TypeError('listener must be a ConsumerRebalanceListener') self.listener = listener - def _ensure_valid_topic_name(self, topic): - """ Ensures that the topic name is valid according to the kafka source. """ - - # See Kafka Source: - # 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): - raise TypeError('All topics must be strings') - if len(topic) == 0: - raise ValueError('All topics must be non-empty strings') - if topic == '.' or topic == '..': - raise ValueError('Topic name cannot be "." or ".."') - if len(topic) > self._MAX_NAME_LENGTH: - raise ValueError('Topic name is illegal, it can\'t be longer than {0} characters, topic: "{1}"'.format(self._MAX_NAME_LENGTH, topic)) - if not self._TOPIC_LEGAL_CHARS.match(topic): - raise ValueError('Topic name "{0}" is illegal, it contains a character other than ASCII alphanumerics, ".", "_" and "-"'.format(topic)) - def change_subscription(self, topics): """Change the topic subscription. @@ -166,7 +145,7 @@ def change_subscription(self, topics): return for t in topics: - self._ensure_valid_topic_name(t) + ensure_valid_topic_name(t) log.info('Updating subscribed topics to: %s', topics) self.subscription = set(topics) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index b8ace0fc1..8da14af1c 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -22,6 +22,7 @@ from kafka.record.legacy_records import LegacyRecordBatchBuilder from kafka.serializer import Serializer from kafka.structs import TopicPartition +from kafka.util import ensure_valid_topic_name log = logging.getLogger(__name__) @@ -593,11 +594,15 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest Raises: KafkaTimeoutError: if unable to fetch topic metadata, or unable to obtain memory buffer prior to configured max_block_ms + TypeError: if topic is not a string + ValueError: if topic is invalid: must be chars (a-zA-Z0-9._-), and less than 250 length + AssertionError: if KafkaProducer is closed, or key and value are both None """ assert not self._closed, 'KafkaProducer already closed!' assert value is not None or self.config['api_version'] >= (0, 8, 1), ( 'Null messages require kafka >= 0.8.1') assert not (value is None and key is None), 'Need at least one: key or value' + ensure_valid_topic_name(topic) key_bytes = value_bytes = None try: assigned_partition = None diff --git a/kafka/util.py b/kafka/util.py index d067a063d..470200b1b 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -1,6 +1,7 @@ from __future__ import absolute_import import binascii +import re import time import weakref @@ -43,6 +44,29 @@ def inner_timeout_ms(fallback=None): return inner_timeout_ms +# Taken from: https://github.com/apache/kafka/blob/39eb31feaeebfb184d98cc5d94da9148c2319d81/clients/src/main/java/org/apache/kafka/common/internals/Topic.java#L29 +TOPIC_MAX_LENGTH = 249 +TOPIC_LEGAL_CHARS = re.compile('^[a-zA-Z0-9._-]+$') + +def ensure_valid_topic_name(topic): + """ Ensures that the topic name is valid according to the kafka source. """ + + # See Kafka Source: + # 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): + raise TypeError('All topics must be strings') + if len(topic) == 0: + raise ValueError('All topics must be non-empty strings') + if topic == '.' or topic == '..': + raise ValueError('Topic name cannot be "." or ".."') + if len(topic) > TOPIC_MAX_LENGTH: + raise ValueError('Topic name is illegal, it can\'t be longer than {0} characters, topic: "{1}"'.format(TOPIC_MAX_LENGTH, topic)) + if not TOPIC_LEGAL_CHARS.match(topic): + raise ValueError('Topic name "{0}" is illegal, it contains a character other than ASCII alphanumerics, ".", "_" and "-"'.format(topic)) + + class WeakMethod(object): """ Callable that weakly references a method and the object it is bound to. It diff --git a/test/test_subscription_state.py b/test/test_util.py similarity index 83% rename from test/test_subscription_state.py rename to test/test_util.py index 9718f6af4..875b252aa 100644 --- a/test/test_subscription_state.py +++ b/test/test_util.py @@ -3,7 +3,7 @@ import pytest -from kafka.consumer.subscription_state import SubscriptionState +from kafka.util import ensure_valid_topic_name @pytest.mark.parametrize(('topic_name', 'expectation'), [ (0, pytest.raises(TypeError)), @@ -20,6 +20,5 @@ ('name+with+plus', pytest.raises(ValueError)), ]) def test_topic_name_validation(topic_name, expectation): - state = SubscriptionState() with expectation: - state._ensure_valid_topic_name(topic_name) + ensure_valid_topic_name(topic_name) From 0720a523184cddd6e9484c7619acf08d33d107aa Mon Sep 17 00:00:00 2001 From: Guillaume Arnaud Date: Mon, 24 Mar 2025 18:00:36 +0100 Subject: [PATCH 087/205] Fix maximum version to send an OffsetFetchRequest (#2563) Co-authored-by: Guillaume Arnaud --- kafka/admin/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 171304da0..392687be5 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -1496,7 +1496,7 @@ def _list_consumer_group_offsets_send_request(self, group_id, A message future """ version = self._client.api_version(OffsetFetchRequest, max_version=5) - if version <= 3: + if version <= 5: if partitions is None: if version <= 1: raise ValueError( From 3f3c42478f1a0f59578896509ee320f37ddbe847 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 11:14:54 -0700 Subject: [PATCH 088/205] KIP-70: Auto-commit offsets on consumer.unsubscribe(), defer assignment changes to rejoin (#2560) --- kafka/consumer/group.py | 16 +++++-- kafka/consumer/subscription_state.py | 11 ++--- kafka/coordinator/consumer.py | 11 ++++- test/test_consumer.py | 64 +++++++++++++++++++--------- test/test_fetcher.py | 2 +- test/test_subscription_state.py | 57 +++++++++++++++++++++++++ 6 files changed, 131 insertions(+), 30 deletions(-) create mode 100644 test/test_subscription_state.py diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 71b295d49..ee3f95be7 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -444,8 +444,15 @@ def assign(self, partitions): no rebalance operation triggered when group membership or cluster and topic metadata change. """ - self._subscription.assign_from_user(partitions) - self._client.set_topics([tp.topic for tp in partitions]) + if not partitions: + self.unsubscribe() + else: + # make sure the offsets of topic partitions the consumer is unsubscribing from + # are committed since there will be no following rebalance + self._coordinator.maybe_auto_commit_offsets_now() + self._subscription.assign_from_user(partitions) + self._client.set_topics([tp.topic for tp in partitions]) + log.debug("Subscribed to partition(s): %s", partitions) def assignment(self): """Get the TopicPartitions currently assigned to this consumer. @@ -959,8 +966,11 @@ def subscription(self): def unsubscribe(self): """Unsubscribe from all topics and clear all assigned partitions.""" + # make sure the offsets of topic partitions the consumer is unsubscribing from + # are committed since there will be no following rebalance + self._coordinator.maybe_auto_commit_offsets_now() self._subscription.unsubscribe() - self._coordinator.close() + self._coordinator.maybe_leave_group() self._client.cluster.need_all_topic_metadata = False self._client.set_topics([]) log.debug("Unsubscribed all topics or patterns and assigned partitions") diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 2b2bcb477..a1675c724 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -1,6 +1,10 @@ from __future__ import absolute_import import abc +try: + from collections import Sequence +except ImportError: + from collections.abc import Sequence import logging import re @@ -114,6 +118,8 @@ def subscribe(self, topics=(), pattern=None, listener=None): self.subscription = set() self.subscribed_pattern = re.compile(pattern) else: + if isinstance(topics, str) or not isinstance(topics, Sequence): + raise TypeError('Topics must be a list (or non-str sequence)') self.change_subscription(topics) if listener and not isinstance(listener, ConsumerRebalanceListener): @@ -151,11 +157,6 @@ def change_subscription(self, topics): self.subscription = set(topics) self._group_subscription.update(topics) - # Remove any assigned partitions which are no longer subscribed to - for tp in set(self.assignment.keys()): - if tp.topic not in self.subscription: - del self.assignment[tp] - def group_subscribe(self, topics): """Add topics to the current group subscription. diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 3d180ca0c..f086b0fd7 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -878,8 +878,15 @@ def _maybe_auto_commit_offsets_async(self): self.next_auto_commit_deadline = time.time() + self.config['retry_backoff_ms'] / 1000 elif time.time() > self.next_auto_commit_deadline: self.next_auto_commit_deadline = time.time() + self.auto_commit_interval - self.commit_offsets_async(self._subscription.all_consumed_offsets(), - self._commit_offsets_async_on_complete) + self._do_auto_commit_offsets_async() + + def maybe_auto_commit_offsets_now(self): + if self.config['enable_auto_commit'] and not self.coordinator_unknown(): + self._do_auto_commit_offsets_async() + + def _do_auto_commit_offsets_async(self): + self.commit_offsets_async(self._subscription.all_consumed_offsets(), + self._commit_offsets_async_on_complete) class ConsumerCoordinatorMetrics(object): diff --git a/test/test_consumer.py b/test/test_consumer.py index 8186125df..0d9477729 100644 --- a/test/test_consumer.py +++ b/test/test_consumer.py @@ -1,26 +1,52 @@ +from __future__ import absolute_import + import pytest -from kafka import KafkaConsumer -from kafka.errors import KafkaConfigurationError +from kafka import KafkaConsumer, TopicPartition +from kafka.errors import KafkaConfigurationError, IllegalStateError + + +def test_session_timeout_larger_than_request_timeout_raises(): + with pytest.raises(KafkaConfigurationError): + KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), group_id='foo', session_timeout_ms=50000, request_timeout_ms=40000) + + +def test_fetch_max_wait_larger_than_request_timeout_raises(): + with pytest.raises(KafkaConfigurationError): + KafkaConsumer(bootstrap_servers='localhost:9092', fetch_max_wait_ms=50000, request_timeout_ms=40000) + + +def test_request_timeout_larger_than_connections_max_idle_ms_raises(): + with pytest.raises(KafkaConfigurationError): + KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), request_timeout_ms=50000, connections_max_idle_ms=40000) + +def test_subscription_copy(): + consumer = KafkaConsumer('foo', api_version=(0, 10, 0)) + sub = consumer.subscription() + assert sub is not consumer.subscription() + assert sub == set(['foo']) + sub.add('fizz') + assert consumer.subscription() == set(['foo']) -class TestKafkaConsumer: - def test_session_timeout_larger_than_request_timeout_raises(self): - with pytest.raises(KafkaConfigurationError): - KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), group_id='foo', session_timeout_ms=50000, request_timeout_ms=40000) - def test_fetch_max_wait_larger_than_request_timeout_raises(self): - with pytest.raises(KafkaConfigurationError): - KafkaConsumer(bootstrap_servers='localhost:9092', fetch_max_wait_ms=50000, request_timeout_ms=40000) +def test_assign(): + # Consumer w/ subscription to topic 'foo' + consumer = KafkaConsumer('foo', api_version=(0, 10, 0)) + assert consumer.assignment() == set() + # Cannot assign manually + with pytest.raises(IllegalStateError): + consumer.assign([TopicPartition('foo', 0)]) - def test_request_timeout_larger_than_connections_max_idle_ms_raises(self): - with pytest.raises(KafkaConfigurationError): - KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), request_timeout_ms=50000, connections_max_idle_ms=40000) + assert 'foo' in consumer._client._topics - def test_subscription_copy(self): - consumer = KafkaConsumer('foo', api_version=(0, 10, 0)) - sub = consumer.subscription() - assert sub is not consumer.subscription() - assert sub == set(['foo']) - sub.add('fizz') - assert consumer.subscription() == set(['foo']) + consumer = KafkaConsumer(api_version=(0, 10, 0)) + assert consumer.assignment() == set() + consumer.assign([TopicPartition('foo', 0)]) + assert consumer.assignment() == set([TopicPartition('foo', 0)]) + assert 'foo' in consumer._client._topics + # Cannot subscribe + with pytest.raises(IllegalStateError): + consumer.subscribe(topics=['foo']) + consumer.assign([]) + assert consumer.assignment() == set() diff --git a/test/test_fetcher.py b/test/test_fetcher.py index a22f78657..f6e1cf5f4 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -148,7 +148,7 @@ def test_update_fetch_positions(fetcher, topic, mocker): def test__reset_offset(fetcher, mocker): tp = TopicPartition("topic", 0) - fetcher._subscriptions.subscribe(topics="topic") + fetcher._subscriptions.subscribe(topics=["topic"]) fetcher._subscriptions.assign_from_subscribed([tp]) fetcher._subscriptions.need_offset_reset(tp) mocked = mocker.patch.object(fetcher, '_retrieve_offsets') diff --git a/test/test_subscription_state.py b/test/test_subscription_state.py new file mode 100644 index 000000000..bb2c81bff --- /dev/null +++ b/test/test_subscription_state.py @@ -0,0 +1,57 @@ +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(): + s = SubscriptionState() + with pytest.raises(TypeError): + s.subscribe(topics='foo') + + s.subscribe(topics=['foo']) + + +def test_change_subscription(): + s = SubscriptionState() + s.subscribe(topics=['foo']) + assert s.subscription == set(['foo']) + s.change_subscription(['bar']) + assert s.subscription == set(['bar']) + + +def test_group_subscribe(): + s = SubscriptionState() + s.subscribe(topics=['foo']) + assert s.subscription == set(['foo']) + s.group_subscribe(['bar']) + assert s.subscription == set(['foo']) + assert s._group_subscription == set(['foo', 'bar']) + + s.reset_group_subscription() + assert s.subscription == set(['foo']) + assert s._group_subscription == set(['foo']) + + +def test_assign_from_subscribed(): + s = SubscriptionState() + s.subscribe(topics=['foo']) + with pytest.raises(ValueError): + s.assign_from_subscribed([TopicPartition('bar', 0)]) + + 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(s, TopicPartitionState) for s in six.itervalues(s.assignment)]) + assert all([not s.has_valid_position for s in six.itervalues(s.assignment)]) + + +def test_change_subscription_after_assignment(): + s = SubscriptionState() + s.subscribe(topics=['foo']) + s.assign_from_subscribed([TopicPartition('foo', 0), TopicPartition('foo', 1)]) + # Changing subscription retains existing assignment until next rebalance + s.change_subscription(['bar']) + assert set(s.assignment.keys()) == set([TopicPartition('foo', 0), TopicPartition('foo', 1)]) From 70ca6d77a107dbbd6d2a08c4abd20db159eca3d3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 13:09:09 -0700 Subject: [PATCH 089/205] Add optional timeout_ms kwarg to consumer.close() / fix potential hang in test_group (#2564) --- kafka/consumer/group.py | 6 ++++-- test/test_consumer_group.py | 13 +++++++------ 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index ee3f95be7..58284a7a9 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -470,19 +470,21 @@ def assignment(self): """ return self._subscription.assigned_partitions() - def close(self, autocommit=True): + def close(self, autocommit=True, timeout_ms=None): """Close the consumer, waiting indefinitely for any needed cleanup. Keyword Arguments: autocommit (bool): If auto-commit is configured for this consumer, this optional flag causes the consumer to attempt to commit any pending consumed offsets prior to close. Default: True + timeout_ms (num, optional): Milliseconds to wait for auto-commit. + Default: None """ if self._closed: return log.debug("Closing the KafkaConsumer.") self._closed = True - self._coordinator.close(autocommit=autocommit) + self._coordinator.close(autocommit=autocommit, timeout_ms=timeout_ms) self._metrics.close() self._client.close() try: diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index c175e142c..9334a4fd1 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -47,7 +47,7 @@ def test_group(kafka_broker, topic): consumers = {} stop = {} threads = {} - messages = collections.defaultdict(list) + messages = collections.defaultdict(lambda: collections.defaultdict(list)) group_id = 'test-group-' + random_string(6) def consumer_thread(i): assert i not in consumers @@ -60,15 +60,15 @@ 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.itervalues(consumers[i].poll(timeout_ms=200)): + for tp, records in six.iteritems(consumers[i].poll(timeout_ms=200)): messages[i][tp].extend(records) - consumers[i].close() + consumers[i].close(timeout_ms=500) consumers[i] = None stop[i] = None num_consumers = 4 for i in range(num_consumers): - t = threading.Thread(target=consumer_thread, args=(i,)) + t = threading.Thread(target=consumer_thread, args=(i,), daemon=True) t.start() threads[i] = t @@ -129,7 +129,8 @@ def consumer_thread(i): for c in range(num_consumers): logging.info('Stopping consumer %s', c) stop[c].set() - threads[c].join() + threads[c].join(timeout=5) + assert not threads[c].is_alive() threads[c] = None @@ -179,4 +180,4 @@ def test_heartbeat_thread(kafka_broker, topic): assert consumer._coordinator.heartbeat.last_poll == last_poll consumer.poll(timeout_ms=100) assert consumer._coordinator.heartbeat.last_poll > last_poll - consumer.close() + consumer.close(timeout_ms=100) From 23d21f520924ab7f59d3dab93955d50c2a9efa48 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 13:16:09 -0700 Subject: [PATCH 090/205] timeout on consumer_factory in test_producer --- test/test_producer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/test_producer.py b/test/test_producer.py index ea2be89a0..598661aab 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -38,7 +38,7 @@ def consumer_factory(**kwargs): try: yield consumer finally: - consumer.close() + consumer.close(timeout_ms=0) @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") From bd244866acdbb92f1c0bb469c6b969857fc3a57a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 13:38:03 -0700 Subject: [PATCH 091/205] More timeout_ms args in coordinator + heartbeat close --- kafka/coordinator/base.py | 20 +++++++++++--------- kafka/coordinator/consumer.py | 2 +- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index c5e56c538..97ba4fa28 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -776,12 +776,12 @@ def _disable_heartbeat_thread(self): if self._heartbeat_thread is not None: self._heartbeat_thread.disable() - def _close_heartbeat_thread(self): + def _close_heartbeat_thread(self, timeout_ms=None): with self._lock: if self._heartbeat_thread is not None: log.info('Stopping heartbeat thread') try: - self._heartbeat_thread.close() + self._heartbeat_thread.close(timeout_ms=timeout_ms) except ReferenceError: pass self._heartbeat_thread = None @@ -790,13 +790,13 @@ def __del__(self): if hasattr(self, '_heartbeat_thread'): self._close_heartbeat_thread() - def close(self): + def close(self, timeout_ms=None): """Close the coordinator, leave the current group, and reset local generation / member_id""" - self._close_heartbeat_thread() - self.maybe_leave_group() + self._close_heartbeat_thread(timeout_ms=timeout_ms) + self.maybe_leave_group(timeout_ms=timeout_ms) - def maybe_leave_group(self): + def maybe_leave_group(self, timeout_ms=None): """Leave the current group and reset local generation/memberId.""" with self._client._lock, self._lock: if (not self.coordinator_unknown() @@ -811,7 +811,7 @@ def maybe_leave_group(self): future = self._client.send(self.coordinator_id, request) future.add_callback(self._handle_leave_group_response) future.add_errback(log.error, "LeaveGroup request failed: %s") - self._client.poll(future=future) + self._client.poll(future=future, timeout_ms=timeout_ms) self.reset_generation() @@ -957,7 +957,7 @@ def disable(self): log.debug('Disabling heartbeat thread') self.enabled = False - def close(self): + def close(self, timeout_ms=None): if self.closed: return self.closed = True @@ -972,7 +972,9 @@ def close(self): self.coordinator._lock.notify() if self.is_alive(): - self.join(self.coordinator.config['heartbeat_interval_ms'] / 1000) + if timeout_ms is None: + timeout_ms = self.coordinator.config['heartbeat_interval_ms'] + self.join(timeout_ms / 1000) if self.is_alive(): log.warning("Heartbeat thread did not fully terminate during close") diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index f086b0fd7..873b1128c 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -449,7 +449,7 @@ def close(self, autocommit=True, timeout_ms=None): if autocommit: self._maybe_auto_commit_offsets_sync(timeout_ms=timeout_ms) finally: - super(ConsumerCoordinator, self).close() + super(ConsumerCoordinator, self).close(timeout_ms=timeout_ms) def _invoke_completed_offset_commit_callbacks(self): while self.completed_offset_commits: From 70574d111a8f7999620846f24c2c78e453e48192 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 13:54:24 -0700 Subject: [PATCH 092/205] KIP-74: Manage assigned partition order in consumer (#2562) --- kafka/consumer/fetcher.py | 116 +++++++++++++++------------ kafka/consumer/subscription_state.py | 45 ++++++----- test/test_fetcher.py | 10 +-- 3 files changed, 96 insertions(+), 75 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 90dfdbbbc..4d73ef435 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -4,7 +4,6 @@ import copy import itertools import logging -import random import sys import time @@ -57,7 +56,6 @@ class Fetcher(six.Iterator): 'max_partition_fetch_bytes': 1048576, 'max_poll_records': sys.maxsize, 'check_crcs': True, - 'iterator_refetch_records': 1, # undocumented -- interface may change 'metric_group_prefix': 'consumer', 'retry_backoff_ms': 100, 'enable_incremental_fetch_sessions': True, @@ -380,10 +378,13 @@ def _append(self, drained, part, max_records, update_offsets): # as long as the partition is still assigned position = self._subscriptions.assignment[tp].position if part.next_fetch_offset == position.offset: - part_records = part.take(max_records) log.debug("Returning fetched records at offset %d for assigned" " partition %s", position.offset, tp) - drained[tp].extend(part_records) + part_records = part.take(max_records) + # list.extend([]) is a noop, but because drained is a defaultdict + # we should avoid initializing the default list unless there are records + if part_records: + drained[tp].extend(part_records) # We want to increment subscription position if (1) we're using consumer.poll(), # or (2) we didn't return any records (consumer iterator will update position # when each message is yielded). There may be edge cases where we re-fetch records @@ -562,13 +563,11 @@ def _handle_list_offsets_response(self, future, response): def _fetchable_partitions(self): fetchable = self._subscriptions.fetchable_partitions() # do not fetch a partition if we have a pending fetch response to process + discard = {fetch.topic_partition for fetch in self._completed_fetches} current = self._next_partition_records - pending = copy.copy(self._completed_fetches) if current: - fetchable.discard(current.topic_partition) - for fetch in pending: - fetchable.discard(fetch.topic_partition) - return fetchable + discard.add(current.topic_partition) + return [tp for tp in fetchable if tp not in discard] def _create_fetch_requests(self): """Create fetch requests for all assigned partitions, grouped by node. @@ -581,7 +580,7 @@ def _create_fetch_requests(self): # create the fetch info as a dict of lists of partition info tuples # which can be passed to FetchRequest() via .items() version = self._client.api_version(FetchRequest, max_version=10) - fetchable = collections.defaultdict(dict) + fetchable = collections.defaultdict(collections.OrderedDict) for partition in self._fetchable_partitions(): node_id = self._client.cluster.leader_for_partition(partition) @@ -695,10 +694,7 @@ def _handle_fetch_response(self, node_id, fetch_offsets, send_time, response): for partition_data in partitions]) metric_aggregator = FetchResponseMetricAggregator(self._sensors, partitions) - # randomized ordering should improve balance for short-lived consumers - random.shuffle(response.topics) for topic, partitions in response.topics: - random.shuffle(partitions) for partition_data in partitions: tp = TopicPartition(topic, partition_data[0]) fetch_offset = fetch_offsets[tp] @@ -733,8 +729,6 @@ def _parse_fetched_data(self, completed_fetch): " since it is no longer fetchable", tp) elif error_type is Errors.NoError: - self._subscriptions.assignment[tp].highwater = highwater - # we are interested in this fetch only if the beginning # offset (of the *request*) matches the current consumed position # Note that the *response* may return a messageset that starts @@ -748,30 +742,35 @@ def _parse_fetched_data(self, completed_fetch): return None records = MemoryRecords(completed_fetch.partition_data[-1]) - if records.has_next(): - log.debug("Adding fetched record for partition %s with" - " offset %d to buffered record list", tp, - position.offset) - parsed_records = self.PartitionRecords(fetch_offset, tp, records, - self.config['key_deserializer'], - self.config['value_deserializer'], - self.config['check_crcs'], - completed_fetch.metric_aggregator) - return parsed_records - elif records.size_in_bytes() > 0: - # we did not read a single message from a non-empty - # buffer because that message's size is larger than - # fetch size, in this case record this exception - record_too_large_partitions = {tp: fetch_offset} - raise RecordTooLargeError( - "There are some messages at [Partition=Offset]: %s " - " whose size is larger than the fetch size %s" - " and hence cannot be ever returned." - " Increase the fetch size, or decrease the maximum message" - " size the broker will allow." % ( - record_too_large_partitions, - self.config['max_partition_fetch_bytes']), - record_too_large_partitions) + log.debug("Preparing to read %s bytes of data for partition %s with offset %d", + records.size_in_bytes(), tp, fetch_offset) + parsed_records = self.PartitionRecords(fetch_offset, tp, records, + self.config['key_deserializer'], + self.config['value_deserializer'], + self.config['check_crcs'], + completed_fetch.metric_aggregator, + self._on_partition_records_drain) + if not records.has_next() and records.size_in_bytes() > 0: + if completed_fetch.response_version < 3: + # Implement the pre KIP-74 behavior of throwing a RecordTooLargeException. + record_too_large_partitions = {tp: fetch_offset} + raise RecordTooLargeError( + "There are some messages at [Partition=Offset]: %s " + " whose size is larger than the fetch size %s" + " and hence cannot be ever returned. Please condier upgrading your broker to 0.10.1.0 or" + " newer to avoid this issue. Alternatively, increase the fetch size on the client (using" + " max_partition_fetch_bytes)" % ( + record_too_large_partitions, + self.config['max_partition_fetch_bytes']), + record_too_large_partitions) + else: + # This should not happen with brokers that support FetchRequest/Response V3 or higher (i.e. KIP-74) + raise Errors.KafkaError("Failed to make progress reading messages at %s=%s." + " Received a non-empty fetch response from the server, but no" + " complete records were found." % (tp, fetch_offset)) + + if highwater >= 0: + self._subscriptions.assignment[tp].highwater = highwater elif error_type in (Errors.NotLeaderForPartitionError, Errors.ReplicaNotAvailableError, @@ -805,14 +804,25 @@ def _parse_fetched_data(self, completed_fetch): if parsed_records is None: completed_fetch.metric_aggregator.record(tp, 0, 0) - return None + if error_type is not Errors.NoError: + # we move the partition to the end if there was an error. This way, it's more likely that partitions for + # the same topic can remain together (allowing for more efficient serialization). + self._subscriptions.move_partition_to_end(tp) + + return parsed_records + + def _on_partition_records_drain(self, partition_records): + # we move the partition to the end if we received some bytes. This way, it's more likely that partitions + # for the same topic can remain together (allowing for more efficient serialization). + if partition_records.bytes_read > 0: + self._subscriptions.move_partition_to_end(partition_records.topic_partition) def close(self): if self._next_partition_records is not None: self._next_partition_records.drain() class PartitionRecords(object): - def __init__(self, fetch_offset, tp, records, key_deserializer, value_deserializer, check_crcs, metric_aggregator): + def __init__(self, fetch_offset, tp, records, key_deserializer, value_deserializer, check_crcs, metric_aggregator, on_drain): self.fetch_offset = fetch_offset self.topic_partition = tp self.leader_epoch = -1 @@ -824,6 +834,7 @@ def __init__(self, fetch_offset, tp, records, key_deserializer, value_deserializ self.record_iterator = itertools.dropwhile( self._maybe_skip_record, self._unpack_records(tp, records, key_deserializer, value_deserializer)) + self.on_drain = on_drain def _maybe_skip_record(self, record): # When fetching an offset that is in the middle of a @@ -845,6 +856,7 @@ def drain(self): if self.record_iterator is not None: self.record_iterator = None self.metric_aggregator.record(self.topic_partition, self.bytes_read, self.records_read) + self.on_drain(self) def take(self, n=None): return list(itertools.islice(self.record_iterator, 0, n)) @@ -943,6 +955,13 @@ def __init__(self, node_id): self.session_partitions = {} def build_next(self, next_partitions): + """ + Arguments: + next_partitions (dict): TopicPartition -> TopicPartitionState + + Returns: + FetchRequestData + """ if self.next_metadata.is_full: log.debug("Built full fetch %s for node %s with %s partition(s).", self.next_metadata, self.node_id, len(next_partitions)) @@ -965,8 +984,8 @@ def build_next(self, next_partitions): altered.add(tp) log.debug("Built incremental fetch %s for node %s. Added %s, altered %s, removed %s out of %s", - self.next_metadata, self.node_id, added, altered, removed, self.session_partitions.keys()) - to_send = {tp: next_partitions[tp] for tp in (added | altered)} + self.next_metadata, self.node_id, added, altered, removed, self.session_partitions.keys()) + to_send = collections.OrderedDict({tp: next_partitions[tp] for tp in next_partitions if tp in (added | altered)}) return FetchRequestData(to_send, removed, self.next_metadata) def handle_response(self, response): @@ -1106,18 +1125,11 @@ def epoch(self): @property def to_send(self): # Return as list of [(topic, [(partition, ...), ...]), ...] - # so it an be passed directly to encoder + # so it can be passed directly to encoder partition_data = collections.defaultdict(list) for tp, partition_info in six.iteritems(self._to_send): partition_data[tp.topic].append(partition_info) - # As of version == 3 partitions will be returned in order as - # they are requested, so to avoid starvation with - # `fetch_max_bytes` option we need this shuffle - # NOTE: we do have partition_data in random order due to usage - # of unordered structures like dicts, but that does not - # guarantee equal distribution, and starting in Python3.6 - # dicts retain insert order. - return random.sample(list(partition_data.items()), k=len(partition_data)) + return list(partition_data.items()) @property def to_forget(self): diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index a1675c724..07a1a109d 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -1,11 +1,13 @@ from __future__ import absolute_import import abc +from collections import defaultdict, OrderedDict try: from collections import Sequence except ImportError: from collections.abc import Sequence import logging +import random import re from kafka.vendor import six @@ -68,7 +70,7 @@ def __init__(self, offset_reset_strategy='earliest'): self.subscribed_pattern = None # regex str or None self._group_subscription = set() self._user_assignment = set() - self.assignment = dict() + self.assignment = OrderedDict() self.listener = None # initialize to true for the consumers to fetch offset upon starting up @@ -200,14 +202,8 @@ def assign_from_user(self, partitions): if self._user_assignment != set(partitions): self._user_assignment = set(partitions) - - for partition in partitions: - if partition not in self.assignment: - self._add_assigned_partition(partition) - - for tp in set(self.assignment.keys()) - self._user_assignment: - del self.assignment[tp] - + self._set_assignment({partition: self.assignment.get(partition, TopicPartitionState()) + for partition in partitions}) self.needs_fetch_committed_offsets = True def assign_from_subscribed(self, assignments): @@ -229,13 +225,25 @@ def assign_from_subscribed(self, assignments): if tp.topic not in self.subscription: raise ValueError("Assigned partition %s for non-subscribed topic." % (tp,)) - # after rebalancing, we always reinitialize the assignment state - self.assignment.clear() - for tp in assignments: - self._add_assigned_partition(tp) + # after rebalancing, we always reinitialize the assignment value + # randomized ordering should improve balance for short-lived consumers + self._set_assignment({partition: TopicPartitionState() for partition in assignments}, randomize=True) self.needs_fetch_committed_offsets = True log.info("Updated partition assignment: %s", 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)] + if randomize: + random.shuffle(topics) + topic_partitions = OrderedDict({topic: [] for topic in topics}) + for tp in six.iterkeys(partition_states): + topic_partitions[tp.topic].append(tp) + for topic in six.iterkeys(topic_partitions): + for tp in topic_partitions[topic]: + self.assignment[tp] = partition_states[tp] + def unsubscribe(self): """Clear all topic subscriptions and partition assignments""" self.subscription = None @@ -283,11 +291,11 @@ def paused_partitions(self): if self.is_paused(partition)) def fetchable_partitions(self): - """Return set of TopicPartitions that should be Fetched.""" - fetchable = set() + """Return ordered list of TopicPartitions that should be Fetched.""" + fetchable = list() for partition, state in six.iteritems(self.assignment): if state.is_fetchable(): - fetchable.add(partition) + fetchable.append(partition) return fetchable def partitions_auto_assigned(self): @@ -348,8 +356,9 @@ def pause(self, partition): def resume(self, partition): self.assignment[partition].resume() - def _add_assigned_partition(self, partition): - self.assignment[partition] = TopicPartitionState() + def move_partition_to_end(self, partition): + if partition in self.assignment: + self.assignment.move_to_end(partition) class TopicPartitionState(object): diff --git a/test/test_fetcher.py b/test/test_fetcher.py index f6e1cf5f4..7822a6f1f 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -451,7 +451,7 @@ def test__unpack_records(mocker): (None, b"c", None), ] memory_records = MemoryRecords(_build_record_batch(messages)) - part_records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock()) + part_records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) records = list(part_records.record_iterator) assert len(records) == 3 assert all(map(lambda x: isinstance(x, ConsumerRecord), records)) @@ -556,7 +556,7 @@ def test_partition_records_offset(mocker): tp = TopicPartition('foo', 0) messages = [(None, b'msg', None) for i in range(batch_start, batch_end)] memory_records = MemoryRecords(_build_record_batch(messages, offset=batch_start)) - records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock()) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) assert records assert records.next_fetch_offset == fetch_offset msgs = records.take(1) @@ -573,7 +573,7 @@ def test_partition_records_offset(mocker): def test_partition_records_empty(mocker): tp = TopicPartition('foo', 0) memory_records = MemoryRecords(_build_record_batch([])) - records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock()) + records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) msgs = records.take() assert len(msgs) == 0 assert not records @@ -586,7 +586,7 @@ def test_partition_records_no_fetch_offset(mocker): tp = TopicPartition('foo', 0) messages = [(None, b'msg', None) for i in range(batch_start, batch_end)] memory_records = MemoryRecords(_build_record_batch(messages, offset=batch_start)) - records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock()) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) msgs = records.take() assert len(msgs) == 0 assert not records @@ -610,7 +610,7 @@ def test_partition_records_compacted_offset(mocker): builder.append(key=None, value=b'msg', timestamp=None, headers=[]) builder.close() memory_records = MemoryRecords(builder.buffer()) - records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock()) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) msgs = records.take() assert len(msgs) == batch_end - fetch_offset - 1 assert msgs[0].offset == fetch_offset + 1 From 6c2c25d3d8db834873f7bc6ff2747885c59cdedd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 24 Mar 2025 16:41:40 -0700 Subject: [PATCH 093/205] Use SubscriptionType to track topics/pattern/user assignment (#2565) --- kafka/consumer/subscription_state.py | 43 +++++++++++++++++++++------- test/test_consumer_integration.py | 4 +-- test/test_coordinator.py | 1 + 3 files changed, 36 insertions(+), 12 deletions(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 07a1a109d..77742109b 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -6,6 +6,12 @@ from collections import Sequence except ImportError: from collections.abc 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 import logging import random import re @@ -20,6 +26,13 @@ log = logging.getLogger(__name__) +class SubscriptionType(IntEnum): + NONE = 0 + AUTO_TOPICS = 1 + AUTO_PATTERN = 2 + USER_ASSIGNED = 3 + + class SubscriptionState(object): """ A class for tracking the topics, partitions, and offsets for the consumer. @@ -67,6 +80,7 @@ def __init__(self, offset_reset_strategy='earliest'): self._default_offset_reset_strategy = offset_reset_strategy self.subscription = None # set() or None + self.subscription_type = SubscriptionType.NONE self.subscribed_pattern = None # regex str or None self._group_subscription = set() self._user_assignment = set() @@ -76,6 +90,14 @@ def __init__(self, offset_reset_strategy='earliest'): # initialize to true for the consumers to fetch offset upon starting up self.needs_fetch_committed_offsets = True + def _set_subscription_type(self, subscription_type): + if not isinstance(subscription_type, SubscriptionType): + raise ValueError('SubscriptionType enum required') + if self.subscription_type == SubscriptionType.NONE: + self.subscription_type = subscription_type + elif self.subscription_type != subscription_type: + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + def subscribe(self, topics=(), pattern=None, listener=None): """Subscribe to a list of topics, or a topic regex pattern. @@ -111,17 +133,19 @@ def subscribe(self, topics=(), pattern=None, listener=None): guaranteed, however, that the partitions revoked/assigned through this interface are from topics subscribed in this call. """ - if self._user_assignment or (topics and pattern): - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) assert topics or pattern, 'Must provide topics or pattern' + if (topics and pattern): + raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) - if pattern: + elif pattern: + self._set_subscription_type(SubscriptionType.AUTO_PATTERN) log.info('Subscribing to pattern: /%s/', pattern) self.subscription = set() self.subscribed_pattern = re.compile(pattern) else: if isinstance(topics, str) or not isinstance(topics, Sequence): raise TypeError('Topics must be a list (or non-str sequence)') + self._set_subscription_type(SubscriptionType.AUTO_TOPICS) self.change_subscription(topics) if listener and not isinstance(listener, ConsumerRebalanceListener): @@ -141,7 +165,7 @@ def change_subscription(self, topics): - a topic name is '.' or '..' or - a topic name does not consist of ASCII-characters/'-'/'_'/'.' """ - if self._user_assignment: + if not self.partitions_auto_assigned(): raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) if isinstance(topics, six.string_types): @@ -168,13 +192,13 @@ def group_subscribe(self, topics): Arguments: topics (list of str): topics to add to the group subscription """ - if self._user_assignment: + if not self.partitions_auto_assigned(): raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) self._group_subscription.update(topics) def reset_group_subscription(self): """Reset the group's subscription to only contain topics subscribed by this consumer.""" - if self._user_assignment: + if not self.partitions_auto_assigned(): raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) assert self.subscription is not None, 'Subscription required' self._group_subscription.intersection_update(self.subscription) @@ -197,9 +221,7 @@ def assign_from_user(self, partitions): Raises: IllegalStateError: if consumer has already called subscribe() """ - if self.subscription is not None: - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) - + self._set_subscription_type(SubscriptionType.USER_ASSIGNED) if self._user_assignment != set(partitions): self._user_assignment = set(partitions) self._set_assignment({partition: self.assignment.get(partition, TopicPartitionState()) @@ -250,6 +272,7 @@ def unsubscribe(self): self._user_assignment.clear() self.assignment.clear() self.subscribed_pattern = None + self.subscription_type = SubscriptionType.NONE def group_subscription(self): """Get the topic subscription for the group. @@ -300,7 +323,7 @@ def fetchable_partitions(self): def partitions_auto_assigned(self): """Return True unless user supplied partitions manually.""" - return self.subscription is not None + return self.subscription_type in (SubscriptionType.AUTO_TOPICS, SubscriptionType.AUTO_PATTERN) def all_consumed_offsets(self): """Returns consumed offsets as {TopicPartition: OffsetAndMetadata}""" diff --git a/test/test_consumer_integration.py b/test/test_consumer_integration.py index af8ec6829..b181845a4 100644 --- a/test/test_consumer_integration.py +++ b/test/test_consumer_integration.py @@ -68,8 +68,8 @@ def test_kafka_consumer_unsupported_encoding( def test_kafka_consumer__blocking(kafka_consumer_factory, topic, send_messages): TIMEOUT_MS = 500 consumer = kafka_consumer_factory(auto_offset_reset='earliest', - enable_auto_commit=False, - consumer_timeout_ms=TIMEOUT_MS) + enable_auto_commit=False, + consumer_timeout_ms=TIMEOUT_MS) # Manual assignment avoids overhead of consumer group mgmt consumer.unsubscribe() diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 09422790e..35749f84d 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -189,6 +189,7 @@ 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'') From 9dd1714058ecc0628647a4cdca8bc9815ebd10ed Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 25 Mar 2025 08:43:14 -0700 Subject: [PATCH 094/205] Also update config api_version to closest compatible (#2567) --- kafka/client_async.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/client_async.py b/kafka/client_async.py index 4de05b33e..8df4566e6 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -276,6 +276,7 @@ def __init__(self, **configs): if compatible_version: log.warning('Configured api_version %s not supported; using %s', self.config['api_version'], compatible_version) + self.config['api_version'] = compatible_version self._api_versions = BROKER_API_VERSIONS[compatible_version] else: raise Errors.UnrecognizedBrokerVersion(self.config['api_version']) From 9c7aed4334905969bc951cf1e4d6264e3e89c211 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 25 Mar 2025 08:50:52 -0700 Subject: [PATCH 095/205] Patch Release 2.1.3 --- CHANGES.md | 22 ++++++++++++++++++++++ docs/changelog.rst | 28 ++++++++++++++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 51 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index e24c5c0aa..d4f9f6317 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,25 @@ +# 2.1.3 (Mar 25, 2025) + +Fixes +* Fix crash when switching to closest compatible api_version in KafkaClient (#2567) +* Fix maximum version to send an OffsetFetchRequest in KafkaAdminClient (#2563) +* Return empty set from consumer.partitions_for_topic when topic not found (#2556) + +Improvements +* KIP-511: Use ApiVersions v4 on initial connect w/ client_software_name + version (#2558) +* KIP-74: Manage assigned partition order in consumer (#2562) +* KIP-70: Auto-commit offsets on consumer.unsubscribe(), defer assignment changes to rejoin (#2560) +* Use SubscriptionType to track topics/pattern/user assignment (#2565) +* Add optional timeout_ms kwarg to consumer.close() (#2564) +* Move ensure_valid_topic_name to kafka.util; use in client and producer (#2561) + +Testing +* Support KRaft / 4.0 brokers in tests (#2559) +* Test older pythons against 4.0 broker + +Compatibility +* Add python 3.13 to compatibility list + # 2.1.2 (Mar 17, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 4695c8dba..f802fc00c 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,34 @@ Changelog ========= +2.1.3 (Mar 25, 2025) +#################### + +Fixes +----- +* Fix crash when switching to closest compatible api_version in KafkaClient (#2567) +* Fix maximum version to send an OffsetFetchRequest in KafkaAdminClient (#2563) +* Return empty set from consumer.partitions_for_topic when topic not found (#2556) + +Improvements +------------ +* KIP-511: Use ApiVersions v4 on initial connect w/ client_software_name + version (#2558) +* KIP-74: Manage assigned partition order in consumer (#2562) +* KIP-70: Auto-commit offsets on consumer.unsubscribe(), defer assignment changes to rejoin (#2560) +* Use SubscriptionType to track topics/pattern/user assignment (#2565) +* Add optional timeout_ms kwarg to consumer.close() (#2564) +* Move ensure_valid_topic_name to kafka.util; use in client and producer (#2561) + +Testing +------- +* Support KRaft / 4.0 brokers in tests (#2559) +* Test older pythons against 4.0 broker + +Compatibility +------------- +* Add python 3.13 to compatibility list + + 2.1.2 (Mar 17, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index b0be43da3..2d31b1c32 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.3.dev' +__version__ = '2.1.3' From a46f3d8d8271c76fa909c56d8752a2466b9c08ef Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 27 Mar 2025 00:17:28 -0400 Subject: [PATCH 096/205] Try import new Sequence before old to avoid DeprecationWarning --- kafka/consumer/subscription_state.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 77742109b..d3b791a44 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -3,9 +3,9 @@ import abc from collections import defaultdict, OrderedDict try: - from collections import Sequence -except ImportError: 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 From 70ec261b0d41448eb9d6b4ff9456d0d2d65edf15 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 27 Mar 2025 09:42:00 -0700 Subject: [PATCH 097/205] Reset SASL state on disconnect (#2571) --- kafka/conn.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index f6af172f4..6963a8b08 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -271,12 +271,10 @@ def __init__(self, host, port, afi, **configs): assert self.config['security_protocol'] in self.SECURITY_PROTOCOLS, ( 'security_protocol must be in ' + ', '.join(self.SECURITY_PROTOCOLS)) - self._sasl_mechanism = None if self.config['security_protocol'] in ('SSL', 'SASL_SSL'): assert ssl_available, "Python wasn't built with SSL support" - if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'): - self._sasl_mechanism = get_sasl_mechanism(self.config['sasl_mechanism'])(**self.config) + self._init_sasl_mechanism() # This is not a general lock / this class is not generally thread-safe yet # However, to avoid pushing responsibility for maintaining @@ -312,6 +310,12 @@ def __init__(self, host, port, afi, **configs): self.config['metric_group_prefix'], self.node_id) + 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'])(**self.config) + else: + self._sasl_mechanism = None + def _dns_lookup(self): self._gai = dns_lookup(self.host, self.port, self.afi) if not self._gai: @@ -747,6 +751,7 @@ def _send_sasl_authenticate(self, sasl_auth_bytes): request = SaslAuthenticateRequest[0](sasl_auth_bytes) self._send(request, blocking=True) else: + log.debug('Sending %d raw sasl auth bytes to server', len(sasl_auth_bytes)) try: self._send_bytes_blocking(Int32.encode(len(sasl_auth_bytes)) + sasl_auth_bytes) except (ConnectionError, TimeoutError) as e: @@ -787,6 +792,7 @@ def _recv_sasl_authenticate(self): return response.auth_bytes else: # unframed bytes w/ SaslHandhake v0 + log.debug('Received %d raw sasl auth bytes from server', nbytes) return data[4:] def _sasl_authenticate(self, future): @@ -930,6 +936,7 @@ def close(self, error=None): self._update_reconnect_backoff() 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']) From d39dd3b5461a2c987cdf7bf53120b338d81a1f66 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 27 Mar 2025 16:39:15 -0400 Subject: [PATCH 098/205] Update Makefile default to 4.0 broker; add make fixture --- Makefile | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/Makefile b/Makefile index c0128e7e2..a624b833f 100644 --- a/Makefile +++ b/Makefile @@ -2,7 +2,7 @@ SHELL = bash -export KAFKA_VERSION ?= 2.4.0 +export KAFKA_VERSION ?= 4.0.0 DIST_BASE_URL ?= https://archive.apache.org/dist/kafka/ # Required to support testing old kafka versions on newer java releases @@ -23,6 +23,9 @@ lint: test: build-integration pytest $(PYTESTS) +fixture: build-integration + python -m test.fixtures kafka + cov-local: build-integration pytest --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka \ --cov-config=.covrc --cov-report html $(TEST_FLAGS) kafka test From e21fe99aea757f00abbe12d174f0c452d2ae0d5a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 27 Mar 2025 14:22:56 -0700 Subject: [PATCH 099/205] Send final error byte x01 on Sasl OAuth failure (#2572) --- kafka/sasl/oauth.py | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/kafka/sasl/oauth.py b/kafka/sasl/oauth.py index 4041a93bd..f1e959cb6 100644 --- a/kafka/sasl/oauth.py +++ b/kafka/sasl/oauth.py @@ -1,10 +1,14 @@ from __future__ import absolute_import import abc +import logging from kafka.sasl.abc import SaslMechanism +log = logging.getLogger(__name__) + + class SaslMechanismOAuth(SaslMechanism): def __init__(self, **config): @@ -12,17 +16,26 @@ def __init__(self, **config): assert isinstance(config['sasl_oauth_token_provider'], AbstractTokenProvider), \ 'sasl_oauth_token_provider must implement kafka.sasl.oauth.AbstractTokenProvider' self.token_provider = config['sasl_oauth_token_provider'] + self._error = None self._is_done = False self._is_authenticated = False def auth_bytes(self): + if self._error: + # Server should respond to this with SaslAuthenticate failure, which ends the auth process + return self._error token = self.token_provider.token() extensions = self._token_extensions() return "n,,\x01auth=Bearer {}{}\x01\x01".format(token, extensions).encode('utf-8') def receive(self, auth_bytes): - self._is_done = True - self._is_authenticated = auth_bytes == b'' + if auth_bytes != b'': + error = auth_bytes.decode('utf-8') + log.debug("Sending x01 response to server after receiving SASL OAuth error: %s", error) + self._error = b'\x01' + else: + self._is_done = True + self._is_authenticated = True def is_done(self): return self._is_done From c75b85b4de28821ef95201e8c833f24e712942d7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 27 Mar 2025 17:34:26 -0700 Subject: [PATCH 100/205] Fix MetadataRequest for no topics (#2573) --- kafka/client_async.py | 4 +++- kafka/protocol/metadata.py | 1 + test/test_client_async.py | 5 +++-- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 8df4566e6..8c071104e 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -978,8 +978,10 @@ def _maybe_refresh_metadata(self, wakeup=False): topics = list(self.config['bootstrap_topics_filter']) api_version = self.api_version(MetadataRequest, max_version=7) - if self.cluster.need_all_topic_metadata or not topics: + 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: request = MetadataRequest[api_version](topics, self.config['allow_auto_create_topics']) else: diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py index 3291be82d..bb22ba997 100644 --- a/kafka/protocol/metadata.py +++ b/kafka/protocol/metadata.py @@ -172,6 +172,7 @@ class MetadataRequest_v0(Request): ('topics', Array(String('utf-8'))) ) ALL_TOPICS = [] # Empty Array (len 0) for topics returns all topics + NO_TOPICS = [] # v0 does not support a 'no topics' request, so we'll just ask for ALL class MetadataRequest_v1(Request): diff --git a/test/test_client_async.py b/test/test_client_async.py index 8582d8fb7..276926116 100644 --- a/test/test_client_async.py +++ b/test/test_client_async.py @@ -32,7 +32,7 @@ def cli(mocker, conn): def test_bootstrap(mocker, conn): conn.state = ConnectionStates.CONNECTED - cli = KafkaClient(api_version=(0, 9)) + cli = KafkaClient(api_version=(2, 1)) mocker.patch.object(cli, '_selector') future = cli.cluster.request_update() cli.poll(future=future) @@ -43,7 +43,7 @@ def test_bootstrap(mocker, conn): kwargs.pop('state_change_callback') kwargs.pop('node_id') assert kwargs == cli.config - conn.send.assert_called_once_with(MetadataRequest[0]([]), blocking=False, request_timeout_ms=None) + conn.send.assert_called_once_with(MetadataRequest[7]([], True), blocking=False, request_timeout_ms=None) assert cli._bootstrap_fails == 0 assert cli.cluster.brokers() == set([BrokerMetadata(0, 'foo', 12, None), BrokerMetadata(1, 'bar', 34, None)]) @@ -330,6 +330,7 @@ def test_maybe_refresh_metadata_update(mocker, client): mocker.patch.object(client, 'least_loaded_node', return_value='foobar') mocker.patch.object(client, '_can_send_request', return_value=True) send = mocker.patch.object(client, 'send') + client.cluster.need_all_topic_metadata = True client.poll(timeout_ms=12345678) client._poll.assert_called_with(9999.999) # request_timeout_ms From a520232f267e396cd1f275799606019f023e0fff Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 28 Mar 2025 08:38:46 -0700 Subject: [PATCH 101/205] Improve connection state logging (#2574) --- kafka/client_async.py | 2 +- kafka/conn.py | 49 ++++++++++++++++++++++--------------------- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 8c071104e..19508b242 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -365,7 +365,7 @@ def _conn_state_change(self, node_id, sock, conn): self._connecting.remove(node_id) try: self._selector.unregister(sock) - except KeyError: + except (KeyError, ValueError): pass if self._sensors: diff --git a/kafka/conn.py b/kafka/conn.py index 6963a8b08..ec516b0f4 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -319,8 +319,8 @@ def _init_sasl_mechanism(self): def _dns_lookup(self): self._gai = dns_lookup(self.host, self.port, self.afi) if not self._gai: - log.error('DNS lookup failed for %s:%i (%s)', - self.host, self.port, self.afi) + log.error('%s: DNS lookup failed for %s:%i (%s)', + self, self.host, self.port, self.afi) return False return True @@ -366,6 +366,7 @@ def connect_blocking(self, timeout=float('inf')): def connect(self): """Attempt to connect and return ConnectionState""" if self.state is ConnectionStates.DISCONNECTED and not self.blacked_out(): + self.state = ConnectionStates.CONNECTING self.last_attempt = time.time() next_lookup = self._next_afi_sockaddr() if not next_lookup: @@ -390,7 +391,6 @@ def connect(self): self._sock.setsockopt(*option) self._sock.setblocking(False) - self.state = ConnectionStates.CONNECTING self.config['state_change_callback'](self.node_id, self._sock, self) log.info('%s: connecting to %s:%d [%s %s]', self, self.host, self.port, self._sock_addr, AFI_NAMES[self._sock_afi]) @@ -412,20 +412,20 @@ def connect(self): log.debug('%s: established TCP connection', self) if self.config['security_protocol'] in ('SSL', 'SASL_SSL'): - log.debug('%s: initiating SSL handshake', self) self.state = ConnectionStates.HANDSHAKE + log.debug('%s: initiating SSL handshake', self) self.config['state_change_callback'](self.node_id, self._sock, self) # _wrap_ssl can alter the connection state -- disconnects on failure self._wrap_ssl() else: - log.debug('%s: checking broker Api Versions', self) self.state = ConnectionStates.API_VERSIONS_SEND + log.debug('%s: checking broker Api Versions', self) self.config['state_change_callback'](self.node_id, self._sock, self) # Connection failed # WSAEINVAL == 10022, but errno.WSAEINVAL is not available on non-win systems elif ret not in (errno.EINPROGRESS, errno.EALREADY, errno.EWOULDBLOCK, 10022): - log.error('Connect attempt to %s returned error %s.' + log.error('%s: Connect attempt returned error %s.' ' Disconnecting.', self, ret) errstr = errno.errorcode.get(ret, 'UNKNOWN') self.close(Errors.KafkaConnectionError('{} {}'.format(ret, errstr))) @@ -438,8 +438,8 @@ def connect(self): if self.state is ConnectionStates.HANDSHAKE: if self._try_handshake(): log.debug('%s: completed SSL handshake.', self) - log.debug('%s: checking broker Api Versions', self) self.state = ConnectionStates.API_VERSIONS_SEND + log.debug('%s: checking broker Api Versions', self) self.config['state_change_callback'](self.node_id, self._sock, self) if self.state in (ConnectionStates.API_VERSIONS_SEND, ConnectionStates.API_VERSIONS_RECV): @@ -447,13 +447,13 @@ def connect(self): # _try_api_versions_check has side-effects: possibly disconnected on socket errors if self.state in (ConnectionStates.API_VERSIONS_SEND, ConnectionStates.API_VERSIONS_RECV): if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'): - log.debug('%s: initiating SASL authentication', self) self.state = ConnectionStates.AUTHENTICATING + log.debug('%s: initiating SASL authentication', self) self.config['state_change_callback'](self.node_id, self._sock, self) else: # security_protocol PLAINTEXT - log.info('%s: Connection complete.', self) self.state = ConnectionStates.CONNECTED + log.info('%s: Connection complete.', self) self._reset_reconnect_backoff() self.config['state_change_callback'](self.node_id, self._sock, self) @@ -462,8 +462,8 @@ def connect(self): if self._try_authenticate(): # _try_authenticate has side-effects: possibly disconnected on socket errors if self.state is ConnectionStates.AUTHENTICATING: - log.info('%s: Connection complete.', self) self.state = ConnectionStates.CONNECTED + log.info('%s: Connection complete.', self) self._reset_reconnect_backoff() self.config['state_change_callback'](self.node_id, self._sock, self) @@ -472,7 +472,7 @@ def connect(self): # Connection timed out request_timeout = self.config['request_timeout_ms'] / 1000.0 if time.time() > request_timeout + self.last_attempt: - log.error('Connection attempt to %s timed out', self) + log.error('%s: Connection attempt timed out', self) self.close(Errors.KafkaConnectionError('timeout')) return self.state @@ -531,7 +531,7 @@ def _try_handshake(self): except (SSLWantReadError, SSLWantWriteError): pass except (SSLZeroReturnError, ConnectionError, TimeoutError, SSLEOFError): - log.warning('SSL connection closed by server during handshake.') + log.warning('%s: SSL connection closed by server during handshake.', self) self.close(Errors.KafkaConnectionError('SSL connection closed by server during handshake')) # Other SSLErrors will be raised to user @@ -611,7 +611,7 @@ def _handle_api_versions_response(self, future, response): for api_key, min_version, max_version, *rest in response.api_versions ]) self._api_version = self._infer_broker_version_from_api_versions(self._api_versions) - log.info('Broker version identified as %s', '.'.join(map(str, self._api_version))) + log.info('%s: Broker version identified as %s', self, '.'.join(map(str, self._api_version))) future.success(self._api_version) self.connect() @@ -621,7 +621,7 @@ def _handle_api_versions_failure(self, future, ex): # after failure connection is closed, so state should already be DISCONNECTED def _handle_check_version_response(self, future, version, _response): - log.info('Broker version identified as %s', '.'.join(map(str, version))) + log.info('%s: Broker version identified as %s', self, '.'.join(map(str, version))) log.info('Set configuration api_version=%s to skip auto' ' check_version requests on startup', version) self._api_versions = BROKER_API_VERSIONS[version] @@ -751,7 +751,7 @@ def _send_sasl_authenticate(self, sasl_auth_bytes): request = SaslAuthenticateRequest[0](sasl_auth_bytes) self._send(request, blocking=True) else: - log.debug('Sending %d raw sasl auth bytes to server', len(sasl_auth_bytes)) + log.debug('%s: Sending %d raw sasl auth bytes to server', self, len(sasl_auth_bytes)) try: self._send_bytes_blocking(Int32.encode(len(sasl_auth_bytes)) + sasl_auth_bytes) except (ConnectionError, TimeoutError) as e: @@ -781,7 +781,7 @@ def _recv_sasl_authenticate(self): latency_ms = (time.time() - timestamp) * 1000 if self._sensors: self._sensors.request_time.record(latency_ms) - log.debug('%s Response %d (%s ms): %s', self, correlation_id, latency_ms, response) + log.debug('%s: Response %d (%s ms): %s', self, correlation_id, latency_ms, response) error_type = Errors.for_code(response.error_code) if error_type is not Errors.NoError: @@ -792,7 +792,7 @@ def _recv_sasl_authenticate(self): return response.auth_bytes else: # unframed bytes w/ SaslHandhake v0 - log.debug('Received %d raw sasl auth bytes from server', nbytes) + log.debug('%s: Received %d raw sasl auth bytes from server', self, nbytes) return data[4:] def _sasl_authenticate(self, future): @@ -956,7 +956,8 @@ def close(self, error=None): # drop lock before state change callback and processing futures self.config['state_change_callback'](self.node_id, sock, self) - sock.close() + if sock: + sock.close() for (_correlation_id, (future, _timestamp, _timeout)) in ifrs: future.failure(error) @@ -1002,7 +1003,7 @@ def _send(self, request, blocking=True, request_timeout_ms=None): correlation_id = self._protocol.send_request(request) - log.debug('%s Request %d (timeout_ms %s): %s', self, correlation_id, request_timeout_ms, request) + log.debug('%s: Request %d (timeout_ms %s): %s', self, correlation_id, request_timeout_ms, request) if request.expect_response(): assert correlation_id not in self.in_flight_requests, 'Correlation ID already in-flight!' sent_time = time.time() @@ -1036,7 +1037,7 @@ def send_pending_requests(self): return True except (ConnectionError, TimeoutError) as e: - log.exception("Error sending request data to %s", self) + log.exception("%s: Error sending request data", self) error = Errors.KafkaConnectionError("%s: %s" % (self, e)) self.close(error=error) return False @@ -1069,7 +1070,7 @@ def send_pending_requests_v2(self): return len(self._send_buffer) == 0 except (ConnectionError, TimeoutError, Exception) as e: - log.exception("Error sending request data to %s", self) + log.exception("%s: Error sending request data", self) error = Errors.KafkaConnectionError("%s: %s" % (self, e)) self.close(error=error) return False @@ -1106,7 +1107,7 @@ def recv(self): if not responses and self.requests_timed_out(): timed_out = self.timed_out_ifrs() timeout_ms = (timed_out[0][2] - timed_out[0][1]) * 1000 - log.warning('%s timed out after %s ms. Closing connection.', + log.warning('%s: timed out after %s ms. Closing connection.', self, timeout_ms) self.close(error=Errors.RequestTimedOutError( 'Request timed out after %s ms' % @@ -1125,7 +1126,7 @@ def recv(self): if self._sensors: self._sensors.request_time.record(latency_ms) - log.debug('%s Response %d (%s ms): %s', self, correlation_id, latency_ms, response) + log.debug('%s: Response %d (%s ms): %s', self, correlation_id, latency_ms, response) self._maybe_throttle(response) responses[i] = (response, future) @@ -1137,7 +1138,7 @@ def _recv(self): err = None with self._lock: if not self._can_send_recv(): - log.warning('%s cannot recv: socket not connected', self) + log.warning('%s: cannot recv: socket not connected', self) return () while len(recvd) < self.config['sock_chunk_buffer_count']: From c15a7fff14068285430d700e780e5e110c01e115 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 28 Mar 2025 10:51:55 -0700 Subject: [PATCH 102/205] Dont block pending FetchRequests when Metadata update requested (#2576) --- kafka/consumer/fetcher.py | 31 +++++++++++++++++++++++-------- test/test_fetcher.py | 2 ++ 2 files changed, 25 insertions(+), 8 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 4d73ef435..61480fb07 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -114,6 +114,7 @@ def __init__(self, client, subscriptions, metrics, **configs): self._sensors = FetchManagerMetrics(metrics, self.config['metric_group_prefix']) self._isolation_level = READ_UNCOMMITTED self._session_handlers = {} + self._nodes_with_pending_fetch_requests = set() def send_fetches(self): """Send FetchRequests for all assigned partitions that do not already have @@ -124,12 +125,12 @@ def send_fetches(self): """ futures = [] for node_id, (request, fetch_offsets) in six.iteritems(self._create_fetch_requests()): - if self._client.ready(node_id): - log.debug("Sending FetchRequest to node %s", node_id) - future = self._client.send(node_id, request, wakeup=False) - future.add_callback(self._handle_fetch_response, node_id, fetch_offsets, time.time()) - future.add_errback(self._handle_fetch_error, node_id) - futures.append(future) + 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) + future.add_callback(self._handle_fetch_response, node_id, fetch_offsets, time.time()) + future.add_errback(self._handle_fetch_error, node_id) + futures.append(future) self._fetch_futures.extend(futures) self._clean_done_fetch_futures() return futures @@ -593,8 +594,20 @@ def _create_fetch_requests(self): " Requesting metadata update", partition) self._client.cluster.request_update() - elif self._client.in_flight_request_count(node_id) > 0: - log.log(0, "Skipping fetch for partition %s because there is an inflight request to node %s", + elif not self._client.connected(node_id) and self._client.connection_delay(node_id) > 0: + # If we try to send during the reconnect backoff window, then the request is just + # going to be failed anyway before being sent, so skip the send for now + log.log(0, "Skipping fetch for partition %s because node %s is awaiting reconnect backoff", + partition, node_id) + + elif self._client.throttle_delay(node_id) > 0: + # If we try to send while throttled, then the request is just + # going to be failed anyway before being sent, so skip the send for now + log.log(0, "Skipping fetch for partition %s because node %s is throttled", + partition, node_id) + + elif node_id in self._nodes_with_pending_fetch_requests: + log.log(0, "Skipping fetch for partition %s because there is a pending fetch request to node %s", partition, node_id) continue @@ -707,12 +720,14 @@ def _handle_fetch_response(self, node_id, fetch_offsets, send_time, response): self._completed_fetches.append(completed_fetch) self._sensors.fetch_latency.record((time.time() - send_time) * 1000) + self._nodes_with_pending_fetch_requests.remove(node_id) def _handle_fetch_error(self, node_id, exception): level = logging.INFO if isinstance(exception, Errors.Cancelled) else logging.ERROR log.log(level, 'Fetch to node %s failed: %s', node_id, exception) if node_id in self._session_handlers: self._session_handlers[node_id].handle_error(exception) + self._nodes_with_pending_fetch_requests.remove(node_id) def _parse_fetched_data(self, completed_fetch): tp = completed_fetch.topic_partition diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 7822a6f1f..854f1fa98 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -423,6 +423,7 @@ def test_fetched_records(fetcher, topic, mocker): ), ]) def test__handle_fetch_response(fetcher, fetch_offsets, fetch_response, num_partitions): + fetcher._nodes_with_pending_fetch_requests.add(0) fetcher._handle_fetch_response(0, fetch_offsets, time.time(), fetch_response) assert len(fetcher._completed_fetches) == num_partitions @@ -438,6 +439,7 @@ def test__handle_fetch_response(fetcher, fetch_offsets, fetch_response, num_part ) ]) def test__handle_fetch_error(fetcher, caplog, exception, log_level): + fetcher._nodes_with_pending_fetch_requests.add(3) fetcher._handle_fetch_error(3, exception) assert len(caplog.records) == 1 assert caplog.records[0].levelname == logging.getLevelName(log_level) From 8af0dd7b95a24a2c2ffffb9b2fec60ac0107f6d4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 28 Mar 2025 13:55:26 -0400 Subject: [PATCH 103/205] Patch Release 2.1.4 --- CHANGES.md | 13 +++++++++++++ docs/changelog.rst | 17 +++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index d4f9f6317..c67ca8ace 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,16 @@ +# 2.1.4 (Mar 28, 2025) + +Fixes +* Dont block pending FetchRequests when Metadata update requested (#2576) +* Fix MetadataRequest for no topics (#2573) +* Send final error byte x01 on Sasl OAuth failure (#2572) +* Reset SASL state on disconnect (#2571) +* Try import new Sequence before old to avoid DeprecationWarning + +Improvements +* Update Makefile default to 4.0 broker; add make fixture +* Improve connection state logging (#2574) + # 2.1.3 (Mar 25, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index f802fc00c..bad0893c2 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,23 @@ Changelog ========= +2.1.4 (Mar 28, 2025) +#################### + +Fixes +----- +* Dont block pending FetchRequests when Metadata update requested (#2576) +* Fix MetadataRequest for no topics (#2573) +* Send final error byte x01 on Sasl OAuth failure (#2572) +* Reset SASL state on disconnect (#2571) +* Try import new Sequence before old to avoid DeprecationWarning + +Improvements +------------ +* Update Makefile default to 4.0 broker; add make fixture +* Improve connection state logging (#2574) + + 2.1.3 (Mar 25, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 2d31b1c32..503eeb92d 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.3' +__version__ = '2.1.4' From 2966a8e28831a2ee983d5b66c14111a6fab8bdde Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 28 Mar 2025 14:04:18 -0400 Subject: [PATCH 104/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 503eeb92d..81baaee65 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.4' +__version__ = '2.1.5.dev' From f7b313364c6184341e612ba916bac6e05169d812 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 3 Apr 2025 10:15:37 -0700 Subject: [PATCH 105/205] Fix python2.7 errors (#2578) --- kafka/conn.py | 7 ++++--- kafka/consumer/fetcher.py | 3 +++ kafka/consumer/subscription_state.py | 6 +++++- test/record/test_default_records.py | 5 +++++ test/record/test_legacy_records.py | 4 ++++ test/record/test_records.py | 3 +++ test/test_conn.py | 5 ++++- test/test_consumer_group.py | 3 ++- test/test_coordinator.py | 24 +++++++++++++++++++----- test/test_producer.py | 4 +++- test/test_subscription_state.py | 4 ++-- test/testutil.py | 16 ++++++++++++++++ 12 files changed, 70 insertions(+), 14 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index ec516b0f4..1febb479a 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -594,7 +594,8 @@ def _handle_api_versions_response(self, future, response): future.failure(error_type()) if error_type is Errors.UnsupportedVersionError: self._api_versions_idx -= 1 - for api_key, min_version, max_version, *rest in response.api_versions: + for api_version_data in response.api_versions: + api_key, min_version, max_version = api_version_data[:3] # If broker provides a lower max_version, skip to that if api_key == response.API_KEY: self._api_versions_idx = min(self._api_versions_idx, max_version) @@ -607,8 +608,8 @@ def _handle_api_versions_response(self, future, response): self.close(error=error_type()) return self._api_versions = dict([ - (api_key, (min_version, max_version)) - for api_key, min_version, max_version, *rest in response.api_versions + (api_version_data[0], (api_version_data[1], api_version_data[2])) + for api_version_data in response.api_versions ]) self._api_version = self._infer_broker_version_from_api_versions(self._api_versions) log.info('%s: Broker version identified as %s', self, '.'.join(map(str, self._api_version))) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 61480fb07..29c2a7182 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -867,6 +867,9 @@ 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 diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index d3b791a44..4cc21020e 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -381,7 +381,11 @@ def resume(self, partition): def move_partition_to_end(self, partition): if partition in self.assignment: - self.assignment.move_to_end(partition) + try: + self.assignment.move_to_end(partition) + except AttributeError: + state = self.assignment.pop(partition) + self.assignment[partition] = state class TopicPartitionState(object): diff --git a/test/record/test_default_records.py b/test/record/test_default_records.py index e1c840fa6..79d3975a5 100644 --- a/test/record/test_default_records.py +++ b/test/record/test_default_records.py @@ -11,6 +11,8 @@ ) from kafka.errors import UnsupportedCodecError +from test.testutil import maybe_skip_unsupported_compression + @pytest.mark.parametrize("compression_type", [ DefaultRecordBatch.CODEC_NONE, @@ -19,6 +21,7 @@ DefaultRecordBatch.CODEC_LZ4 ]) def test_read_write_serde_v2(compression_type): + maybe_skip_unsupported_compression(compression_type) builder = DefaultRecordBatchBuilder( magic=2, compression_type=compression_type, is_transactional=1, producer_id=123456, producer_epoch=123, base_sequence=9999, @@ -186,6 +189,8 @@ def test_default_batch_size_limit(): ]) @pytest.mark.parametrize("magic", [0, 1]) def test_unavailable_codec(magic, compression_type, name, checker_name): + if not getattr(kafka.codec, checker_name)(): + pytest.skip('%s compression_type not installed' % (compression_type,)) builder = DefaultRecordBatchBuilder( magic=2, compression_type=compression_type, is_transactional=0, producer_id=-1, producer_epoch=-1, base_sequence=-1, diff --git a/test/record/test_legacy_records.py b/test/record/test_legacy_records.py index b15b53704..c692d35a1 100644 --- a/test/record/test_legacy_records.py +++ b/test/record/test_legacy_records.py @@ -10,6 +10,8 @@ import kafka.codec from kafka.errors import UnsupportedCodecError +from test.testutil import maybe_skip_unsupported_compression + @pytest.mark.parametrize("magic", [0, 1]) def test_read_write_serde_v0_v1_no_compression(magic): @@ -39,6 +41,7 @@ def test_read_write_serde_v0_v1_no_compression(magic): ]) @pytest.mark.parametrize("magic", [0, 1]) def test_read_write_serde_v0_v1_with_compression(compression_type, magic): + maybe_skip_unsupported_compression(compression_type) builder = LegacyRecordBatchBuilder( magic=magic, compression_type=compression_type, batch_size=9999999) for offset in range(10): @@ -179,6 +182,7 @@ def test_legacy_batch_size_limit(magic): ]) @pytest.mark.parametrize("magic", [0, 1]) def test_unavailable_codec(magic, compression_type, name, checker_name): + maybe_skip_unsupported_compression(compression_type) builder = LegacyRecordBatchBuilder( magic=magic, compression_type=compression_type, batch_size=1024) builder.append(0, timestamp=None, key=None, value=b"M") diff --git a/test/record/test_records.py b/test/record/test_records.py index cab95922d..dc9c95ff8 100644 --- a/test/record/test_records.py +++ b/test/record/test_records.py @@ -4,6 +4,8 @@ from kafka.record import MemoryRecords, MemoryRecordsBuilder from kafka.errors import CorruptRecordException +from test.testutil import maybe_skip_unsupported_compression + # This is real live data from Kafka 11 broker record_batch_data_v2 = [ # First Batch value == "123" @@ -179,6 +181,7 @@ def test_memory_records_corrupt(): @pytest.mark.parametrize("compression_type", [0, 1, 2, 3]) @pytest.mark.parametrize("magic", [0, 1, 2]) def test_memory_records_builder(magic, compression_type): + maybe_skip_unsupported_compression(compression_type) builder = MemoryRecordsBuilder( magic=magic, compression_type=compression_type, batch_size=1024 * 10) base_size = builder.size_in_bytes() # V2 has a header before diff --git a/test/test_conn.py b/test/test_conn.py index 6af01498f..b5deb748c 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -11,6 +11,7 @@ import pytest from kafka.conn import BrokerConnection, ConnectionStates, collect_hosts +from kafka.future import Future from kafka.protocol.api import RequestHeader from kafka.protocol.group import HeartbeatResponse from kafka.protocol.metadata import MetadataRequest @@ -69,8 +70,10 @@ def test_connect(_socket, conn, states): assert conn.state is state -def test_api_versions_check(_socket): +def test_api_versions_check(_socket, mocker): conn = BrokerConnection('localhost', 9092, socket.AF_INET) + mocker.patch.object(conn, '_send', return_value=Future()) + mocker.patch.object(conn, 'recv', return_value=[]) assert conn._api_versions_future is None conn.connect() assert conn._api_versions_future is not None diff --git a/test/test_consumer_group.py b/test/test_consumer_group.py index 9334a4fd1..b2908c757 100644 --- a/test/test_consumer_group.py +++ b/test/test_consumer_group.py @@ -68,7 +68,8 @@ def consumer_thread(i): num_consumers = 4 for i in range(num_consumers): - t = threading.Thread(target=consumer_thread, args=(i,), daemon=True) + t = threading.Thread(target=consumer_thread, args=(i,)) + t.daemon = True t.start() threads[i] = t diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 35749f84d..eac1a1e62 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -25,14 +25,25 @@ from kafka.structs import OffsetAndMetadata, TopicPartition from kafka.util import WeakMethod - @pytest.fixture -def client(conn): - return KafkaClient(api_version=(0, 9)) +def client(conn, mocker): + cli = KafkaClient(api_version=(0, 9)) + mocker.patch.object(cli, '_init_connect', return_value=True) + try: + yield cli + finally: + cli._close() @pytest.fixture -def coordinator(client): - return ConsumerCoordinator(client, SubscriptionState(), Metrics()) +def coordinator(client, mocker): + metrics = Metrics() + coord = ConsumerCoordinator(client, SubscriptionState(), metrics) + try: + yield coord + finally: + mocker.patch.object(coord, 'coordinator_unknown', return_value=True) # avoid attempting to leave group during close() + coord.close(timeout_ms=0) + metrics.close() def test_init(client, coordinator): @@ -55,6 +66,7 @@ def test_autocommit_enable_api_version(conn, api_version): assert coordinator.config['enable_auto_commit'] is False else: assert coordinator.config['enable_auto_commit'] is True + coordinator.close() def test_protocol_type(coordinator): @@ -117,6 +129,7 @@ def test_pattern_subscription(conn, api_version): else: assert set(coordinator._subscription.assignment.keys()) == {TopicPartition('foo1', 0), TopicPartition('foo2', 0)} + coordinator.close() def test_lookup_assignor(coordinator): @@ -398,6 +411,7 @@ def test_maybe_auto_commit_offsets_sync(mocker, api_version, group_id, enable, assert commit_sync.call_count == (1 if commit_offsets else 0) assert mock_warn.call_count == (1 if warn else 0) assert mock_exc.call_count == (1 if exc else 0) + coordinator.close() @pytest.fixture diff --git a/test/test_producer.py b/test/test_producer.py index 598661aab..3d1de06d3 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -8,7 +8,7 @@ from kafka import KafkaConsumer, KafkaProducer, TopicPartition from kafka.producer.buffer import SimpleBufferPool -from test.testutil import env_kafka_version, random_string +from test.testutil import env_kafka_version, random_string, maybe_skip_unsupported_compression def test_buffer_pool(): @@ -44,6 +44,7 @@ def consumer_factory(**kwargs): @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") @pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd']) def test_end_to_end(kafka_broker, compression): + maybe_skip_unsupported_compression(compression) if compression == 'lz4': if env_kafka_version() < (0, 8, 2): pytest.skip('LZ4 requires 0.8.2') @@ -104,6 +105,7 @@ def test_kafka_producer_gc_cleanup(): @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") @pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd']) def test_kafka_producer_proper_record_metadata(kafka_broker, compression): + maybe_skip_unsupported_compression(compression) if compression == 'zstd' and env_kafka_version() < (2, 1, 0): pytest.skip('zstd requires 2.1.0 or more') connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) diff --git a/test/test_subscription_state.py b/test/test_subscription_state.py index bb2c81bff..773606525 100644 --- a/test/test_subscription_state.py +++ b/test/test_subscription_state.py @@ -44,8 +44,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(s, TopicPartitionState) for s in six.itervalues(s.assignment)]) - assert all([not s.has_valid_position for s in six.itervalues(s.assignment)]) + 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)]) def test_change_subscription_after_assignment(): diff --git a/test/testutil.py b/test/testutil.py index dd4e267a8..b5dab1c02 100644 --- a/test/testutil.py +++ b/test/testutil.py @@ -6,6 +6,10 @@ import string import time +import pytest + +import kafka.codec + def special_to_underscore(string, _matcher=re.compile(r'[^a-zA-Z0-9_]+')): return _matcher.sub('_', string) @@ -36,6 +40,18 @@ def assert_message_count(messages, num_messages): assert len(unique_messages) == num_messages, 'Expected %d unique messages, got %d' % (num_messages, len(unique_messages)) +def maybe_skip_unsupported_compression(compression_type): + codecs = {1: 'gzip', 2: 'snappy', 3: 'lz4', 4: 'zstd'} + if not compression_type: + return + elif compression_type in codecs: + compression_type = codecs[compression_type] + + checker = getattr(kafka.codec, 'has_' + compression_type, None) + if checker and not checker(): + pytest.skip("Compression libraries not installed for %s" % (compression_type,)) + + class Timer(object): def __enter__(self): self.start = time.time() From 0024227dc6b90ddb94559d9f79c389cac28afd03 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 3 Apr 2025 15:38:53 -0700 Subject: [PATCH 106/205] Raise UnsupportedVersionError from coordinator (#2579) --- kafka/consumer/group.py | 3 ++- kafka/coordinator/base.py | 11 ++++++++++- kafka/coordinator/consumer.py | 12 ++++++++---- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 58284a7a9..6e6a88724 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -972,7 +972,8 @@ def unsubscribe(self): # are committed since there will be no following rebalance self._coordinator.maybe_auto_commit_offsets_now() self._subscription.unsubscribe() - self._coordinator.maybe_leave_group() + if self.config['api_version'] >= (0, 9): + self._coordinator.maybe_leave_group() self._client.cluster.need_all_topic_metadata = False self._client.set_topics([]) log.debug("Unsubscribed all topics or patterns and assigned partitions") diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 97ba4fa28..410e92fc9 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -395,12 +395,16 @@ def ensure_active_group(self, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms is not None """ + if self.config['api_version'] < (0, 9): + raise Errors.UnsupportedVersionError('Group Coordinator APIs require 0.9+ broker') inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) self._start_heartbeat_thread() self.join_group(timeout_ms=inner_timeout_ms()) def join_group(self, timeout_ms=None): + if self.config['api_version'] < (0, 9): + raise Errors.UnsupportedVersionError('Group Coordinator APIs require 0.9+ broker') inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') while self.need_rejoin(): self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) @@ -763,6 +767,8 @@ def request_rejoin(self): self.rejoin_needed = True def _start_heartbeat_thread(self): + if self.config['api_version'] < (0, 9): + raise Errors.UnsupportedVersionError('Heartbeat APIs require 0.9+ broker') with self._lock: if self._heartbeat_thread is None: log.info('Starting new heartbeat thread') @@ -794,10 +800,13 @@ def close(self, timeout_ms=None): """Close the coordinator, leave the current group, and reset local generation / member_id""" self._close_heartbeat_thread(timeout_ms=timeout_ms) - self.maybe_leave_group(timeout_ms=timeout_ms) + if self.config['api_version'] >= (0, 9): + self.maybe_leave_group(timeout_ms=timeout_ms) 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 diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 873b1128c..773df38bd 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -494,7 +494,8 @@ def commit_offsets_async(self, offsets, callback=None): return future def _do_commit_offsets_async(self, offsets, callback=None): - assert self.config['api_version'] >= (0, 8, 1), 'Unsupported Broker API' + if self.config['api_version'] < (0, 8, 1): + raise Errors.UnsupportedVersionError('OffsetCommitRequest requires 0.8.1+ broker') assert all(map(lambda k: isinstance(k, TopicPartition), offsets)) assert all(map(lambda v: isinstance(v, OffsetAndMetadata), offsets.values())) @@ -516,7 +517,8 @@ def commit_offsets_sync(self, offsets, timeout_ms=None): Raises error on failure """ - assert self.config['api_version'] >= (0, 8, 1), 'Unsupported Broker API' + if self.config['api_version'] < (0, 8, 1): + raise Errors.UnsupportedVersionError('OffsetCommitRequest requires 0.8.1+ broker') assert all(map(lambda k: isinstance(k, TopicPartition), offsets)) assert all(map(lambda v: isinstance(v, OffsetAndMetadata), offsets.values())) @@ -573,7 +575,8 @@ def _send_offset_commit_request(self, offsets): Returns: Future: indicating whether the commit was successful or not """ - assert self.config['api_version'] >= (0, 8, 1), 'Unsupported Broker API' + if self.config['api_version'] < (0, 8, 1): + raise Errors.UnsupportedVersionError('OffsetCommitRequest requires 0.8.1+ broker') assert all(map(lambda k: isinstance(k, TopicPartition), offsets)) assert all(map(lambda v: isinstance(v, OffsetAndMetadata), offsets.values())) @@ -761,7 +764,8 @@ def _send_offset_fetch_request(self, partitions): Returns: Future: resolves to dict of offsets: {TopicPartition: OffsetAndMetadata} """ - assert self.config['api_version'] >= (0, 8, 1), 'Unsupported Broker API' + if self.config['api_version'] < (0, 8, 1): + raise Errors.UnsupportedVersionError('OffsetFetchRequest requires 0.8.1+ broker') assert all(map(lambda k: isinstance(k, TopicPartition), partitions)) if not partitions: return Future().success({}) From 56eb39dbf5a70211527b6f0aa40c2ea3e01831a8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 09:52:31 -0700 Subject: [PATCH 107/205] Drop unused kafka.producer.buffer / SimpleBufferPool (#2580) --- kafka/producer/buffer.py | 115 --------------------------- kafka/producer/kafka.py | 29 +++---- kafka/producer/record_accumulator.py | 33 +------- test/test_producer.py | 13 --- test/test_sender.py | 3 +- 5 files changed, 14 insertions(+), 179 deletions(-) delete mode 100644 kafka/producer/buffer.py diff --git a/kafka/producer/buffer.py b/kafka/producer/buffer.py deleted file mode 100644 index 100801700..000000000 --- a/kafka/producer/buffer.py +++ /dev/null @@ -1,115 +0,0 @@ -from __future__ import absolute_import, division - -import collections -import io -import threading -import time - -from kafka.metrics.stats import Rate - -import kafka.errors as Errors - - -class SimpleBufferPool(object): - """A simple pool of BytesIO objects with a weak memory ceiling.""" - def __init__(self, memory, poolable_size, metrics=None, metric_group_prefix='producer-metrics'): - """Create a new buffer pool. - - Arguments: - memory (int): maximum memory that this buffer pool can allocate - poolable_size (int): memory size per buffer to cache in the free - list rather than deallocating - """ - self._poolable_size = poolable_size - self._lock = threading.RLock() - - buffers = int(memory / poolable_size) if poolable_size else 0 - self._free = collections.deque([io.BytesIO() for _ in range(buffers)]) - - self._waiters = collections.deque() - self.wait_time = None - if metrics: - self.wait_time = metrics.sensor('bufferpool-wait-time') - self.wait_time.add(metrics.metric_name( - 'bufferpool-wait-ratio', metric_group_prefix, - 'The fraction of time an appender waits for space allocation.'), - Rate()) - - def allocate(self, size, max_time_to_block_ms): - """ - Allocate a buffer of the given size. This method blocks if there is not - enough memory and the buffer pool is configured with blocking mode. - - Arguments: - size (int): The buffer size to allocate in bytes [ignored] - max_time_to_block_ms (int): The maximum time in milliseconds to - block for buffer memory to be available - - Returns: - io.BytesIO - """ - with self._lock: - # check if we have a free buffer of the right size pooled - if self._free: - return self._free.popleft() - - elif self._poolable_size == 0: - return io.BytesIO() - - else: - # we are out of buffers and will have to block - buf = None - more_memory = threading.Condition(self._lock) - self._waiters.append(more_memory) - # loop over and over until we have a buffer or have reserved - # enough memory to allocate one - while buf is None: - start_wait = time.time() - more_memory.wait(max_time_to_block_ms / 1000.0) - end_wait = time.time() - if self.wait_time: - self.wait_time.record(end_wait - start_wait) - - if self._free: - buf = self._free.popleft() - else: - self._waiters.remove(more_memory) - raise Errors.KafkaTimeoutError( - "Failed to allocate memory within the configured" - " max blocking time") - - # remove the condition for this thread to let the next thread - # in line start getting memory - removed = self._waiters.popleft() - assert removed is more_memory, 'Wrong condition' - - # signal any additional waiters if there is more memory left - # over for them - if self._free and self._waiters: - self._waiters[0].notify() - - # unlock and return the buffer - return buf - - def deallocate(self, buf): - """ - Return buffers to the pool. If they are of the poolable size add them - to the free list, otherwise just mark the memory as free. - - Arguments: - buffer_ (io.BytesIO): The buffer to return - """ - with self._lock: - # BytesIO.truncate here makes the pool somewhat pointless - # but we stick with the BufferPool API until migrating to - # bytesarray / memoryview. The buffer we return must not - # expose any prior data on read(). - buf.truncate(0) - self._free.append(buf) - if self._waiters: - self._waiters[0].notify() - - def queued(self): - """The number of threads blocked waiting on memory.""" - with self._lock: - return len(self._waiters) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 8da14af1c..df86e907e 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -6,6 +6,7 @@ import socket import threading import time +import warnings import weakref from kafka.vendor import six @@ -72,11 +73,6 @@ class KafkaProducer(object): can lead to fewer, more efficient requests when not under maximal load at the cost of a small amount of latency. - The buffer_memory controls the total amount of memory available to the - producer for buffering. If records are sent faster than they can be - transmitted to the server then this buffer space will be exhausted. When - the buffer space is exhausted additional send calls will block. - The key_serializer and value_serializer instruct how to turn the key and value objects the user provides into bytes. @@ -166,12 +162,6 @@ class KafkaProducer(object): messages with the same key are assigned to the same partition. When a key is None, the message is delivered to a random partition (filtered to partitions with available leaders only, if possible). - buffer_memory (int): The total bytes of memory the producer should use - to buffer records waiting to be sent to the server. If records are - sent faster than they can be delivered to the server the producer - will block up to max_block_ms, raising an exception on timeout. - In the current implementation, this setting is an approximation. - Default: 33554432 (32MB) connections_max_idle_ms: Close idle connections after the number of milliseconds specified by this config. The broker closes idle connections after connections.max.idle.ms, so this avoids hitting @@ -319,7 +309,6 @@ class KafkaProducer(object): 'batch_size': 16384, 'linger_ms': 0, 'partitioner': DefaultPartitioner(), - 'buffer_memory': 33554432, 'connections_max_idle_ms': 9 * 60 * 1000, 'max_block_ms': 60000, 'max_request_size': 1048576, @@ -361,6 +350,8 @@ class KafkaProducer(object): 'kafka_client': KafkaClient, } + DEPRECATED_CONFIGS = ('buffer_memory',) + _COMPRESSORS = { 'gzip': (has_gzip, LegacyRecordBatchBuilder.CODEC_GZIP), 'snappy': (has_snappy, LegacyRecordBatchBuilder.CODEC_SNAPPY), @@ -376,6 +367,11 @@ def __init__(self, **configs): if key in configs: self.config[key] = configs.pop(key) + for key in self.DEPRECATED_CONFIGS: + if key in configs: + configs.pop(key) + warnings.warn('Deprecated Producer config: %s' % (key,), DeprecationWarning) + # Only check for extra config keys in top-level class assert not configs, 'Unrecognized configs: %s' % (configs,) @@ -640,9 +636,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest tp = TopicPartition(topic, partition) log.debug("Sending (key=%r value=%r headers=%r) to %s", key, value, headers, tp) result = self._accumulator.append(tp, timestamp_ms, - key_bytes, value_bytes, headers, - self.config['max_block_ms'], - estimated_size=message_size) + key_bytes, value_bytes, headers) future, batch_is_full, new_batch_created = result if batch_is_full or new_batch_created: log.debug("Waking up the sender since %s is either full or" @@ -697,11 +691,6 @@ def _ensure_valid_record_size(self, size): "The message is %d bytes when serialized which is larger than" " the maximum request size you have configured with the" " max_request_size configuration" % (size,)) - if size > self.config['buffer_memory']: - raise Errors.MessageSizeTooLargeError( - "The message is %d bytes when serialized which is larger than" - " the total memory buffer you have configured with the" - " buffer_memory configuration." % (size,)) def _wait_on_metadata(self, topic, max_wait): """ diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 4f08b8c08..6e7fa60f7 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -7,7 +7,6 @@ import time import kafka.errors as Errors -from kafka.producer.buffer import SimpleBufferPool from kafka.producer.future import FutureRecordMetadata, FutureProduceResult from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition @@ -36,7 +35,7 @@ def get(self): class ProducerBatch(object): - def __init__(self, tp, records, buffer): + def __init__(self, tp, records): self.max_record_size = 0 now = time.time() self.created = now @@ -48,7 +47,6 @@ def __init__(self, tp, records, buffer): self.topic_partition = tp self.produce_future = FutureProduceResult(tp) self._retry = False - self._buffer = buffer # We only save it, we don't write to it @property def record_count(self): @@ -123,9 +121,6 @@ def in_retry(self): def set_retry(self): self._retry = True - def buffer(self): - return self._buffer - def __str__(self): return 'ProducerBatch(topic_partition=%s, record_count=%d)' % ( self.topic_partition, self.records.next_offset()) @@ -145,12 +140,6 @@ class RecordAccumulator(object): A small batch size will make batching less common and may reduce throughput (a batch size of zero will disable batching entirely). Default: 16384 - buffer_memory (int): The total bytes of memory the producer should use - to buffer records waiting to be sent to the server. If records are - sent faster than they can be delivered to the server the producer - will block up to max_block_ms, raising an exception on timeout. - In the current implementation, this setting is an approximation. - Default: 33554432 (32MB) compression_attrs (int): The compression type for all data generated by the producer. Valid values are gzip(1), snappy(2), lz4(3), or none(0). @@ -168,7 +157,6 @@ class RecordAccumulator(object): all retries in a short period of time. Default: 100 """ DEFAULT_CONFIG = { - 'buffer_memory': 33554432, 'batch_size': 16384, 'compression_attrs': 0, 'linger_ms': 0, @@ -189,18 +177,13 @@ def __init__(self, **configs): self._appends_in_progress = AtomicInteger() self._batches = collections.defaultdict(collections.deque) # TopicPartition: [ProducerBatch] self._tp_locks = {None: threading.Lock()} # TopicPartition: Lock, plus a lock to add entries - self._free = SimpleBufferPool(self.config['buffer_memory'], - self.config['batch_size'], - metrics=self.config['metrics'], - metric_group_prefix=self.config['metric_group_prefix']) self._incomplete = IncompleteProducerBatches() # The following variables should only be accessed by the sender thread, # so we don't need to protect them w/ locking. self.muted = set() self._drain_index = 0 - def append(self, tp, timestamp_ms, key, value, headers, max_time_to_block_ms, - estimated_size=0): + def append(self, tp, timestamp_ms, key, value, headers): """Add a record to the accumulator, return the append result. The append result will contain the future metadata, and flag for @@ -213,8 +196,6 @@ def append(self, tp, timestamp_ms, key, value, headers, max_time_to_block_ms, key (bytes): The key for the record value (bytes): The value for the record headers (List[Tuple[str, bytes]]): The header fields for the record - max_time_to_block_ms (int): The maximum time in milliseconds to - block for buffer memory to be available Returns: tuple: (future, batch_is_full, new_batch_created) @@ -240,9 +221,6 @@ def append(self, tp, timestamp_ms, key, value, headers, max_time_to_block_ms, batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False - size = max(self.config['batch_size'], estimated_size) - log.debug("Allocating a new %d byte message buffer for %s", size, tp) # trace - buf = self._free.allocate(size, max_time_to_block_ms) with self._tp_locks[tp]: # Need to check if producer is closed again after grabbing the # dequeue lock. @@ -254,7 +232,6 @@ def append(self, tp, timestamp_ms, key, value, headers, max_time_to_block_ms, if future is not None: # Somebody else found us a batch, return the one we # waited for! Hopefully this doesn't happen often... - self._free.deallocate(buf) batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False @@ -264,7 +241,7 @@ def append(self, tp, timestamp_ms, key, value, headers, max_time_to_block_ms, self.config['batch_size'] ) - batch = ProducerBatch(tp, records, buf) + batch = ProducerBatch(tp, records) future = batch.try_append(timestamp_ms, key, value, headers) if not future: raise Exception() @@ -384,7 +361,6 @@ def ready(self, cluster): unknown_leaders_exist = False now = time.time() - exhausted = bool(self._free.queued() > 0) # several threads are accessing self._batches -- to simplify # concurrent access, we iterate over a snapshot of partitions # and lock each partition separately as needed @@ -414,7 +390,7 @@ def ready(self, cluster): full = bool(len(dq) > 1 or batch.records.is_full()) expired = bool(waited_time >= time_to_wait) - sendable = (full or expired or exhausted or self._closed or + sendable = (full or expired or self._closed or self._flush_in_progress()) if sendable and not backing_off: @@ -506,7 +482,6 @@ def drain(self, cluster, nodes, max_size): def deallocate(self, batch): """Deallocate the record batch.""" self._incomplete.remove(batch) - self._free.deallocate(batch.buffer()) def _flush_in_progress(self): """Are there any threads currently waiting on a flush?""" diff --git a/test/test_producer.py b/test/test_producer.py index 3d1de06d3..069362f26 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -7,22 +7,9 @@ import pytest from kafka import KafkaConsumer, KafkaProducer, TopicPartition -from kafka.producer.buffer import SimpleBufferPool from test.testutil import env_kafka_version, random_string, maybe_skip_unsupported_compression -def test_buffer_pool(): - pool = SimpleBufferPool(1000, 1000) - - buf1 = pool.allocate(1000, 1000) - message = ''.join(map(str, range(100))) - buf1.write(message.encode('utf-8')) - pool.deallocate(buf1) - - buf2 = pool.allocate(1000, 1000) - assert buf2.read() == b'' - - @contextmanager def producer_factory(**kwargs): producer = KafkaProducer(**kwargs) diff --git a/test/test_sender.py b/test/test_sender.py index 3da1a9f42..1656bbfe9 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -42,10 +42,9 @@ def sender(client, accumulator, metrics, mocker): def test_produce_request(sender, mocker, api_version, produce_version): sender._client._api_versions = BROKER_API_VERSIONS[api_version] tp = TopicPartition('foo', 0) - buffer = io.BytesIO() records = MemoryRecordsBuilder( magic=1, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records, buffer) + batch = ProducerBatch(tp, records) records.close() produce_request = sender._produce_request(0, 0, 0, [batch]) assert isinstance(produce_request, ProduceRequest[produce_version]) From c52f25a1589bc38ed05224f869147294bb2a549f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 11:01:47 -0700 Subject: [PATCH 108/205] Pass metrics_enabled=False to disable metrics (#2581) --- kafka/consumer/fetcher.py | 23 ++++++++++++++++------- kafka/consumer/group.py | 27 +++++++++++++++++---------- kafka/coordinator/base.py | 19 +++++++++++++------ kafka/coordinator/consumer.py | 15 ++++++++++----- kafka/producer/kafka.py | 27 ++++++++++++++++++--------- kafka/producer/record_accumulator.py | 2 -- kafka/producer/sender.py | 21 ++++++++++++++------- test/conftest.py | 23 +++++++++++++++++++++++ test/test_coordinator.py | 16 ++++++---------- test/test_fetcher.py | 5 ++--- test/test_sender.py | 13 +------------ 11 files changed, 120 insertions(+), 71 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 29c2a7182..508e35a0b 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -56,12 +56,13 @@ class Fetcher(six.Iterator): 'max_partition_fetch_bytes': 1048576, 'max_poll_records': sys.maxsize, 'check_crcs': True, + 'metrics': None, 'metric_group_prefix': 'consumer', 'retry_backoff_ms': 100, 'enable_incremental_fetch_sessions': True, } - def __init__(self, client, subscriptions, metrics, **configs): + def __init__(self, client, subscriptions, **configs): """Initialize a Kafka Message Fetcher. Keyword Arguments: @@ -111,7 +112,10 @@ def __init__(self, client, subscriptions, metrics, **configs): self._next_partition_records = None # Holds a single PartitionRecords until fully consumed self._iterator = None self._fetch_futures = collections.deque() - self._sensors = FetchManagerMetrics(metrics, self.config['metric_group_prefix']) + if self.config['metrics']: + self._sensors = FetchManagerMetrics(self.config['metrics'], self.config['metric_group_prefix']) + else: + self._sensors = None self._isolation_level = READ_UNCOMMITTED self._session_handlers = {} self._nodes_with_pending_fetch_requests = set() @@ -391,7 +395,7 @@ def _append(self, drained, part, max_records, update_offsets): # when each message is yielded). There may be edge cases where we re-fetch records # that we'll end up skipping, but for now we'll live with that. highwater = self._subscriptions.assignment[tp].highwater - if highwater is not None: + if highwater is not None and self._sensors: self._sensors.records_fetch_lag.record(highwater - part.next_fetch_offset) if update_offsets or not part_records: # TODO: save leader_epoch @@ -705,7 +709,10 @@ def _handle_fetch_response(self, node_id, fetch_offsets, send_time, response): partitions = set([TopicPartition(topic, partition_data[0]) for topic, partitions in response.topics for partition_data in partitions]) - metric_aggregator = FetchResponseMetricAggregator(self._sensors, partitions) + if self._sensors: + metric_aggregator = FetchResponseMetricAggregator(self._sensors, partitions) + else: + metric_aggregator = None for topic, partitions in response.topics: for partition_data in partitions: @@ -719,7 +726,8 @@ def _handle_fetch_response(self, node_id, fetch_offsets, send_time, response): ) self._completed_fetches.append(completed_fetch) - self._sensors.fetch_latency.record((time.time() - send_time) * 1000) + if self._sensors: + self._sensors.fetch_latency.record((time.time() - send_time) * 1000) self._nodes_with_pending_fetch_requests.remove(node_id) def _handle_fetch_error(self, node_id, exception): @@ -816,7 +824,7 @@ def _parse_fetched_data(self, completed_fetch): raise error_type('Unexpected error while fetching data') finally: - if parsed_records is None: + if parsed_records is None and completed_fetch.metric_aggregator: completed_fetch.metric_aggregator.record(tp, 0, 0) if error_type is not Errors.NoError: @@ -873,7 +881,8 @@ def __bool__(self): def drain(self): if self.record_iterator is not None: self.record_iterator = None - self.metric_aggregator.record(self.topic_partition, self.bytes_read, self.records_read) + if self.metric_aggregator: + self.metric_aggregator.record(self.topic_partition, self.bytes_read, self.records_read) self.on_drain(self) def take(self, n=None): diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 6e6a88724..4a39dc135 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -234,6 +234,7 @@ class KafkaConsumer(six.Iterator): metric_reporters (list): A list of classes to use as metrics reporters. Implementing the AbstractMetricsReporter interface allows plugging in classes that will be notified of new metric creation. Default: [] + metrics_enabled (bool): Whether to track metrics on this instance. Default True. metrics_num_samples (int): The number of samples maintained to compute metrics. Default: 2 metrics_sample_window_ms (int): The maximum age in milliseconds of @@ -315,6 +316,7 @@ class KafkaConsumer(six.Iterator): 'api_version_auto_timeout_ms': 2000, 'connections_max_idle_ms': 9 * 60 * 1000, 'metric_reporters': [], + 'metrics_enabled': True, 'metrics_num_samples': 2, 'metrics_sample_window_ms': 30000, 'metric_group_prefix': 'consumer', @@ -358,13 +360,15 @@ def __init__(self, *topics, **configs): "fetch_max_wait_ms ({})." .format(connections_max_idle_ms, request_timeout_ms, fetch_max_wait_ms)) - metrics_tags = {'client-id': self.config['client_id']} - metric_config = MetricConfig(samples=self.config['metrics_num_samples'], - time_window_ms=self.config['metrics_sample_window_ms'], - tags=metrics_tags) - reporters = [reporter() for reporter in self.config['metric_reporters']] - self._metrics = Metrics(metric_config, reporters) - # TODO _metrics likely needs to be passed to KafkaClient, etc. + if self.config['metrics_enabled']: + metrics_tags = {'client-id': self.config['client_id']} + metric_config = MetricConfig(samples=self.config['metrics_num_samples'], + time_window_ms=self.config['metrics_sample_window_ms'], + tags=metrics_tags) + reporters = [reporter() for reporter in self.config['metric_reporters']] + self._metrics = Metrics(metric_config, reporters) + else: + self._metrics = None # api_version was previously a str. Accept old format for now if isinstance(self.config['api_version'], str): @@ -402,9 +406,9 @@ def __init__(self, *topics, **configs): self._subscription = SubscriptionState(self.config['auto_offset_reset']) self._fetcher = Fetcher( - self._client, self._subscription, self._metrics, **self.config) + self._client, self._subscription, metrics=self._metrics, **self.config) self._coordinator = ConsumerCoordinator( - self._client, self._subscription, self._metrics, + self._client, self._subscription, metrics=self._metrics, assignors=self.config['partition_assignment_strategy'], **self.config) self._closed = False @@ -485,7 +489,8 @@ def close(self, autocommit=True, timeout_ms=None): log.debug("Closing the KafkaConsumer.") self._closed = True self._coordinator.close(autocommit=autocommit, timeout_ms=timeout_ms) - self._metrics.close() + if self._metrics: + self._metrics.close() self._client.close() try: self.config['key_deserializer'].close() @@ -989,6 +994,8 @@ def metrics(self, raw=False): This is an unstable interface. It may change in future releases without warning. """ + if not self._metrics: + return if raw: return self._metrics.metrics.copy() diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 410e92fc9..0c238fde8 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -84,10 +84,11 @@ class BaseCoordinator(object): 'max_poll_interval_ms': 300000, 'retry_backoff_ms': 100, 'api_version': (0, 10, 1), + 'metrics': None, 'metric_group_prefix': '', } - def __init__(self, client, metrics, **configs): + def __init__(self, client, **configs): """ Keyword Arguments: group_id (str): name of the consumer group to join for dynamic @@ -130,8 +131,11 @@ def __init__(self, client, metrics, **configs): self.coordinator_id = None self._find_coordinator_future = None self._generation = Generation.NO_GENERATION - self.sensors = GroupCoordinatorMetrics(self.heartbeat, metrics, - self.config['metric_group_prefix']) + if self.config['metrics']: + self._sensors = GroupCoordinatorMetrics(self.heartbeat, self.config['metrics'], + self.config['metric_group_prefix']) + else: + self._sensors = None @abc.abstractmethod def protocol_type(self): @@ -531,7 +535,8 @@ def _handle_join_group_response(self, future, send_time, response): if error_type is Errors.NoError: log.debug("Received successful JoinGroup response for group %s: %s", self.group_id, response) - self.sensors.join_latency.record((time.time() - send_time) * 1000) + if self._sensors: + self._sensors.join_latency.record((time.time() - send_time) * 1000) with self._lock: if self.state is not MemberState.REBALANCING: # if the consumer was woken up before a rebalance completes, @@ -650,7 +655,8 @@ def _send_sync_group_request(self, request): def _handle_sync_group_response(self, future, send_time, response): error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - self.sensors.sync_latency.record((time.time() - send_time) * 1000) + if self._sensors: + self._sensors.sync_latency.record((time.time() - send_time) * 1000) future.success(response.member_assignment) return @@ -856,7 +862,8 @@ def _send_heartbeat_request(self): return future def _handle_heartbeat_response(self, future, send_time, response): - self.sensors.heartbeat_latency.record((time.time() - send_time) * 1000) + if self._sensors: + self._sensors.heartbeat_latency.record((time.time() - send_time) * 1000) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: log.debug("Received successful heartbeat response for group %s", diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 773df38bd..4bc7ba9cb 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -39,10 +39,11 @@ class ConsumerCoordinator(BaseCoordinator): 'retry_backoff_ms': 100, 'api_version': (0, 10, 1), 'exclude_internal_topics': True, + 'metrics': None, 'metric_group_prefix': 'consumer' } - def __init__(self, client, subscription, metrics, **configs): + def __init__(self, client, subscription, **configs): """Initialize the coordination manager. Keyword Arguments: @@ -78,7 +79,7 @@ def __init__(self, client, subscription, metrics, **configs): True the only way to receive records from an internal topic is subscribing to it. Requires 0.10+. Default: True """ - super(ConsumerCoordinator, self).__init__(client, metrics, **configs) + super(ConsumerCoordinator, self).__init__(client, **configs) self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: @@ -120,8 +121,11 @@ def __init__(self, client, subscription, metrics, **configs): else: self.next_auto_commit_deadline = time.time() + self.auto_commit_interval - self.consumer_sensors = ConsumerCoordinatorMetrics( - metrics, self.config['metric_group_prefix'], self._subscription) + if self.config['metrics']: + self._consumer_sensors = ConsumerCoordinatorMetrics( + self.config['metrics'], self.config['metric_group_prefix'], self._subscription) + else: + self._consumer_sensors = None self._cluster.request_update() self._cluster.add_listener(WeakMethod(self._handle_metadata_update)) @@ -686,7 +690,8 @@ def _send_offset_commit_request(self, offsets): def _handle_offset_commit_response(self, offsets, future, send_time, response): # TODO look at adding request_latency_ms to response (like java kafka) - self.consumer_sensors.commit_latency.record((time.time() - send_time) * 1000) + if self._consumer_sensors: + self._consumer_sensors.commit_latency.record((time.time() - send_time) * 1000) unauthorized_topics = set() for topic, partitions in response.topics: diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index df86e907e..f0eb37a8f 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -267,6 +267,7 @@ class KafkaProducer(object): metric_reporters (list): A list of classes to use as metrics reporters. Implementing the AbstractMetricsReporter interface allows plugging in classes that will be notified of new metric creation. Default: [] + metrics_enabled (bool): Whether to track metrics on this instance. Default True. metrics_num_samples (int): The number of samples maintained to compute metrics. Default: 2 metrics_sample_window_ms (int): The maximum age in milliseconds of @@ -336,6 +337,7 @@ class KafkaProducer(object): 'api_version': None, 'api_version_auto_timeout_ms': 2000, 'metric_reporters': [], + 'metrics_enabled': True, 'metrics_num_samples': 2, 'metrics_sample_window_ms': 30000, 'selector': selectors.DefaultSelector, @@ -393,12 +395,15 @@ def __init__(self, **configs): str(self.config['api_version']), deprecated) # Configure metrics - metrics_tags = {'client-id': self.config['client_id']} - metric_config = MetricConfig(samples=self.config['metrics_num_samples'], - time_window_ms=self.config['metrics_sample_window_ms'], - tags=metrics_tags) - reporters = [reporter() for reporter in self.config['metric_reporters']] - self._metrics = Metrics(metric_config, reporters) + if self.config['metrics_enabled']: + metrics_tags = {'client-id': self.config['client_id']} + metric_config = MetricConfig(samples=self.config['metrics_num_samples'], + time_window_ms=self.config['metrics_sample_window_ms'], + tags=metrics_tags) + reporters = [reporter() for reporter in self.config['metric_reporters']] + self._metrics = Metrics(metric_config, reporters) + else: + self._metrics = None client = self.config['kafka_client']( metrics=self._metrics, metric_group_prefix='producer', @@ -424,11 +429,12 @@ def __init__(self, **configs): self.config['compression_attrs'] = compression_attrs message_version = self._max_usable_produce_magic() - self._accumulator = RecordAccumulator(message_version=message_version, metrics=self._metrics, **self.config) + self._accumulator = RecordAccumulator(message_version=message_version, **self.config) self._metadata = client.cluster guarantee_message_order = bool(self.config['max_in_flight_requests_per_connection'] == 1) self._sender = Sender(client, self._metadata, - self._accumulator, self._metrics, + self._accumulator, + metrics=self._metrics, guarantee_message_order=guarantee_message_order, **self.config) self._sender.daemon = True @@ -524,7 +530,8 @@ def __getattr__(self, name): timeout) self._sender.force_close() - self._metrics.close() + if self._metrics: + self._metrics.close() try: self.config['key_serializer'].close() except AttributeError: @@ -773,6 +780,8 @@ def metrics(self, raw=False): This is an unstable interface. It may change in future releases without warning. """ + if not self._metrics: + return if raw: return self._metrics.metrics.copy() diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 6e7fa60f7..ba823500d 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -162,8 +162,6 @@ class RecordAccumulator(object): 'linger_ms': 0, 'retry_backoff_ms': 100, 'message_version': 0, - 'metrics': None, - 'metric_group_prefix': 'producer-metrics', } def __init__(self, **configs): diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 0e2ea577e..20af28d07 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -29,11 +29,12 @@ class Sender(threading.Thread): 'acks': 1, 'retries': 0, 'request_timeout_ms': 30000, + 'metrics': None, 'guarantee_message_order': False, 'client_id': 'kafka-python-' + __version__, } - def __init__(self, client, metadata, accumulator, metrics, **configs): + def __init__(self, client, metadata, accumulator, **configs): super(Sender, self).__init__() self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: @@ -47,7 +48,10 @@ def __init__(self, client, metadata, accumulator, metrics, **configs): self._running = True self._force_close = False self._topics_to_add = set() - self._sensors = SenderMetrics(metrics, self._client, self._metadata) + if self.config['metrics']: + self._sensors = SenderMetrics(self.config['metrics'], self._client, self._metadata) + else: + self._sensors = None def run(self): """The main run loop for the sender thread.""" @@ -123,10 +127,12 @@ def run_once(self): expired_batches = self._accumulator.abort_expired_batches( self.config['request_timeout_ms'], self._metadata) - for expired_batch in expired_batches: - self._sensors.record_errors(expired_batch.topic_partition.topic, expired_batch.record_count) - self._sensors.update_produce_request_metrics(batches_by_node) + if self._sensors: + for expired_batch in expired_batches: + self._sensors.record_errors(expired_batch.topic_partition.topic, expired_batch.record_count) + self._sensors.update_produce_request_metrics(batches_by_node) + requests = self._create_produce_requests(batches_by_node) # If we have any nodes that are ready to send + have sendable data, # poll with 0 timeout so this can immediately loop and try sending more @@ -237,7 +243,8 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star self.config['retries'] - batch.attempts - 1, error) self._accumulator.reenqueue(batch) - self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) + if self._sensors: + self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) else: if error is Errors.TopicAuthorizationFailedError: error = error(batch.topic_partition.topic) @@ -245,7 +252,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star # tell the user the result of their request batch.done(base_offset, timestamp_ms, error, log_start_offset) self._accumulator.deallocate(batch) - if error is not None: + if error is not None and self._sensors: self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) if getattr(error, 'invalid_metadata', False): diff --git a/test/conftest.py b/test/conftest.py index 4c4c503e7..ba76d6cc5 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -195,3 +195,26 @@ def _send_messages(number_range, partition=0, topic=topic, producer=kafka_produc return [msg for (msg, f) in messages_and_futures] return _send_messages + + +@pytest.fixture +def metrics(): + from kafka.metrics import Metrics + + metrics = Metrics() + try: + yield metrics + finally: + metrics.close() + + +@pytest.fixture +def client(conn, mocker): + from kafka import KafkaClient + + cli = KafkaClient(api_version=(0, 9)) + mocker.patch.object(cli, '_init_connect', return_value=True) + try: + yield cli + finally: + cli._close() diff --git a/test/test_coordinator.py b/test/test_coordinator.py index eac1a1e62..1d1a6df50 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -16,7 +16,6 @@ ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment) import kafka.errors as Errors from kafka.future import Future -from kafka.metrics import Metrics from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.commit import ( OffsetCommitRequest, OffsetCommitResponse, @@ -35,15 +34,13 @@ def client(conn, mocker): cli._close() @pytest.fixture -def coordinator(client, mocker): - metrics = Metrics() - coord = ConsumerCoordinator(client, SubscriptionState(), metrics) +def coordinator(client, metrics, mocker): + coord = ConsumerCoordinator(client, SubscriptionState(), metrics=metrics) try: yield coord finally: mocker.patch.object(coord, 'coordinator_unknown', return_value=True) # avoid attempting to leave group during close() coord.close(timeout_ms=0) - metrics.close() def test_init(client, coordinator): @@ -53,10 +50,10 @@ def test_init(client, coordinator): @pytest.mark.parametrize("api_version", [(0, 8, 0), (0, 8, 1), (0, 8, 2), (0, 9)]) -def test_autocommit_enable_api_version(conn, api_version): +def test_autocommit_enable_api_version(conn, metrics, api_version): coordinator = ConsumerCoordinator(KafkaClient(api_version=api_version), SubscriptionState(), - Metrics(), + metrics=metrics, enable_auto_commit=True, session_timeout_ms=30000, # session_timeout_ms and max_poll_interval_ms max_poll_interval_ms=30000, # should be the same to avoid KafkaConfigurationError @@ -100,10 +97,10 @@ def test_group_protocols(coordinator): @pytest.mark.parametrize('api_version', [(0, 8, 0), (0, 8, 1), (0, 8, 2), (0, 9)]) -def test_pattern_subscription(conn, api_version): +def test_pattern_subscription(conn, metrics, api_version): coordinator = ConsumerCoordinator(KafkaClient(api_version=api_version), SubscriptionState(), - Metrics(), + metrics=metrics, api_version=api_version, session_timeout_ms=10000, max_poll_interval_ms=10000) @@ -390,7 +387,6 @@ def test_maybe_auto_commit_offsets_sync(mocker, api_version, group_id, enable, mock_exc = mocker.patch('kafka.coordinator.consumer.log.exception') client = KafkaClient(api_version=api_version) coordinator = ConsumerCoordinator(client, SubscriptionState(), - Metrics(), api_version=api_version, session_timeout_ms=30000, max_poll_interval_ms=30000, diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 854f1fa98..184acc9e1 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -15,7 +15,6 @@ from kafka.consumer.subscription_state import SubscriptionState import kafka.errors as Errors from kafka.future import Future -from kafka.metrics import Metrics from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.fetch import FetchRequest, FetchResponse from kafka.protocol.list_offsets import ListOffsetsResponse @@ -43,13 +42,13 @@ def topic(): @pytest.fixture -def fetcher(client, subscription_state, topic): +def fetcher(client, metrics, subscription_state, topic): subscription_state.subscribe(topics=[topic]) assignment = [TopicPartition(topic, i) for i in range(3)] subscription_state.assign_from_subscribed(assignment) for tp in assignment: subscription_state.seek(tp, 0) - return Fetcher(client, subscription_state, Metrics()) + return Fetcher(client, subscription_state, metrics=metrics) def _build_record_batch(msgs, compression=0, offset=0, magic=2): diff --git a/test/test_sender.py b/test/test_sender.py index 1656bbfe9..b037d2b48 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -5,7 +5,6 @@ import io from kafka.client_async import KafkaClient -from kafka.metrics import Metrics from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.produce import ProduceRequest from kafka.producer.record_accumulator import RecordAccumulator, ProducerBatch @@ -14,24 +13,14 @@ from kafka.structs import TopicPartition -@pytest.fixture -def client(): - return KafkaClient(bootstrap_servers=(), api_version=(0, 9)) - - @pytest.fixture def accumulator(): return RecordAccumulator() -@pytest.fixture -def metrics(): - return Metrics() - - @pytest.fixture def sender(client, accumulator, metrics, mocker): - return Sender(client, client.cluster, accumulator, metrics) + return Sender(client, client.cluster, accumulator, metrics=metrics) @pytest.mark.parametrize(("api_version", "produce_version"), [ From 6b9076bab593651987f1b89a36506129e8d43aff Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 11:29:35 -0700 Subject: [PATCH 109/205] Use __slots__ for metrics (#2583) --- kafka/metrics/compound_stat.py | 2 ++ kafka/metrics/kafka_metric.py | 4 +++- kafka/metrics/metric_config.py | 2 ++ kafka/metrics/metric_name.py | 1 + kafka/metrics/quota.py | 2 ++ kafka/metrics/stats/avg.py | 2 ++ kafka/metrics/stats/count.py | 2 ++ kafka/metrics/stats/histogram.py | 6 ++++++ kafka/metrics/stats/max_stat.py | 2 ++ kafka/metrics/stats/min_stat.py | 2 ++ kafka/metrics/stats/percentile.py | 2 ++ kafka/metrics/stats/percentiles.py | 3 +++ kafka/metrics/stats/rate.py | 3 +++ kafka/metrics/stats/sampled_stat.py | 2 ++ kafka/metrics/stats/sensor.py | 4 ++++ kafka/metrics/stats/total.py | 2 ++ 16 files changed, 40 insertions(+), 1 deletion(-) diff --git a/kafka/metrics/compound_stat.py b/kafka/metrics/compound_stat.py index f8b149c54..f5b482da2 100644 --- a/kafka/metrics/compound_stat.py +++ b/kafka/metrics/compound_stat.py @@ -21,6 +21,8 @@ def stats(self): class NamedMeasurable(object): + __slots__ = ('_name', '_stat') + def __init__(self, metric_name, measurable_stat): self._name = metric_name self._stat = measurable_stat diff --git a/kafka/metrics/kafka_metric.py b/kafka/metrics/kafka_metric.py index 9fb8d89f1..fef684850 100644 --- a/kafka/metrics/kafka_metric.py +++ b/kafka/metrics/kafka_metric.py @@ -4,6 +4,8 @@ class KafkaMetric(object): + __slots__ = ('_metric_name', '_measurable', '_config') + # NOTE java constructor takes a lock instance def __init__(self, metric_name, measurable, config): if not metric_name: @@ -33,4 +35,4 @@ def config(self, config): def value(self, time_ms=None): if time_ms is None: time_ms = time.time() * 1000 - return self.measurable.measure(self.config, time_ms) + return self._measurable.measure(self._config, time_ms) diff --git a/kafka/metrics/metric_config.py b/kafka/metrics/metric_config.py index 2e55abfcb..7e5ead1fe 100644 --- a/kafka/metrics/metric_config.py +++ b/kafka/metrics/metric_config.py @@ -5,6 +5,8 @@ class MetricConfig(object): """Configuration values for metrics""" + __slots__ = ('quota', '_samples', 'event_window', 'time_window_ms', 'tags') + def __init__(self, quota=None, samples=2, event_window=sys.maxsize, time_window_ms=30 * 1000, tags=None): """ diff --git a/kafka/metrics/metric_name.py b/kafka/metrics/metric_name.py index 32a7e3a4b..b8ab2a3ad 100644 --- a/kafka/metrics/metric_name.py +++ b/kafka/metrics/metric_name.py @@ -38,6 +38,7 @@ class MetricName(object): # as messages are sent we record the sizes sensor.record(message_size) """ + __slots__ = ('_name', '_group', '_description', '_tags', '_hash') def __init__(self, name, group, description=None, tags=None): """ diff --git a/kafka/metrics/quota.py b/kafka/metrics/quota.py index 237edf841..36a30c44e 100644 --- a/kafka/metrics/quota.py +++ b/kafka/metrics/quota.py @@ -3,6 +3,8 @@ class Quota(object): """An upper or lower bound for metrics""" + __slots__ = ('_bound', '_upper') + def __init__(self, bound, is_upper): self._bound = bound self._upper = is_upper diff --git a/kafka/metrics/stats/avg.py b/kafka/metrics/stats/avg.py index cfbaec309..906d95573 100644 --- a/kafka/metrics/stats/avg.py +++ b/kafka/metrics/stats/avg.py @@ -7,6 +7,8 @@ class Avg(AbstractSampledStat): """ An AbstractSampledStat that maintains a simple average over its samples. """ + __slots__ = ('_initial_value', '_samples', '_current') + def __init__(self): super(Avg, self).__init__(0.0) diff --git a/kafka/metrics/stats/count.py b/kafka/metrics/stats/count.py index 6e0a2d545..6cd6d2abe 100644 --- a/kafka/metrics/stats/count.py +++ b/kafka/metrics/stats/count.py @@ -7,6 +7,8 @@ class Count(AbstractSampledStat): """ An AbstractSampledStat that maintains a simple count of what it has seen. """ + __slots__ = ('_initial_value', '_samples', '_current') + def __init__(self): super(Count, self).__init__(0.0) diff --git a/kafka/metrics/stats/histogram.py b/kafka/metrics/stats/histogram.py index ecc6c9db4..2c8afbfb3 100644 --- a/kafka/metrics/stats/histogram.py +++ b/kafka/metrics/stats/histogram.py @@ -4,6 +4,8 @@ class Histogram(object): + __slots__ = ('_hist', '_count', '_bin_scheme') + def __init__(self, bin_scheme): self._hist = [0.0] * bin_scheme.bins self._count = 0.0 @@ -40,6 +42,8 @@ def __str__(self): return '{%s}' % ','.join(values) class ConstantBinScheme(object): + __slots__ = ('_min', '_max', '_bins', '_bucket_width') + def __init__(self, bins, min_val, max_val): if bins < 2: raise ValueError('Must have at least 2 bins.') @@ -69,6 +73,8 @@ def to_bin(self, x): return int(((x - self._min) / self._bucket_width) + 1) class LinearBinScheme(object): + __slots__ = ('_bins', '_max', '_scale') + def __init__(self, num_bins, max_val): self._bins = num_bins self._max = max_val diff --git a/kafka/metrics/stats/max_stat.py b/kafka/metrics/stats/max_stat.py index 08aebddfd..9c5eeb6fd 100644 --- a/kafka/metrics/stats/max_stat.py +++ b/kafka/metrics/stats/max_stat.py @@ -5,6 +5,8 @@ class Max(AbstractSampledStat): """An AbstractSampledStat that gives the max over its samples.""" + __slots__ = ('_initial_value', '_samples', '_current') + def __init__(self): super(Max, self).__init__(float('-inf')) diff --git a/kafka/metrics/stats/min_stat.py b/kafka/metrics/stats/min_stat.py index 072106d8a..6bebe57e0 100644 --- a/kafka/metrics/stats/min_stat.py +++ b/kafka/metrics/stats/min_stat.py @@ -7,6 +7,8 @@ class Min(AbstractSampledStat): """An AbstractSampledStat that gives the min over its samples.""" + __slots__ = ('_initial_value', '_samples', '_current') + def __init__(self): super(Min, self).__init__(float(sys.maxsize)) diff --git a/kafka/metrics/stats/percentile.py b/kafka/metrics/stats/percentile.py index 3a86a84a9..75e64ce5e 100644 --- a/kafka/metrics/stats/percentile.py +++ b/kafka/metrics/stats/percentile.py @@ -2,6 +2,8 @@ class Percentile(object): + __slots__ = ('_metric_name', '_percentile') + def __init__(self, metric_name, percentile): self._metric_name = metric_name self._percentile = float(percentile) diff --git a/kafka/metrics/stats/percentiles.py b/kafka/metrics/stats/percentiles.py index 6d702e80f..c36543ffa 100644 --- a/kafka/metrics/stats/percentiles.py +++ b/kafka/metrics/stats/percentiles.py @@ -13,6 +13,9 @@ class BucketSizing(object): class Percentiles(AbstractSampledStat, AbstractCompoundStat): """A compound stat that reports one or more percentiles""" + __slots__ = ('_initial_value', '_samples', '_current', + '_percentiles', '_buckets', '_bin_scheme') + def __init__(self, size_in_bytes, bucketing, max_val, min_val=0.0, percentiles=None): super(Percentiles, self).__init__(0.0) diff --git a/kafka/metrics/stats/rate.py b/kafka/metrics/stats/rate.py index 68393fbf7..4d0ba0f27 100644 --- a/kafka/metrics/stats/rate.py +++ b/kafka/metrics/stats/rate.py @@ -37,6 +37,8 @@ class Rate(AbstractMeasurableStat): occurrences (e.g. the count of values measured over the time interval) or other such values. """ + __slots__ = ('_stat', '_unit') + def __init__(self, time_unit=TimeUnit.SECONDS, sampled_stat=None): self._stat = sampled_stat or SampledTotal() self._unit = time_unit @@ -105,6 +107,7 @@ def convert(self, time_ms): class SampledTotal(AbstractSampledStat): + __slots__ = ('_initial_value', '_samples', '_current') def __init__(self, initial_value=None): if initial_value is not None: raise ValueError('initial_value cannot be set on SampledTotal') diff --git a/kafka/metrics/stats/sampled_stat.py b/kafka/metrics/stats/sampled_stat.py index 146687916..fe8970dbf 100644 --- a/kafka/metrics/stats/sampled_stat.py +++ b/kafka/metrics/stats/sampled_stat.py @@ -22,6 +22,8 @@ class AbstractSampledStat(AbstractMeasurableStat): Subclasses of this class define different statistics measured using this basic pattern. """ + __slots__ = ('_initial_value', '_samples', '_current') + def __init__(self, initial_value): self._initial_value = initial_value self._samples = [] diff --git a/kafka/metrics/stats/sensor.py b/kafka/metrics/stats/sensor.py index 571723f97..9f7ac45f5 100644 --- a/kafka/metrics/stats/sensor.py +++ b/kafka/metrics/stats/sensor.py @@ -15,6 +15,10 @@ class Sensor(object): the `record(double)` api and would maintain a set of metrics about request sizes such as the average or max. """ + __slots__ = ('_lock', '_registry', '_name', '_parents', '_metrics', + '_stats', '_config', '_inactive_sensor_expiration_time_ms', + '_last_record_time') + def __init__(self, registry, name, parents, config, inactive_sensor_expiration_time_seconds): if not name: diff --git a/kafka/metrics/stats/total.py b/kafka/metrics/stats/total.py index 5b3bb87fd..a78e99733 100644 --- a/kafka/metrics/stats/total.py +++ b/kafka/metrics/stats/total.py @@ -5,6 +5,8 @@ class Total(AbstractMeasurableStat): """An un-windowed cumulative total maintained over all time.""" + __slots__ = ('_total') + def __init__(self, value=0.0): self._total = value From 37576e831cda518436a16017011872e543a92cd7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 12:22:32 -0700 Subject: [PATCH 110/205] Move benchmark scripts to kafka.benchmarks module (#2584) --- benchmarks/load_example.py | 66 ------- {benchmarks => kafka/benchmarks}/README.md | 0 kafka/benchmarks/__init__.py | 0 .../benchmarks}/consumer_performance.py | 79 +++------ kafka/benchmarks/load_example.py | 110 ++++++++++++ .../benchmarks}/producer_performance.py | 81 ++++----- .../benchmarks}/record_batch_compose.py | 9 +- .../benchmarks}/record_batch_read.py | 9 +- .../benchmarks}/varint_speed.py | 161 +++++++++--------- pyproject.toml | 1 + requirements-dev.txt | 1 + 11 files changed, 255 insertions(+), 262 deletions(-) delete mode 100755 benchmarks/load_example.py rename {benchmarks => kafka/benchmarks}/README.md (100%) create mode 100644 kafka/benchmarks/__init__.py rename {benchmarks => kafka/benchmarks}/consumer_performance.py (67%) mode change 100755 => 100644 create mode 100644 kafka/benchmarks/load_example.py rename {benchmarks => kafka/benchmarks}/producer_performance.py (71%) mode change 100755 => 100644 rename {benchmarks => kafka/benchmarks}/record_batch_compose.py (89%) rename {benchmarks => kafka/benchmarks}/record_batch_read.py (90%) rename {benchmarks => kafka/benchmarks}/varint_speed.py (81%) diff --git a/benchmarks/load_example.py b/benchmarks/load_example.py deleted file mode 100755 index eef113e9a..000000000 --- a/benchmarks/load_example.py +++ /dev/null @@ -1,66 +0,0 @@ -#!/usr/bin/env python -from __future__ import print_function -import threading, logging, time - -from kafka import KafkaConsumer, KafkaProducer - -msg_size = 524288 - -producer_stop = threading.Event() -consumer_stop = threading.Event() - -class Producer(threading.Thread): - big_msg = b'1' * msg_size - - def run(self): - producer = KafkaProducer(bootstrap_servers='localhost:9092') - self.sent = 0 - - while not producer_stop.is_set(): - producer.send('my-topic', self.big_msg) - self.sent += 1 - producer.flush() - - -class Consumer(threading.Thread): - - def run(self): - consumer = KafkaConsumer(bootstrap_servers='localhost:9092', - auto_offset_reset='earliest') - consumer.subscribe(['my-topic']) - self.valid = 0 - self.invalid = 0 - - for message in consumer: - if len(message.value) == msg_size: - self.valid += 1 - else: - self.invalid += 1 - - if consumer_stop.is_set(): - break - - consumer.close() - -def main(): - threads = [ - Producer(), - Consumer() - ] - - for t in threads: - t.start() - - time.sleep(10) - producer_stop.set() - consumer_stop.set() - print('Messages sent: %d' % threads[0].sent) - print('Messages recvd: %d' % threads[1].valid) - print('Messages invalid: %d' % threads[1].invalid) - -if __name__ == "__main__": - logging.basicConfig( - format='%(asctime)s.%(msecs)s:%(name)s:%(thread)d:%(levelname)s:%(process)d:%(message)s', - level=logging.INFO - ) - main() diff --git a/benchmarks/README.md b/kafka/benchmarks/README.md similarity index 100% rename from benchmarks/README.md rename to kafka/benchmarks/README.md diff --git a/kafka/benchmarks/__init__.py b/kafka/benchmarks/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/benchmarks/consumer_performance.py b/kafka/benchmarks/consumer_performance.py old mode 100755 new mode 100644 similarity index 67% rename from benchmarks/consumer_performance.py rename to kafka/benchmarks/consumer_performance.py index 9e3b6a919..c35a164c2 --- a/benchmarks/consumer_performance.py +++ b/kafka/benchmarks/consumer_performance.py @@ -4,43 +4,16 @@ from __future__ import absolute_import, print_function import argparse -import logging import pprint import sys import threading +import time import traceback -from kafka.vendor.six.moves import range - -from kafka import KafkaConsumer, KafkaProducer -from test.fixtures import KafkaFixture, ZookeeperFixture - -logging.basicConfig(level=logging.ERROR) - - -def start_brokers(n): - print('Starting {0} {1}-node cluster...'.format(KafkaFixture.kafka_version, n)) - print('-> 1 Zookeeper') - zk = ZookeeperFixture.instance() - print('---> {0}:{1}'.format(zk.host, zk.port)) - print() - - partitions = min(n, 3) - replicas = min(n, 3) - print('-> {0} Brokers [{1} partitions / {2} replicas]'.format(n, partitions, replicas)) - brokers = [ - KafkaFixture.instance(i, zk, zk_chroot='', - partitions=partitions, replicas=replicas) - for i in range(n) - ] - for broker in brokers: - print('---> {0}:{1}'.format(broker.host, broker.port)) - print() - return brokers +from kafka import KafkaConsumer class ConsumerPerformance(object): - @staticmethod def run(args): try: @@ -53,28 +26,17 @@ def run(args): pass if v == 'None': v = None + elif v == 'False': + v = False + elif v == 'True': + v = True props[k] = v - if args.brokers: - brokers = start_brokers(args.brokers) - props['bootstrap_servers'] = ['{0}:{1}'.format(broker.host, broker.port) - for broker in brokers] - print('---> bootstrap_servers={0}'.format(props['bootstrap_servers'])) - print() - - print('-> Producing records') - record = bytes(bytearray(args.record_size)) - producer = KafkaProducer(compression_type=args.fixture_compression, - **props) - for i in range(args.num_records): - producer.send(topic=args.topic, value=record) - producer.flush() - producer.close() - print('-> OK!') - print() - print('Initializing Consumer...') + props['bootstrap_servers'] = args.bootstrap_servers props['auto_offset_reset'] = 'earliest' + if 'group_id' not in props: + props['group_id'] = 'kafka-consumer-benchmark' if 'consumer_timeout_ms' not in props: props['consumer_timeout_ms'] = 10000 props['metrics_sample_window_ms'] = args.stats_interval * 1000 @@ -92,14 +54,18 @@ def run(args): print('-> OK!') print() + start_time = time.time() records = 0 for msg in consumer: records += 1 if records >= args.num_records: break - print('Consumed {0} records'.format(records)) + end_time = time.time() timer_stop.set() + timer.join() + print('Consumed {0} records'.format(records)) + print('Execution time:', end_time - start_time, 'secs') except Exception: exc_info = sys.exc_info() @@ -143,18 +109,17 @@ def get_args_parser(): parser = argparse.ArgumentParser( description='This tool is used to verify the consumer performance.') + parser.add_argument( + '--bootstrap-servers', type=str, nargs='+', default=(), + help='host:port for cluster bootstrap servers') parser.add_argument( '--topic', type=str, - help='Topic for consumer test', + help='Topic for consumer test (default: kafka-python-benchmark-test)', default='kafka-python-benchmark-test') parser.add_argument( '--num-records', type=int, - help='number of messages to consume', + help='number of messages to consume (default: 1000000)', default=1000000) - parser.add_argument( - '--record-size', type=int, - help='message size in bytes', - default=100) parser.add_argument( '--consumer-config', type=str, nargs='+', default=(), help='kafka consumer related configuration properties like ' @@ -162,13 +127,9 @@ def get_args_parser(): parser.add_argument( '--fixture-compression', type=str, help='specify a compression type for use with broker fixtures / producer') - parser.add_argument( - '--brokers', type=int, - help='Number of kafka brokers to start', - default=0) parser.add_argument( '--stats-interval', type=int, - help='Interval in seconds for stats reporting to console', + help='Interval in seconds for stats reporting to console (default: 5)', default=5) parser.add_argument( '--raw-metrics', action='store_true', diff --git a/kafka/benchmarks/load_example.py b/kafka/benchmarks/load_example.py new file mode 100644 index 000000000..29796a74c --- /dev/null +++ b/kafka/benchmarks/load_example.py @@ -0,0 +1,110 @@ +#!/usr/bin/env python +from __future__ import print_function + +import argparse +import logging +import threading +import time + +from kafka import KafkaConsumer, KafkaProducer + + +class Producer(threading.Thread): + + def __init__(self, bootstrap_servers, topic, stop_event, msg_size): + super(Producer, self).__init__() + self.bootstrap_servers = bootstrap_servers + self.topic = topic + self.stop_event = stop_event + self.big_msg = b'1' * msg_size + + def run(self): + producer = KafkaProducer(bootstrap_servers=self.bootstrap_servers) + self.sent = 0 + + while not self.stop_event.is_set(): + producer.send(self.topic, self.big_msg) + self.sent += 1 + producer.flush() + producer.close() + + +class Consumer(threading.Thread): + def __init__(self, bootstrap_servers, topic, stop_event, msg_size): + super(Consumer, self).__init__() + self.bootstrap_servers = bootstrap_servers + self.topic = topic + self.stop_event = stop_event + self.msg_size = msg_size + + def run(self): + consumer = KafkaConsumer(bootstrap_servers=self.bootstrap_servers, + auto_offset_reset='earliest') + consumer.subscribe([self.topic]) + self.valid = 0 + self.invalid = 0 + + for message in consumer: + if len(message.value) == self.msg_size: + self.valid += 1 + else: + print('Invalid message:', len(message.value), self.msg_size) + self.invalid += 1 + + if self.stop_event.is_set(): + break + consumer.close() + + +def get_args_parser(): + parser = argparse.ArgumentParser( + description='This tool is used to demonstrate consumer and producer load.') + + parser.add_argument( + '--bootstrap-servers', type=str, nargs='+', default=('localhost:9092'), + help='host:port for cluster bootstrap servers (default: localhost:9092)') + parser.add_argument( + '--topic', type=str, + help='Topic for load test (default: kafka-python-benchmark-load-example)', + default='kafka-python-benchmark-load-example') + parser.add_argument( + '--msg-size', type=int, + help='Message size, in bytes, for load test (default: 524288)', + default=524288) + parser.add_argument( + '--load-time', type=int, + help='number of seconds to run load test (default: 10)', + default=10) + parser.add_argument( + '--log-level', type=str, + help='Optional logging level for load test: ERROR|INFO|DEBUG etc', + default=None) + return parser + + +def main(args): + if args.log_level: + logging.basicConfig( + format='%(asctime)s.%(msecs)s:%(name)s:%(thread)d:%(levelname)s:%(process)d:%(message)s', + level=getattr(logging, args.log_level)) + producer_stop = threading.Event() + consumer_stop = threading.Event() + threads = [ + Producer(args.bootstrap_servers, args.topic, producer_stop, args.msg_size), + Consumer(args.bootstrap_servers, args.topic, consumer_stop, args.msg_size) + ] + + for t in threads: + t.start() + + time.sleep(args.load_time) + producer_stop.set() + consumer_stop.set() + print('Messages sent: %d' % threads[0].sent) + print('Messages recvd: %d' % threads[1].valid) + print('Messages invalid: %d' % threads[1].invalid) + + +if __name__ == "__main__": + args = get_args_parser().parse_args() + main(args) diff --git a/benchmarks/producer_performance.py b/kafka/benchmarks/producer_performance.py old mode 100755 new mode 100644 similarity index 71% rename from benchmarks/producer_performance.py rename to kafka/benchmarks/producer_performance.py index c0de6fd23..1a1092960 --- a/benchmarks/producer_performance.py +++ b/kafka/benchmarks/producer_performance.py @@ -7,37 +7,15 @@ import pprint import sys import threading +import time import traceback from kafka.vendor.six.moves import range from kafka import KafkaProducer -from test.fixtures import KafkaFixture, ZookeeperFixture - - -def start_brokers(n): - print('Starting {0} {1}-node cluster...'.format(KafkaFixture.kafka_version, n)) - print('-> 1 Zookeeper') - zk = ZookeeperFixture.instance() - print('---> {0}:{1}'.format(zk.host, zk.port)) - print() - - partitions = min(n, 3) - replicas = min(n, 3) - print('-> {0} Brokers [{1} partitions / {2} replicas]'.format(n, partitions, replicas)) - brokers = [ - KafkaFixture.instance(i, zk, zk_chroot='', - partitions=partitions, replicas=replicas) - for i in range(n) - ] - for broker in brokers: - print('---> {0}:{1}'.format(broker.host, broker.port)) - print() - return brokers class ProducerPerformance(object): - @staticmethod def run(args): try: @@ -50,18 +28,14 @@ def run(args): pass if v == 'None': v = None + elif v == 'False': + v = False + elif v == 'True': + v = True props[k] = v - if args.brokers: - brokers = start_brokers(args.brokers) - props['bootstrap_servers'] = ['{0}:{1}'.format(broker.host, broker.port) - for broker in brokers] - print("---> bootstrap_servers={0}".format(props['bootstrap_servers'])) - print() - print('-> OK!') - print() - print('Initializing producer...') + props['bootstrap_servers'] = args.bootstrap_servers record = bytes(bytearray(args.record_size)) props['metrics_sample_window_ms'] = args.stats_interval * 1000 @@ -79,11 +53,29 @@ def run(args): print('-> OK!') print() - for i in range(args.num_records): - producer.send(topic=args.topic, value=record) - producer.flush() - + def _benchmark(): + results = [] + for i in range(args.num_records): + results.append(producer.send(topic=args.topic, value=record)) + print("Send complete...") + producer.flush() + producer.close() + count_success, count_failure = 0, 0 + for r in results: + if r.succeeded(): + count_success += 1 + elif r.failed(): + count_failure += 1 + else: + raise ValueError(r) + print("%d suceeded, %d failed" % (count_success, count_failure)) + + start_time = time.time() + _benchmark() + end_time = time.time() timer_stop.set() + timer.join() + print('Execution time:', end_time - start_time, 'secs') except Exception: exc_info = sys.exc_info() @@ -101,6 +93,8 @@ def __init__(self, interval, producer, event=None, raw_metrics=False): def print_stats(self): metrics = self.producer.metrics() + if not metrics: + return if self.raw_metrics: pprint.pprint(metrics) else: @@ -125,29 +119,28 @@ def get_args_parser(): parser = argparse.ArgumentParser( description='This tool is used to verify the producer performance.') + parser.add_argument( + '--bootstrap-servers', type=str, nargs='+', default=(), + help='host:port for cluster bootstrap server') parser.add_argument( '--topic', type=str, - help='Topic name for test', + help='Topic name for test (default: kafka-python-benchmark-test)', default='kafka-python-benchmark-test') parser.add_argument( '--num-records', type=int, - help='number of messages to produce', + help='number of messages to produce (default: 1000000)', default=1000000) parser.add_argument( '--record-size', type=int, - help='message size in bytes', + help='message size in bytes (default: 100)', default=100) parser.add_argument( '--producer-config', type=str, nargs='+', default=(), help='kafka producer related configuaration properties like ' 'bootstrap_servers,client_id etc..') - parser.add_argument( - '--brokers', type=int, - help='Number of kafka brokers to start', - default=0) parser.add_argument( '--stats-interval', type=int, - help='Interval in seconds for stats reporting to console', + help='Interval in seconds for stats reporting to console (default: 5)', default=5) parser.add_argument( '--raw-metrics', action='store_true', diff --git a/benchmarks/record_batch_compose.py b/kafka/benchmarks/record_batch_compose.py similarity index 89% rename from benchmarks/record_batch_compose.py rename to kafka/benchmarks/record_batch_compose.py index 5bdefa7af..5b07fd59a 100644 --- a/benchmarks/record_batch_compose.py +++ b/kafka/benchmarks/record_batch_compose.py @@ -71,7 +71,8 @@ def func(loops, magic): return res -runner = pyperf.Runner() -runner.bench_time_func('batch_append_v0', func, 0) -runner.bench_time_func('batch_append_v1', func, 1) -runner.bench_time_func('batch_append_v2', func, 2) +if __name__ == '__main__': + runner = pyperf.Runner() + runner.bench_time_func('batch_append_v0', func, 0) + runner.bench_time_func('batch_append_v1', func, 1) + runner.bench_time_func('batch_append_v2', func, 2) diff --git a/benchmarks/record_batch_read.py b/kafka/benchmarks/record_batch_read.py similarity index 90% rename from benchmarks/record_batch_read.py rename to kafka/benchmarks/record_batch_read.py index aa5e9c1e5..2ef32298d 100644 --- a/benchmarks/record_batch_read.py +++ b/kafka/benchmarks/record_batch_read.py @@ -76,7 +76,8 @@ def func(loops, magic): return res -runner = pyperf.Runner() -runner.bench_time_func('batch_read_v0', func, 0) -runner.bench_time_func('batch_read_v1', func, 1) -runner.bench_time_func('batch_read_v2', func, 2) +if __name__ == '__main__': + runner = pyperf.Runner() + runner.bench_time_func('batch_read_v0', func, 0) + runner.bench_time_func('batch_read_v1', func, 1) + runner.bench_time_func('batch_read_v2', func, 2) diff --git a/benchmarks/varint_speed.py b/kafka/benchmarks/varint_speed.py similarity index 81% rename from benchmarks/varint_speed.py rename to kafka/benchmarks/varint_speed.py index fd63d0ac1..b2628a1b5 100644 --- a/benchmarks/varint_speed.py +++ b/kafka/benchmarks/varint_speed.py @@ -113,8 +113,6 @@ def encode_varint_1(num): raise ValueError("Out of double range") return buf[:i + 1] -_assert_valid_enc(encode_varint_1) - def encode_varint_2(value, int2byte=six.int2byte): value = (value << 1) ^ (value >> 63) @@ -128,8 +126,6 @@ def encode_varint_2(value, int2byte=six.int2byte): value >>= 7 return res + int2byte(bits) -_assert_valid_enc(encode_varint_2) - def encode_varint_3(value, buf): append = buf.append @@ -145,12 +141,6 @@ def encode_varint_3(value, buf): return value -for encoded, decoded in test_data: - res = bytearray() - encode_varint_3(decoded, res) - assert res == encoded - - def encode_varint_4(value, int2byte=six.int2byte): value = (value << 1) ^ (value >> 63) @@ -185,12 +175,6 @@ def encode_varint_4(value, int2byte=six.int2byte): return res + int2byte(bits) -_assert_valid_enc(encode_varint_4) - -# import dis -# dis.dis(encode_varint_4) - - def encode_varint_5(value, buf, pos=0): value = (value << 1) ^ (value >> 63) @@ -204,12 +188,6 @@ def encode_varint_5(value, buf, pos=0): buf[pos] = bits return pos + 1 -for encoded, decoded in test_data: - res = bytearray(10) - written = encode_varint_5(decoded, res) - assert res[:written] == encoded - - def encode_varint_6(value, buf): append = buf.append value = (value << 1) ^ (value >> 63) @@ -253,12 +231,6 @@ def encode_varint_6(value, buf): return i -for encoded, decoded in test_data: - res = bytearray() - encode_varint_6(decoded, res) - assert res == encoded - - def size_of_varint_1(value): """ Number of bytes needed to encode an integer in variable-length format. """ @@ -271,8 +243,6 @@ def size_of_varint_1(value): break return res -_assert_valid_size(size_of_varint_1) - def size_of_varint_2(value): """ Number of bytes needed to encode an integer in variable-length format. @@ -298,8 +268,6 @@ def size_of_varint_2(value): return 9 return 10 -_assert_valid_size(size_of_varint_2) - if six.PY3: def _read_byte(memview, pos): @@ -351,8 +319,6 @@ def decode_varint_1(buffer, pos=0): # Normalize sign return (value >> 1) ^ -(value & 1), i + 1 -_assert_valid_dec(decode_varint_1) - def decode_varint_2(buffer, pos=0): result = 0 @@ -369,9 +335,6 @@ def decode_varint_2(buffer, pos=0): raise ValueError("Out of int64 range") -_assert_valid_dec(decode_varint_2) - - def decode_varint_3(buffer, pos=0): result = buffer[pos] if not (result & 0x81): @@ -393,51 +356,79 @@ def decode_varint_3(buffer, pos=0): raise ValueError("Out of int64 range") -_assert_valid_dec(decode_varint_3) - -# import dis -# dis.dis(decode_varint_3) - -runner = pyperf.Runner() -# Encode algorithms returning a bytes result -for bench_func in [ - encode_varint_1, - encode_varint_2, - encode_varint_4]: - for i, value in enumerate(BENCH_VALUES_ENC): - runner.bench_func( - '{}_{}byte'.format(bench_func.__name__, i + 1), - bench_func, value) - -# Encode algorithms writing to the buffer -for bench_func in [ - encode_varint_3, - encode_varint_5, - encode_varint_6]: - for i, value in enumerate(BENCH_VALUES_ENC): - fname = bench_func.__name__ - runner.timeit( - '{}_{}byte'.format(fname, i + 1), - stmt="{}({}, buffer)".format(fname, value), - setup="from __main__ import {}; buffer = bytearray(10)".format( - fname) - ) - -# Size algorithms -for bench_func in [ - size_of_varint_1, - size_of_varint_2]: - for i, value in enumerate(BENCH_VALUES_ENC): - runner.bench_func( - '{}_{}byte'.format(bench_func.__name__, i + 1), - bench_func, value) - -# Decode algorithms -for bench_func in [ - decode_varint_1, - decode_varint_2, - decode_varint_3]: - for i, value in enumerate(BENCH_VALUES_DEC): - runner.bench_func( - '{}_{}byte'.format(bench_func.__name__, i + 1), - bench_func, value) +if __name__ == '__main__': + _assert_valid_enc(encode_varint_1) + _assert_valid_enc(encode_varint_2) + + for encoded, decoded in test_data: + res = bytearray() + encode_varint_3(decoded, res) + assert res == encoded + + _assert_valid_enc(encode_varint_4) + + # import dis + # dis.dis(encode_varint_4) + + for encoded, decoded in test_data: + res = bytearray(10) + written = encode_varint_5(decoded, res) + assert res[:written] == encoded + + for encoded, decoded in test_data: + res = bytearray() + encode_varint_6(decoded, res) + assert res == encoded + + _assert_valid_size(size_of_varint_1) + _assert_valid_size(size_of_varint_2) + _assert_valid_dec(decode_varint_1) + _assert_valid_dec(decode_varint_2) + _assert_valid_dec(decode_varint_3) + + # import dis + # dis.dis(decode_varint_3) + + runner = pyperf.Runner() + # Encode algorithms returning a bytes result + for bench_func in [ + encode_varint_1, + encode_varint_2, + encode_varint_4]: + for i, value in enumerate(BENCH_VALUES_ENC): + runner.bench_func( + '{}_{}byte'.format(bench_func.__name__, i + 1), + bench_func, value) + + # Encode algorithms writing to the buffer + for bench_func in [ + encode_varint_3, + encode_varint_5, + encode_varint_6]: + for i, value in enumerate(BENCH_VALUES_ENC): + fname = bench_func.__name__ + runner.timeit( + '{}_{}byte'.format(fname, i + 1), + stmt="{}({}, buffer)".format(fname, value), + setup="from __main__ import {}; buffer = bytearray(10)".format( + fname) + ) + + # Size algorithms + for bench_func in [ + size_of_varint_1, + size_of_varint_2]: + for i, value in enumerate(BENCH_VALUES_ENC): + runner.bench_func( + '{}_{}byte'.format(bench_func.__name__, i + 1), + bench_func, value) + + # Decode algorithms + for bench_func in [ + decode_varint_1, + decode_varint_2, + decode_varint_3]: + for i, value in enumerate(BENCH_VALUES_DEC): + runner.bench_func( + '{}_{}byte'.format(bench_func.__name__, i + 1), + bench_func, value) diff --git a/pyproject.toml b/pyproject.toml index 2a675c111..d575a8959 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -39,6 +39,7 @@ lz4 = ["lz4"] snappy = ["python-snappy"] zstd = ["zstandard"] testing = ["pytest", "mock; python_version < '3.3'", "pytest-mock", "pytest-timeout"] +benchmarks = ["pyperf"] [tool.setuptools] include-package-data = false diff --git a/requirements-dev.txt b/requirements-dev.txt index 3bc51fd78..8de5e28d4 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -6,6 +6,7 @@ lz4 mock; python_version < '3.3' py pylint +pyperf pytest pytest-cov pytest-mock From c1a5517358998ee7a1beb37c3657ca2e8b8ee036 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 12:25:16 -0700 Subject: [PATCH 111/205] Patch Release 2.1.5 --- CHANGES.md | 12 ++++++++++++ docs/changelog.rst | 16 ++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index c67ca8ace..749b83afb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,15 @@ +# 2.1.5 (Apr 4, 2025) + +Fixes +* Fix python2.7 errors (#2578) + +Improvements +* Move benchmark scripts to kafka.benchmarks module (#2584) +* Use __slots__ for metrics (#2583) +* Pass `metrics_enabled=False` to disable metrics (#2581) +* Drop unused kafka.producer.buffer / SimpleBufferPool (#2580) +* Raise UnsupportedVersionError from coordinator (#2579) + # 2.1.4 (Mar 28, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index bad0893c2..e77885af7 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,22 @@ Changelog ========= +2.1.5 (Apr 4, 2025) +################### + +Fixes +------ +* Fix python2.7 errors (#2578) + +Improvements +------------ +* Move benchmark scripts to kafka.benchmarks module (#2584) +* Use __slots__ for metrics (#2583) +* Pass `metrics_enabled=False` to disable metrics (#2581) +* Drop unused kafka.producer.buffer / SimpleBufferPool (#2580) +* Raise UnsupportedVersionError from coordinator (#2579) + + 2.1.4 (Mar 28, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 81baaee65..cfb007cbc 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.5.dev' +__version__ = '2.1.5' From ef6d3369db3fb81b49ca0b3288eb06e5f2eebf6a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 12:31:57 -0700 Subject: [PATCH 112/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index cfb007cbc..d92194f41 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.1.5' +__version__ = '2.2.0.dev' From 28cbad6a502d39e47d16990d097a6efc9245ab93 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 6 Apr 2025 08:51:27 -0700 Subject: [PATCH 113/205] KIP-98: Add idempotent producer support (#2569) --- kafka/client_async.py | 40 ++++++++- kafka/producer/kafka.py | 57 ++++++++++-- kafka/producer/record_accumulator.py | 58 +++++++++--- kafka/producer/sender.py | 126 +++++++++++++++++++++++---- kafka/producer/transaction_state.py | 96 ++++++++++++++++++++ kafka/protocol/init_producer_id.py | 46 ++++++++++ kafka/record/default_records.py | 9 ++ kafka/record/memory_records.py | 27 +++++- test/test_producer.py | 2 +- test/test_record_accumulator.py | 75 ++++++++++++++++ test/test_sender.py | 3 + 11 files changed, 498 insertions(+), 41 deletions(-) create mode 100644 kafka/producer/transaction_state.py create mode 100644 kafka/protocol/init_producer_id.py create mode 100644 test/test_record_accumulator.py diff --git a/kafka/client_async.py b/kafka/client_async.py index 19508b242..30258b7bd 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -27,7 +27,7 @@ from kafka.metrics.stats.rate import TimeUnit from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.metadata import MetadataRequest -from kafka.util import Dict, WeakMethod, ensure_valid_topic_name +from kafka.util import Dict, WeakMethod, ensure_valid_topic_name, timeout_ms_fn # 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 @@ -400,6 +400,11 @@ def maybe_connect(self, node_id, wakeup=True): return True return False + def connection_failed(self, node_id): + if node_id not in self._conns: + return False + return self._conns[node_id].connect_failed() + def _should_recycle_connection(self, conn): # Never recycle unless disconnected if not conn.disconnected(): @@ -1157,6 +1162,39 @@ def bootstrap_connected(self): else: return False + def await_ready(self, node_id, timeout_ms=30000): + """ + Invokes `poll` to discard pending disconnects, followed by `client.ready` and 0 or more `client.poll` + invocations until the connection to `node` is ready, the timeoutMs expires or the connection fails. + + It returns `true` if the call completes normally or `false` if the timeoutMs expires. If the connection fails, + an `IOException` is thrown instead. Note that if the `NetworkClient` has been configured with a positive + connection timeoutMs, it is possible for this method to raise an `IOException` for a previous connection which + has recently disconnected. + + This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with + care. + """ + inner_timeout_ms = timeout_ms_fn(timeout_ms, None) + self.poll(timeout_ms=0) + if self.is_ready(node_id): + return True + + while not self.is_ready(node_id) and inner_timeout_ms() > 0: + if self.connection_failed(node_id): + raise Errors.KafkaConnectionError("Connection to %s failed." % (node_id,)) + self.maybe_connect(node_id) + self.poll(timeout_ms=inner_timeout_ms()) + return self.is_ready(node_id) + + def send_and_receive(self, node_id, request): + future = self.send(node_id, request) + self.poll(future=future) + assert future.is_done + if future.failed(): + raise future.exception + return future.value + # OrderedDict requires python2.7+ try: diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index f0eb37a8f..320a1657f 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -19,6 +19,7 @@ from kafka.producer.future import FutureRecordMetadata, FutureProduceResult from kafka.producer.record_accumulator import AtomicInteger, RecordAccumulator from kafka.producer.sender import Sender +from kafka.producer.transaction_state import TransactionState from kafka.record.default_records import DefaultRecordBatchBuilder from kafka.record.legacy_records import LegacyRecordBatchBuilder from kafka.serializer import Serializer @@ -93,6 +94,19 @@ 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. + 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. + + Note that enabling idempotence requires + `max_in_flight_requests_per_connection` to be set to 1 and `retries` + cannot be zero. Additionally, `acks` must be set to 'all'. If these + values are left at their defaults, the producer will override the + defaults to be suitable. If the values are set to something + incompatible with the idempotent producer, a KafkaConfigurationError + will be raised. + acks (0, 1, 'all'): The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the durability of records that are sent. The @@ -303,6 +317,7 @@ class KafkaProducer(object): 'client_id': None, 'key_serializer': None, 'value_serializer': None, + 'enable_idempotence': False, 'acks': 1, 'bootstrap_topics_filter': set(), 'compression_type': None, @@ -365,6 +380,7 @@ class KafkaProducer(object): def __init__(self, **configs): log.debug("Starting the Kafka producer") # trace self.config = copy.copy(self.DEFAULT_CONFIG) + user_provided_configs = set(configs.keys()) for key in self.config: if key in configs: self.config[key] = configs.pop(key) @@ -428,13 +444,41 @@ def __init__(self, **configs): assert checker(), "Libraries for {} compression codec not found".format(ct) self.config['compression_attrs'] = compression_attrs - message_version = self._max_usable_produce_magic() - self._accumulator = RecordAccumulator(message_version=message_version, **self.config) + self._transaction_state = None + if self.config['enable_idempotence']: + self._transaction_state = TransactionState() + if 'retries' not in user_provided_configs: + log.info("Overriding the default 'retries' config to 3 since the idempotent producer is enabled.") + self.config['retries'] = 3 + elif self.config['retries'] == 0: + raise Errors.KafkaConfigurationError("Must set 'retries' to non-zero when using the idempotent producer.") + + if 'max_in_flight_requests_per_connection' not in user_provided_configs: + log.info("Overriding the default 'max_in_flight_requests_per_connection' to 1 since idempontence is enabled.") + self.config['max_in_flight_requests_per_connection'] = 1 + elif self.config['max_in_flight_requests_per_connection'] != 1: + raise Errors.KafkaConfigurationError("Must set 'max_in_flight_requests_per_connection' to 1 in order" + " to use the idempotent producer." + " Otherwise we cannot guarantee idempotence.") + + if 'acks' not in user_provided_configs: + log.info("Overriding the default 'acks' config to 'all' since idempotence is enabled") + self.config['acks'] = -1 + elif self.config['acks'] != -1: + raise Errors.KafkaConfigurationError("Must set 'acks' config to 'all' in order to use the idempotent" + " producer. Otherwise we cannot guarantee idempotence") + + message_version = self.max_usable_produce_magic(self.config['api_version']) + self._accumulator = RecordAccumulator( + transaction_state=self._transaction_state, + message_version=message_version, + **self.config) self._metadata = client.cluster guarantee_message_order = bool(self.config['max_in_flight_requests_per_connection'] == 1) self._sender = Sender(client, self._metadata, self._accumulator, metrics=self._metrics, + transaction_state=self._transaction_state, guarantee_message_order=guarantee_message_order, **self.config) self._sender.daemon = True @@ -548,16 +592,17 @@ def partitions_for(self, topic): max_wait = self.config['max_block_ms'] / 1000 return self._wait_on_metadata(topic, max_wait) - def _max_usable_produce_magic(self): - if self.config['api_version'] >= (0, 11): + @classmethod + def max_usable_produce_magic(cls, api_version): + if api_version >= (0, 11): return 2 - elif self.config['api_version'] >= (0, 10, 0): + elif api_version >= (0, 10, 0): return 1 else: return 0 def _estimate_size_in_bytes(self, key, value, headers=[]): - magic = self._max_usable_produce_magic() + magic = self.max_usable_produce_magic(self.config['api_version']) if magic == 2: return DefaultRecordBatchBuilder.estimate_size_in_bytes( key, value, headers) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index ba823500d..60fa0a323 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -35,9 +35,9 @@ def get(self): class ProducerBatch(object): - def __init__(self, tp, records): + def __init__(self, tp, records, now=None): self.max_record_size = 0 - now = time.time() + now = time.time() if now is None else now self.created = now self.drained = None self.attempts = 0 @@ -52,13 +52,18 @@ def __init__(self, tp, records): def record_count(self): return self.records.next_offset() - def try_append(self, timestamp_ms, key, value, headers): + @property + def producer_id(self): + return self.records.producer_id if self.records else None + + 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 = time.time() + self.last_append = now future = FutureRecordMetadata(self.produce_future, metadata.offset, metadata.timestamp, metadata.crc, len(key) if key is not None else -1, @@ -81,7 +86,7 @@ def done(self, base_offset=None, timestamp_ms=None, exception=None, log_start_of log_start_offset, exception) # trace self.produce_future.failure(exception) - def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full): + def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full, now=None): """Expire batches if metadata is not available A batch whose metadata is not available should be expired if one @@ -93,7 +98,7 @@ def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full) * the batch is in retry AND request timeout has elapsed after the backoff period ended. """ - now = time.time() + now = time.time() if now is None else now since_append = now - self.last_append since_ready = now - (self.created + linger_ms / 1000.0) since_backoff = now - (self.last_attempt + retry_backoff_ms / 1000.0) @@ -121,6 +126,10 @@ def in_retry(self): def set_retry(self): self._retry = True + @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()) @@ -161,6 +170,7 @@ class RecordAccumulator(object): 'compression_attrs': 0, 'linger_ms': 0, 'retry_backoff_ms': 100, + 'transaction_state': None, 'message_version': 0, } @@ -171,6 +181,7 @@ def __init__(self, **configs): self.config[key] = configs.pop(key) self._closed = False + self._transaction_state = self.config['transaction_state'] self._flushes_in_progress = AtomicInteger() self._appends_in_progress = AtomicInteger() self._batches = collections.defaultdict(collections.deque) # TopicPartition: [ProducerBatch] @@ -233,6 +244,10 @@ def append(self, tp, timestamp_ms, key, value, headers): batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False + if self._transaction_state and self.config['message_version'] < 2: + raise Errors.UnsupportedVersionError("Attempting to use idempotence with a broker which" + " does not support the required message format (v2)." + " The broker must be version 0.11 or later.") records = MemoryRecordsBuilder( self.config['message_version'], self.config['compression_attrs'], @@ -310,9 +325,9 @@ def abort_expired_batches(self, request_timeout_ms, cluster): return expired_batches - def reenqueue(self, batch): + def reenqueue(self, batch, now=None): """Re-enqueue the given record batch in the accumulator to retry.""" - now = time.time() + now = time.time() if now is None else now batch.attempts += 1 batch.last_attempt = now batch.last_append = now @@ -323,7 +338,7 @@ def reenqueue(self, batch): with self._tp_locks[batch.topic_partition]: dq.appendleft(batch) - def ready(self, cluster): + def ready(self, cluster, now=None): """ Get a list of nodes whose partitions are ready to be sent, and the earliest time at which any non-sendable partition will be ready; @@ -357,7 +372,7 @@ def ready(self, cluster): ready_nodes = set() next_ready_check = 9999999.99 unknown_leaders_exist = False - now = time.time() + now = time.time() if now is None else now # several threads are accessing self._batches -- to simplify # concurrent access, we iterate over a snapshot of partitions @@ -412,7 +427,7 @@ def has_unsent(self): return True return False - def drain(self, cluster, nodes, max_size): + def drain(self, cluster, nodes, max_size, now=None): """ Drain all the data for the given nodes and collate them into a list of batches that will fit within the specified size on a per-node basis. @@ -430,7 +445,7 @@ def drain(self, cluster, nodes, max_size): if not nodes: return {} - now = time.time() + now = time.time() if now is None else now batches = {} for node_id in nodes: size = 0 @@ -463,7 +478,26 @@ def drain(self, cluster, nodes, max_size): # single request break else: + producer_id_and_epoch = None + if self._transaction_state: + producer_id_and_epoch = self._transaction_state.producer_id_and_epoch + if not producer_id_and_epoch.is_valid: + # we cannot send the batch until we have refreshed the PID + log.debug("Waiting to send ready batches because transaction producer id is not valid") + break + batch = dq.popleft() + if producer_id_and_epoch and not batch.in_retry(): + # If the batch is in retry, then we should not change the pid and + # sequence number, since this may introduce duplicates. In particular, + # the previous attempt may actually have been accepted, and if we change + # the pid and sequence here, this attempt will also be accepted, causing + # a duplicate. + sequence_number = self._transaction_state.sequence_number(batch.topic_partition) + log.debug("Dest: %s: %s producer_id=%s epoch=%s sequence=%s", + node_id, batch.topic_partition, producer_id_and_epoch.producer_id, producer_id_and_epoch.epoch, + sequence_number) + batch.records.set_producer_state(producer_id_and_epoch.producer_id, producer_id_and_epoch.epoch, sequence_number) batch.records.close() size += batch.records.size_in_bytes() ready.append(batch) diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 20af28d07..24b84a9b1 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -11,6 +11,7 @@ from kafka import errors as Errors from kafka.metrics.measurable import AnonMeasurable from kafka.metrics.stats import Avg, Max, Rate +from kafka.protocol.init_producer_id import InitProducerIdRequest from kafka.protocol.produce import ProduceRequest from kafka.structs import TopicPartition from kafka.version import __version__ @@ -29,8 +30,12 @@ class Sender(threading.Thread): 'acks': 1, 'retries': 0, 'request_timeout_ms': 30000, + 'retry_backoff_ms': 100, 'metrics': None, 'guarantee_message_order': False, + 'transaction_state': None, + 'transactional_id': None, + 'transaction_timeout_ms': 60000, 'client_id': 'kafka-python-' + __version__, } @@ -52,6 +57,7 @@ def __init__(self, client, metadata, accumulator, **configs): self._sensors = SenderMetrics(self.config['metrics'], self._client, self._metadata) else: self._sensors = None + self._transaction_state = self.config['transaction_state'] def run(self): """The main run loop for the sender thread.""" @@ -95,6 +101,8 @@ def run_once(self): while self._topics_to_add: self._client.add_topic(self._topics_to_add.pop()) + self._maybe_wait_for_producer_id() + # get the list of partitions with data ready to send result = self._accumulator.ready(self._metadata) ready_nodes, next_ready_check_delay, unknown_leaders_exist = result @@ -128,6 +136,13 @@ def run_once(self): expired_batches = self._accumulator.abort_expired_batches( self.config['request_timeout_ms'], self._metadata) + # Reset the producer_id if an expired batch has previously been sent to the broker. + # See the documentation of `TransactionState.reset_producer_id` to understand why + # we need to reset the producer id here. + if self._transaction_state and any([batch.in_retry() for batch in expired_batches]): + self._transaction_state.reset_producer_id() + return + if self._sensors: for expired_batch in expired_batches: self._sensors.record_errors(expired_batch.topic_partition.topic, expired_batch.record_count) @@ -185,6 +200,41 @@ def add_topic(self, topic): self._topics_to_add.add(topic) self.wakeup() + def _maybe_wait_for_producer_id(self): + if not self._transaction_state: + return + + while not self._transaction_state.has_pid(): + try: + node_id = self._client.least_loaded_node() + if node_id is None or not self._client.await_ready(node_id): + log.debug("Could not find an available broker to send InitProducerIdRequest to." + + " Will back off and try again.") + time.sleep(self._client.least_loaded_node_refresh_ms() / 1000) + continue + version = self._client.api_version(InitProducerIdRequest, max_version=1) + request = InitProducerIdRequest[version]( + transactional_id=self.config['transactional_id'], + transaction_timeout_ms=self.config['transaction_timeout_ms'], + ) + response = self._client.send_and_receive(node_id, request) + error_type = Errors.for_code(response.error_code) + if error_type is Errors.NoError: + self._transaction_state.set_producer_id_and_epoch(response.producer_id, response.producer_epoch) + return + elif getattr(error_type, 'retriable', False): + log.debug("Retriable error from InitProducerId response: %s", error_type.__name__) + if getattr(error_type, 'invalid_metadata', False): + self._metadata.request_update() + else: + log.error("Received a non-retriable error from InitProducerId response: %s", error_type.__name__) + break + except Errors.KafkaConnectionError: + log.debug("Broker %s disconnected while awaiting InitProducerId response", node_id) + except Errors.RequestTimedOutError: + log.debug("InitProducerId request to node %s timed out", node_id) + time.sleep(self.config['retry_backoff_ms'] / 1000) + def _failed_produce(self, batches, node_id, error): log.error("Error sending produce request to node %d: %s", node_id, error) # trace for batch in batches: @@ -221,6 +271,17 @@ def _handle_produce_response(self, node_id, send_time, batches, response): for batch in batches: self._complete_batch(batch, None, -1) + def _fail_batch(batch, *args, **kwargs): + if self._transaction_state and self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: + # 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 + # numbers for all existing partitions. + self._transaction_state.reset_producer_id() + batch.done(*args, **kwargs) + self._accumulator.deallocate(batch) + if self._sensors: + self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) + def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_start_offset=None): """Complete or retry the given batch of records. @@ -235,28 +296,55 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star if error is Errors.NoError: error = None - if error is not None and self._can_retry(batch, error): - # retry - log.warning("Got error produce response on topic-partition %s," - " retrying (%d attempts left). Error: %s", - batch.topic_partition, - self.config['retries'] - batch.attempts - 1, - error) - self._accumulator.reenqueue(batch) - if self._sensors: - self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) - else: - if error is Errors.TopicAuthorizationFailedError: - error = error(batch.topic_partition.topic) + if error is not None: + if self._can_retry(batch, error): + # retry + log.warning("Got error produce response on topic-partition %s," + " retrying (%d attempts left). Error: %s", + batch.topic_partition, + self.config['retries'] - batch.attempts - 1, + error) + + # If idempotence is enabled only retry the request if the current PID is the same as the pid of the batch. + if not self._transaction_state or self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: + log.debug("Retrying batch to topic-partition %s. Sequence number: %s", + batch.topic_partition, + self._transaction_state.sequence_number(batch.topic_partition) if self._transaction_state else None) + self._accumulator.reenqueue(batch) + if self._sensors: + self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) + else: + log.warning("Attempted to retry sending a batch but the producer id changed from %s to %s. This batch will be dropped" % ( + batch.producer_id, self._transaction_state.producer_id_and_epoch.producer_id)) + self._fail_batch(batch, base_offset=base_offset, timestamp_ms=timestamp_ms, exception=error, log_start_offset=log_start_offset) + else: + if error is Errors.OutOfOrderSequenceNumberError and batch.producer_id == self._transaction_state.producer_id_and_epoch.producer_id: + log.error("The broker received an out of order sequence number error for produer_id %s, topic-partition %s" + " at offset %s. This indicates data loss on the broker, and should be investigated.", + batch.producer_id, batch.topic_partition, base_offset) + + if error is Errors.TopicAuthorizationFailedError: + error = error(batch.topic_partition.topic) + + # tell the user the result of their request + self._fail_batch(batch, base_offset=base_offset, timestamp_ms=timestamp_ms, exception=error, log_start_offset=log_start_offset) + + if error is Errors.UnknownTopicOrPartitionError: + log.warning("Received unknown topic or partition error in produce request on partition %s." + " The topic/partition may not exist or the user may not have Describe access to it", + batch.topic_partition) + + if getattr(error, 'invalid_metadata', False): + self._metadata.request_update() - # tell the user the result of their request - batch.done(base_offset, timestamp_ms, error, log_start_offset) + else: + batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, log_start_offset=log_start_offset) self._accumulator.deallocate(batch) - if error is not None and self._sensors: - self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) - if getattr(error, 'invalid_metadata', False): - self._metadata.request_update() + if self._transaction_state and self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: + self._transaction_state.increment_sequence_number(batch.topic_partition, batch.record_count) + log.debug("Incremented sequence number for topic-partition %s to %s", batch.topic_partition, + self._transaction_state.sequence_number(batch.topic_partition)) # Unmute the completed partition. if self.config['guarantee_message_order']: diff --git a/kafka/producer/transaction_state.py b/kafka/producer/transaction_state.py new file mode 100644 index 000000000..05cdc5766 --- /dev/null +++ b/kafka/producer/transaction_state.py @@ -0,0 +1,96 @@ +from __future__ import absolute_import, division + +import collections +import threading +import time + +from kafka.errors import IllegalStateError + + +NO_PRODUCER_ID = -1 +NO_PRODUCER_EPOCH = -1 + + +class ProducerIdAndEpoch(object): + __slots__ = ('producer_id', 'epoch') + + def __init__(self, producer_id, epoch): + self.producer_id = producer_id + self.epoch = epoch + + @property + def is_valid(self): + return NO_PRODUCER_ID < self.producer_id + + def __str__(self): + return "ProducerIdAndEpoch(producer_id={}, epoch={})".format(self.producer_id, self.epoch) + +class TransactionState(object): + __slots__ = ('producer_id_and_epoch', '_sequence_numbers', '_lock') + + def __init__(self): + self.producer_id_and_epoch = ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH) + self._sequence_numbers = collections.defaultdict(lambda: 0) + self._lock = threading.Condition() + + def has_pid(self): + return self.producer_id_and_epoch.is_valid + + + def await_producer_id_and_epoch(self, max_wait_time_ms): + """ + A blocking call to get the pid and epoch for the producer. If the PID and epoch has not been set, this method + will block for at most maxWaitTimeMs. It is expected that this method be called from application thread + contexts (ie. through Producer.send). The PID it self will be retrieved in the background thread. + + Arguments: + max_wait_time_ms (numeric): The maximum time to block. + + Returns: + ProducerIdAndEpoch object. Callers must check the 'is_valid' property of the returned object to ensure that a + valid pid and epoch is actually returned. + """ + with self._lock: + start = time.time() + elapsed = 0 + while not self.has_pid() and elapsed < max_wait_time_ms: + self._lock.wait(max_wait_time_ms / 1000) + elapsed = time.time() - start + return self.producer_id_and_epoch + + def set_producer_id_and_epoch(self, producer_id, epoch): + """ + Set the pid and epoch atomically. This method will signal any callers blocked on the `pidAndEpoch` method + once the pid is set. This method will be called on the background thread when the broker responds with the pid. + """ + with self._lock: + self.producer_id_and_epoch = ProducerIdAndEpoch(producer_id, epoch) + if self.producer_id_and_epoch.is_valid: + self._lock.notify_all() + + def reset_producer_id(self): + """ + This method is used when the producer needs to reset it's internal state because of an irrecoverable exception + from the broker. + + We need to reset the producer id and associated state when we have sent a batch to the broker, but we either get + a non-retriable exception or we run out of retries, or the batch expired in the producer queue after it was already + sent to the broker. + + In all of these cases, we don't know whether batch was actually committed on the broker, and hence whether the + sequence number was actually updated. If we don't reset the producer state, we risk the chance that all future + messages will return an OutOfOrderSequenceException. + """ + with self._lock: + self.producer_id_and_epoch = ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH) + self._sequence_numbers.clear() + + def sequence_number(self, tp): + with self._lock: + return self._sequence_numbers[tp] + + def increment_sequence_number(self, tp, increment): + with self._lock: + if tp not in self._sequence_numbers: + raise IllegalStateError("Attempt to increment sequence number for a partition with no current sequence.") + self._sequence_numbers[tp] += increment diff --git a/kafka/protocol/init_producer_id.py b/kafka/protocol/init_producer_id.py new file mode 100644 index 000000000..8426fe00b --- /dev/null +++ b/kafka/protocol/init_producer_id.py @@ -0,0 +1,46 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Int16, Int32, Int64, Schema, String + + +class InitProducerIdResponse_v0(Response): + API_KEY = 22 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ('producer_id', Int64), + ('producer_epoch', Int16), + ) + + +class InitProducerIdResponse_v1(Response): + API_KEY = 22 + API_VERSION = 1 + SCHEMA = InitProducerIdResponse_v0.SCHEMA + + +class InitProducerIdRequest_v0(Request): + API_KEY = 22 + API_VERSION = 0 + RESPONSE_TYPE = InitProducerIdResponse_v0 + SCHEMA = Schema( + ('transactional_id', String('utf-8')), + ('transaction_timeout_ms', Int32), + ) + + +class InitProducerIdRequest_v1(Request): + API_KEY = 22 + API_VERSION = 1 + RESPONSE_TYPE = InitProducerIdResponse_v1 + SCHEMA = InitProducerIdRequest_v0.SCHEMA + + +InitProducerIdRequest = [ + InitProducerIdRequest_v0, InitProducerIdRequest_v1, +] +InitProducerIdResponse = [ + InitProducerIdResponse_v0, InitProducerIdResponse_v1, +] diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index 0d69d72a2..855306bbd 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -448,6 +448,15 @@ def __init__( self._buffer = bytearray(self.HEADER_STRUCT.size) + def set_producer_state(self, producer_id, producer_epoch, base_sequence): + self._producer_id = producer_id + self._producer_epoch = producer_epoch + self._base_sequence = base_sequence + + @property + def producer_id(self): + return self._producer_id + def _get_attributes(self, include_compression_type=True): attrs = 0 if include_compression_type: diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index 72baea547..a803047ea 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -22,7 +22,7 @@ import struct -from kafka.errors import CorruptRecordException +from kafka.errors import CorruptRecordException, IllegalStateError, UnsupportedVersionError from kafka.record.abc import ABCRecords from kafka.record.legacy_records import LegacyRecordBatch, LegacyRecordBatchBuilder from kafka.record.default_records import DefaultRecordBatch, DefaultRecordBatchBuilder @@ -113,7 +113,7 @@ def next_batch(self, _min_slice=MIN_SLICE, class MemoryRecordsBuilder(object): __slots__ = ("_builder", "_batch_size", "_buffer", "_next_offset", "_closed", - "_bytes_written") + "_magic", "_bytes_written", "_producer_id") def __init__(self, magic, compression_type, batch_size, offset=0): assert magic in [0, 1, 2], "Not supported magic" @@ -123,15 +123,18 @@ def __init__(self, magic, compression_type, batch_size, offset=0): magic=magic, compression_type=compression_type, is_transactional=False, producer_id=-1, producer_epoch=-1, base_sequence=-1, batch_size=batch_size) + self._producer_id = -1 else: self._builder = LegacyRecordBatchBuilder( magic=magic, compression_type=compression_type, batch_size=batch_size) + self._producer_id = None self._batch_size = batch_size self._buffer = None self._next_offset = offset self._closed = False + self._magic = magic self._bytes_written = 0 def skip(self, offsets_to_skip): @@ -155,6 +158,24 @@ def append(self, timestamp, key, value, headers=[]): self._next_offset += 1 return metadata + def set_producer_state(self, producer_id, producer_epoch, base_sequence): + if self._magic < 2: + raise UnsupportedVersionError('Producer State requires Message format v2+') + elif self._closed: + # Sequence numbers are assigned when the batch is closed while the accumulator is being drained. + # If the resulting ProduceRequest to the partition leader failed for a retriable error, the batch will + # be re queued. In this case, we should not attempt to set the state again, since changing the pid and sequence + # once a batch has been sent to the broker risks introducing duplicates. + raise IllegalStateError("Trying to set producer state of an already closed batch. This indicates a bug on the client.") + self._builder.set_producer_state(producer_id, producer_epoch, base_sequence) + self._producer_id = producer_id + + @property + def producer_id(self): + if self._magic < 2: + raise UnsupportedVersionError('Producer State requires Message format v2+') + return self._producer_id + def close(self): # This method may be called multiple times on the same batch # i.e., on retries @@ -164,6 +185,8 @@ def close(self): if not self._closed: self._bytes_written = self._builder.size() self._buffer = bytes(self._builder.build()) + if self._magic == 2: + self._producer_id = self._builder.producer_id self._builder = None self._closed = True diff --git a/test/test_producer.py b/test/test_producer.py index 069362f26..303832b9f 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -100,7 +100,7 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression): retries=5, max_block_ms=30000, compression_type=compression) as producer: - magic = producer._max_usable_produce_magic() + magic = producer.max_usable_produce_magic(producer.config['api_version']) # record headers are supported in 0.11.0 if env_kafka_version() < (0, 11, 0): diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py new file mode 100644 index 000000000..babff5617 --- /dev/null +++ b/test/test_record_accumulator.py @@ -0,0 +1,75 @@ +# pylint: skip-file +from __future__ import absolute_import + +import pytest +import io + +from kafka.errors import KafkaTimeoutError +from kafka.producer.future import FutureRecordMetadata, RecordMetadata +from kafka.producer.record_accumulator import RecordAccumulator, ProducerBatch +from kafka.record.memory_records import MemoryRecordsBuilder +from kafka.structs import TopicPartition + + +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) + 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, log_start_offset=0) + 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, log_start_offset=0, + 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.set_retry() + assert batch.in_retry() + +def test_producer_batch_maybe_expire(): + tp = TopicPartition('foo', 0) + records = MemoryRecordsBuilder( + magic=2, compression_type=0, batch_size=100000) + batch = ProducerBatch(tp, records, now=1) + future = batch.try_append(0, b'key', b'value', [], now=2) + request_timeout_ms = 5000 + retry_backoff_ms = 200 + linger_ms = 1000 + is_full = True + batch.maybe_expire(request_timeout_ms, retry_backoff_ms, linger_ms, is_full, now=20) + assert batch.is_done + assert future.is_done + assert future.failed() + assert isinstance(future.exception, KafkaTimeoutError) diff --git a/test/test_sender.py b/test/test_sender.py index b037d2b48..eedc43d25 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -6,6 +6,7 @@ from kafka.client_async import KafkaClient 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 @@ -24,6 +25,7 @@ def sender(client, accumulator, metrics, mocker): @pytest.mark.parametrize(("api_version", "produce_version"), [ + ((2, 1), 7), ((0, 10, 0), 2), ((0, 9), 1), ((0, 8, 0), 0) @@ -31,6 +33,7 @@ def sender(client, accumulator, metrics, mocker): def test_produce_request(sender, mocker, api_version, produce_version): sender._client._api_versions = BROKER_API_VERSIONS[api_version] tp = TopicPartition('foo', 0) + magic = KafkaProducer.max_usable_produce_magic(api_version) records = MemoryRecordsBuilder( magic=1, compression_type=0, batch_size=100000) batch = ProducerBatch(tp, records) From 103025feb536a13cc9c1017ecebb7f87b81d0c85 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Apr 2025 14:42:20 -0700 Subject: [PATCH 114/205] fixup Sender._fail_batch --- kafka/producer/sender.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 24b84a9b1..96a50cbbc 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -271,7 +271,7 @@ def _handle_produce_response(self, node_id, send_time, batches, response): for batch in batches: self._complete_batch(batch, None, -1) - def _fail_batch(batch, *args, **kwargs): + def _fail_batch(self, batch, *args, **kwargs): if self._transaction_state and self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: # 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 From c2fe7c3bc1e252e217ea0a1116bfbc613b01918c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Apr 2025 15:02:07 -0700 Subject: [PATCH 115/205] KIP-98: Add Consumer support for READ_COMMITTED (#2582) --- kafka/consumer/fetcher.py | 98 +++++++++++++++++++++++----- kafka/consumer/group.py | 4 ++ kafka/consumer/subscription_state.py | 2 +- kafka/protocol/fetch.py | 6 ++ kafka/record/abc.py | 10 +++ kafka/record/default_records.py | 45 +++++++++++++ kafka/record/legacy_records.py | 11 +++- test/test_fetcher.py | 10 +-- 8 files changed, 162 insertions(+), 24 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 508e35a0b..ed0c50a5d 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -12,7 +12,7 @@ import kafka.errors as Errors from kafka.future import Future from kafka.metrics.stats import Avg, Count, Max, Rate -from kafka.protocol.fetch import FetchRequest +from kafka.protocol.fetch import FetchRequest, AbortedTransaction from kafka.protocol.list_offsets import ( ListOffsetsRequest, OffsetResetStrategy, UNKNOWN_OFFSET ) @@ -28,6 +28,11 @@ READ_UNCOMMITTED = 0 READ_COMMITTED = 1 +ISOLATION_LEVEL_CONFIG = { + 'read_uncommitted': READ_UNCOMMITTED, + 'read_committed': READ_COMMITTED, +} + ConsumerRecord = collections.namedtuple("ConsumerRecord", ["topic", "partition", "leader_epoch", "offset", "timestamp", "timestamp_type", "key", "value", "headers", "checksum", "serialized_key_size", "serialized_value_size", "serialized_header_size"]) @@ -60,6 +65,7 @@ class Fetcher(six.Iterator): 'metric_group_prefix': 'consumer', 'retry_backoff_ms': 100, 'enable_incremental_fetch_sessions': True, + 'isolation_level': 'read_uncommitted', } def __init__(self, client, subscriptions, **configs): @@ -100,12 +106,18 @@ def __init__(self, client, subscriptions, **configs): consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may 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' """ self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: self.config[key] = configs[key] + if self.config['isolation_level'] not in ISOLATION_LEVEL_CONFIG: + raise Errors.KafkaConfigurationError('Unrecognized isolation_level') + self._client = client self._subscriptions = subscriptions self._completed_fetches = collections.deque() # Unparsed responses @@ -116,7 +128,7 @@ def __init__(self, client, subscriptions, **configs): self._sensors = FetchManagerMetrics(self.config['metrics'], self.config['metric_group_prefix']) else: self._sensors = None - self._isolation_level = READ_UNCOMMITTED + self._isolation_level = ISOLATION_LEVEL_CONFIG[self.config['isolation_level']] self._session_handlers = {} self._nodes_with_pending_fetch_requests = set() @@ -244,7 +256,7 @@ def _reset_offset(self, partition, timeout_ms=None): else: raise NoOffsetForPartitionError(partition) - log.debug("Resetting offset for partition %s to %s offset.", + log.debug("Resetting offset for partition %s to offset %s.", partition, strategy) offsets = self._retrieve_offsets({partition: timestamp}, timeout_ms=timeout_ms) @@ -765,14 +777,21 @@ def _parse_fetched_data(self, completed_fetch): return None records = MemoryRecords(completed_fetch.partition_data[-1]) + aborted_transactions = None + if completed_fetch.response_version >= 11: + aborted_transactions = completed_fetch.partition_data[-3] + elif completed_fetch.response_version >= 4: + aborted_transactions = completed_fetch.partition_data[-2] log.debug("Preparing to read %s bytes of data for partition %s with offset %d", records.size_in_bytes(), tp, fetch_offset) parsed_records = self.PartitionRecords(fetch_offset, tp, records, - self.config['key_deserializer'], - self.config['value_deserializer'], - self.config['check_crcs'], - completed_fetch.metric_aggregator, - self._on_partition_records_drain) + key_deserializer=self.config['key_deserializer'], + value_deserializer=self.config['value_deserializer'], + check_crcs=self.config['check_crcs'], + isolation_level=self._isolation_level, + aborted_transactions=aborted_transactions, + metric_aggregator=completed_fetch.metric_aggregator, + on_drain=self._on_partition_records_drain) if not records.has_next() and records.size_in_bytes() > 0: if completed_fetch.response_version < 3: # Implement the pre KIP-74 behavior of throwing a RecordTooLargeException. @@ -845,13 +864,23 @@ def close(self): self._next_partition_records.drain() class PartitionRecords(object): - def __init__(self, fetch_offset, tp, records, key_deserializer, value_deserializer, check_crcs, metric_aggregator, on_drain): + def __init__(self, fetch_offset, tp, records, + key_deserializer=None, value_deserializer=None, + check_crcs=True, isolation_level=READ_UNCOMMITTED, + aborted_transactions=None, # raw data from response / list of (producer_id, first_offset) tuples + metric_aggregator=None, on_drain=lambda x: None): self.fetch_offset = fetch_offset self.topic_partition = tp self.leader_epoch = -1 self.next_fetch_offset = fetch_offset self.bytes_read = 0 self.records_read = 0 + self.isolation_level = isolation_level + self.aborted_producer_ids = set() + self.aborted_transactions = collections.deque( + sorted([AbortedTransaction(*data) for data in aborted_transactions] if aborted_transactions else [], + key=lambda txn: txn.first_offset) + ) self.metric_aggregator = metric_aggregator self.check_crcs = check_crcs self.record_iterator = itertools.dropwhile( @@ -900,18 +929,35 @@ def _unpack_records(self, tp, records, key_deserializer, value_deserializer): "Record batch for partition %s at offset %s failed crc check" % ( self.topic_partition, batch.base_offset)) + # Try DefaultsRecordBatch / message log format v2 - # base_offset, last_offset_delta, and control batches + # base_offset, last_offset_delta, aborted transactions, and control batches if batch.magic == 2: self.leader_epoch = batch.leader_epoch + if self.isolation_level == READ_COMMITTED and batch.has_producer_id(): + # remove from the aborted transaction queue all aborted transactions which have begun + # before the current batch's last offset and add the associated producerIds to the + # aborted producer set + self._consume_aborted_transactions_up_to(batch.last_offset) + + producer_id = batch.producer_id + if self._contains_abort_marker(batch): + try: + self.aborted_producer_ids.remove(producer_id) + except KeyError: + pass + elif self._is_batch_aborted(batch): + log.debug("Skipping aborted record batch from partition %s with producer_id %s and" + " offsets %s to %s", + self.topic_partition, producer_id, batch.base_offset, batch.last_offset) + self.next_fetch_offset = batch.next_offset + batch = records.next_batch() + continue + # Control batches have a single record indicating whether a transaction - # was aborted or committed. - # When isolation_level is READ_COMMITTED (currently unsupported) - # we should also skip all messages from aborted transactions - # For now we only support READ_UNCOMMITTED and so we ignore the - # abort/commit signal. + # was aborted or committed. These are not returned to the consumer. if batch.is_control_batch: - self.next_fetch_offset = next(batch).offset + 1 + self.next_fetch_offset = batch.next_offset batch = records.next_batch() continue @@ -944,7 +990,7 @@ def _unpack_records(self, tp, records, key_deserializer, value_deserializer): # unnecessary re-fetching of the same batch (in the worst case, the consumer could get stuck # fetching the same batch repeatedly). if last_batch and last_batch.magic == 2: - self.next_fetch_offset = last_batch.base_offset + last_batch.last_offset_delta + 1 + self.next_fetch_offset = last_batch.next_offset self.drain() # If unpacking raises StopIteration, it is erroneously @@ -961,6 +1007,24 @@ def _deserialize(self, f, topic, bytes_): return f.deserialize(topic, bytes_) return f(bytes_) + def _consume_aborted_transactions_up_to(self, offset): + if not self.aborted_transactions: + return + + while self.aborted_transactions and self.aborted_transactions[0].first_offset <= offset: + self.aborted_producer_ids.add(self.aborted_transactions.popleft().producer_id) + + def _is_batch_aborted(self, batch): + return batch.is_transactional and batch.producer_id in self.aborted_producer_ids + + def _contains_abort_marker(self, batch): + if not batch.is_control_batch: + return False + record = next(batch) + if not record: + return False + return record.abort + class FetchSessionHandler(object): """ diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 4a39dc135..7fff6e795 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -121,6 +121,9 @@ class KafkaConsumer(six.Iterator): consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may 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' allow_auto_create_topics (bool): Enable/disable auto topic creation on metadata request. Only available with api_version >= (0, 11). Default: True @@ -290,6 +293,7 @@ class KafkaConsumer(six.Iterator): 'auto_commit_interval_ms': 5000, 'default_offset_commit_callback': lambda offsets, response: True, 'check_crcs': True, + 'isolation_level': 'read_uncommitted', 'allow_auto_create_topics': True, 'metadata_max_age_ms': 5 * 60 * 1000, 'partition_assignment_strategy': (RangePartitionAssignor, RoundRobinPartitionAssignor), diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 4cc21020e..0ff2ae91b 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -1,7 +1,7 @@ from __future__ import absolute_import import abc -from collections import defaultdict, OrderedDict +from collections import OrderedDict try: from collections.abc import Sequence except ImportError: diff --git a/kafka/protocol/fetch.py b/kafka/protocol/fetch.py index d193eafcf..036a37eb8 100644 --- a/kafka/protocol/fetch.py +++ b/kafka/protocol/fetch.py @@ -1,9 +1,15 @@ from __future__ import absolute_import +import collections + from kafka.protocol.api import Request, Response from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String, Bytes +AbortedTransaction = collections.namedtuple("AbortedTransaction", + ["producer_id", "first_offset"]) + + class FetchResponse_v0(Response): API_KEY = 1 API_VERSION = 0 diff --git a/kafka/record/abc.py b/kafka/record/abc.py index df7178562..c78f0da69 100644 --- a/kafka/record/abc.py +++ b/kafka/record/abc.py @@ -110,6 +110,16 @@ def __iter__(self): if needed. """ + @abc.abstractproperty + def base_offset(self): + """ Return base offset for batch + """ + + @abc.abstractproperty + def size_in_bytes(self): + """ Return size of batch in bytes (includes header overhead) + """ + @abc.abstractproperty def magic(self): """ Return magic value (0, 1, 2) for batch. diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index 855306bbd..2158c48cb 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -104,6 +104,9 @@ class DefaultRecordBase(object): LOG_APPEND_TIME = 1 CREATE_TIME = 0 + NO_PRODUCER_ID = -1 + NO_SEQUENCE = -1 + MAX_INT = 2147483647 def _assert_has_codec(self, compression_type): if compression_type == self.CODEC_GZIP: @@ -136,6 +139,10 @@ def __init__(self, buffer): def base_offset(self): return self._header_data[0] + @property + def size_in_bytes(self): + return self._header_data[1] + self.AFTER_LEN_OFFSET + @property def leader_epoch(self): return self._header_data[2] @@ -156,6 +163,14 @@ def attributes(self): def last_offset_delta(self): return self._header_data[6] + @property + def last_offset(self): + return self.base_offset + self.last_offset_delta + + @property + def next_offset(self): + return self.last_offset + 1 + @property def compression_type(self): return self.attributes & self.CODEC_MASK @@ -180,6 +195,36 @@ def first_timestamp(self): def max_timestamp(self): return self._header_data[8] + @property + def producer_id(self): + return self._header_data[9] + + def has_producer_id(self): + return self.producer_id > self.NO_PRODUCER_ID + + @property + def producer_epoch(self): + return self._header_data[10] + + @property + def base_sequence(self): + return self._header_data[11] + + @property + def last_sequence(self): + if self.base_sequence == self.NO_SEQUENCE: + return self.NO_SEQUENCE + return self._increment_sequence(self.base_sequence, self.last_offset_delta) + + def _increment_sequence(self, base, increment): + if base > (self.MAX_INT - increment): + return increment - (self.MAX_INT - base) - 1 + return base + increment + + @property + def records_count(self): + return self._header_data[12] + def _maybe_uncompress(self): if not self._decompressed: compression_type = self.compression_type diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index 920b4fcc6..c126374b8 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -129,7 +129,7 @@ def _assert_has_codec(self, compression_type): class LegacyRecordBatch(ABCRecordBatch, LegacyRecordBase): - __slots__ = ("_buffer", "_magic", "_offset", "_crc", "_timestamp", + __slots__ = ("_buffer", "_magic", "_offset", "_length", "_crc", "_timestamp", "_attributes", "_decompressed") def __init__(self, buffer, magic): @@ -141,11 +141,20 @@ def __init__(self, buffer, magic): assert magic == magic_ self._offset = offset + self._length = length self._crc = crc self._timestamp = timestamp self._attributes = attrs self._decompressed = False + @property + def base_offset(self): + return self._offset + + @property + def size_in_bytes(self): + return self._length + self.LOG_OVERHEAD + @property def timestamp_type(self): """0 for CreateTime; 1 for LogAppendTime; None if unsupported. diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 184acc9e1..3fc0c55ae 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -452,7 +452,7 @@ def test__unpack_records(mocker): (None, b"c", None), ] memory_records = MemoryRecords(_build_record_batch(messages)) - part_records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) + part_records = Fetcher.PartitionRecords(0, tp, memory_records) records = list(part_records.record_iterator) assert len(records) == 3 assert all(map(lambda x: isinstance(x, ConsumerRecord), records)) @@ -557,7 +557,7 @@ def test_partition_records_offset(mocker): tp = TopicPartition('foo', 0) messages = [(None, b'msg', None) for i in range(batch_start, batch_end)] memory_records = MemoryRecords(_build_record_batch(messages, offset=batch_start)) - records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records) assert records assert records.next_fetch_offset == fetch_offset msgs = records.take(1) @@ -574,7 +574,7 @@ def test_partition_records_offset(mocker): def test_partition_records_empty(mocker): tp = TopicPartition('foo', 0) memory_records = MemoryRecords(_build_record_batch([])) - records = Fetcher.PartitionRecords(0, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) + records = Fetcher.PartitionRecords(0, tp, memory_records) msgs = records.take() assert len(msgs) == 0 assert not records @@ -587,7 +587,7 @@ def test_partition_records_no_fetch_offset(mocker): tp = TopicPartition('foo', 0) messages = [(None, b'msg', None) for i in range(batch_start, batch_end)] memory_records = MemoryRecords(_build_record_batch(messages, offset=batch_start)) - records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records) msgs = records.take() assert len(msgs) == 0 assert not records @@ -611,7 +611,7 @@ def test_partition_records_compacted_offset(mocker): builder.append(key=None, value=b'msg', timestamp=None, headers=[]) builder.close() memory_records = MemoryRecords(builder.buffer()) - records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records, None, None, False, mocker.MagicMock(), lambda x: None) + records = Fetcher.PartitionRecords(fetch_offset, tp, memory_records) msgs = records.take() assert len(msgs) == batch_end - fetch_offset - 1 assert msgs[0].offset == fetch_offset + 1 From d2d1cdde5fcaaff7bea956b594fb1c058bc8871e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Apr 2025 15:02:31 -0700 Subject: [PATCH 116/205] Rename Coordinator errors to generic not group (#2585) --- kafka/admin/client.py | 4 ++-- kafka/coordinator/base.py | 22 +++++++++++----------- kafka/coordinator/consumer.py | 18 +++++++++--------- kafka/errors.py | 29 ++++++++++++----------------- test/test_admin_integration.py | 4 ++-- test/test_coordinator.py | 14 +++++++------- 6 files changed, 43 insertions(+), 48 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 392687be5..94de5a863 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -1460,9 +1460,9 @@ def list_consumer_groups(self, broker_ids=None): list: List of tuples of Consumer Groups. Raises: - GroupCoordinatorNotAvailableError: The coordinator is not + CoordinatorNotAvailableError: The coordinator is not available, so cannot process requests. - GroupLoadInProgressError: The coordinator is loading and + CoordinatorLoadInProgressError: The coordinator is loading and hence can't process requests. """ # While we return a list, internally use a set to prevent duplicates diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 0c238fde8..4f413c768 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -478,7 +478,7 @@ def _send_join_group_request(self): group leader """ if self.coordinator_unknown(): - e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) + e = Errors.CoordinatorNotAvailableError(self.coordinator_id) return Future().failure(e) elif not self._client.ready(self.coordinator_id, metadata_priority=False): @@ -555,7 +555,7 @@ def _handle_join_group_response(self, future, send_time, response): else: self._on_join_follower().chain(future) - elif error_type is Errors.GroupLoadInProgressError: + elif error_type is Errors.CoordinatorLoadInProgressError: log.debug("Attempt to join group %s rejected since coordinator %s" " is loading the group.", self.group_id, self.coordinator_id) # backoff and retry @@ -567,8 +567,8 @@ def _handle_join_group_response(self, future, send_time, response): log.debug("Attempt to join group %s failed due to unknown member id", self.group_id) future.failure(error) - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError): + elif error_type in (Errors.CoordinatorNotAvailableError, + Errors.NotCoordinatorError): # re-discover the coordinator and retry with backoff self.coordinator_dead(error_type()) log.debug("Attempt to join group %s failed due to obsolete " @@ -636,7 +636,7 @@ def _on_join_leader(self, response): def _send_sync_group_request(self, request): if self.coordinator_unknown(): - e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) + e = Errors.CoordinatorNotAvailableError(self.coordinator_id) return Future().failure(e) # We assume that coordinator is ready if we're sending SyncGroup @@ -674,8 +674,8 @@ def _handle_sync_group_response(self, future, send_time, response): log.debug("SyncGroup for group %s failed due to %s", self.group_id, error) self.reset_generation() future.failure(error) - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError): + elif error_type in (Errors.CoordinatorNotAvailableError, + Errors.NotCoordinatorError): error = error_type() log.debug("SyncGroup for group %s failed due to %s", self.group_id, error) self.coordinator_dead(error) @@ -732,7 +732,7 @@ def _handle_group_coordinator_response(self, future, response): self.heartbeat.reset_timeouts() future.success(self.coordinator_id) - elif error_type is Errors.GroupCoordinatorNotAvailableError: + elif error_type is Errors.CoordinatorNotAvailableError: log.debug("Group Coordinator Not Available; retry") future.failure(error_type()) elif error_type is Errors.GroupAuthorizationFailedError: @@ -842,7 +842,7 @@ def _handle_leave_group_response(self, response): def _send_heartbeat_request(self): """Send a heartbeat request""" if self.coordinator_unknown(): - e = Errors.GroupCoordinatorNotAvailableError(self.coordinator_id) + e = Errors.CoordinatorNotAvailableError(self.coordinator_id) return Future().failure(e) elif not self._client.ready(self.coordinator_id, metadata_priority=False): @@ -869,8 +869,8 @@ def _handle_heartbeat_response(self, future, send_time, response): log.debug("Received successful heartbeat response for group %s", self.group_id) future.success(None) - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError): + elif error_type in (Errors.CoordinatorNotAvailableError, + Errors.NotCoordinatorError): log.warning("Heartbeat failed for group %s: coordinator (node %s)" " is either not started or not valid", self.group_id, self.coordinator()) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 4bc7ba9cb..2944c7ec7 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -590,7 +590,7 @@ def _send_offset_commit_request(self, offsets): node_id = self.coordinator() if node_id is None: - return Future().failure(Errors.GroupCoordinatorNotAvailableError) + return Future().failure(Errors.CoordinatorNotAvailableError) # create the offset commit request @@ -719,14 +719,14 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): " %s", self.group_id, tp, error_type.__name__) future.failure(error_type()) return - elif error_type is Errors.GroupLoadInProgressError: + elif error_type is Errors.CoordinatorLoadInProgressError: # just retry log.debug("OffsetCommit for group %s failed: %s", self.group_id, error_type.__name__) future.failure(error_type(self.group_id)) return - elif error_type in (Errors.GroupCoordinatorNotAvailableError, - Errors.NotCoordinatorForGroupError, + elif error_type in (Errors.CoordinatorNotAvailableError, + Errors.NotCoordinatorError, Errors.RequestTimedOutError): log.debug("OffsetCommit for group %s failed: %s", self.group_id, error_type.__name__) @@ -777,7 +777,7 @@ def _send_offset_fetch_request(self, partitions): node_id = self.coordinator() if node_id is None: - return Future().failure(Errors.GroupCoordinatorNotAvailableError) + return Future().failure(Errors.CoordinatorNotAvailableError) # Verify node is ready if not self._client.ready(node_id): @@ -812,10 +812,10 @@ def _handle_offset_fetch_response(self, future, response): error_type = Errors.for_code(response.error_code) log.debug("Offset fetch failed: %s", error_type.__name__) error = error_type() - if error_type is Errors.GroupLoadInProgressError: + if error_type is Errors.CoordinatorLoadInProgressError: # Retry future.failure(error) - elif error_type is Errors.NotCoordinatorForGroupError: + elif error_type is Errors.NotCoordinatorError: # re-discover the coordinator and retry self.coordinator_dead(error) future.failure(error) @@ -841,10 +841,10 @@ def _handle_offset_fetch_response(self, future, response): error = error_type() log.debug("Group %s failed to fetch offset for partition" " %s: %s", self.group_id, tp, error) - if error_type is Errors.GroupLoadInProgressError: + if error_type is Errors.CoordinatorLoadInProgressError: # just retry future.failure(error) - elif error_type is Errors.NotCoordinatorForGroupError: + elif error_type is Errors.NotCoordinatorError: # re-discover the coordinator and retry self.coordinator_dead(error) future.failure(error) diff --git a/kafka/errors.py b/kafka/errors.py index aaba89d39..76a93568e 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -218,33 +218,28 @@ class NetworkExceptionError(BrokerResponseError): invalid_metadata = True -class GroupLoadInProgressError(BrokerResponseError): +class CoordinatorLoadInProgressError(BrokerResponseError): errno = 14 - message = 'OFFSETS_LOAD_IN_PROGRESS' - description = ('The broker returns this error code for an offset fetch' - ' request if it is still loading offsets (after a leader' - ' change for that offsets topic partition), or in response' - ' to group membership requests (such as heartbeats) when' - ' group metadata is being loaded by the coordinator.') + message = 'COORDINATOR_LOAD_IN_PROGRESS' + description = ('The broker returns this error code for txn or group requests,' + ' when the coordinator is loading and hence cant process requests') retriable = True -class GroupCoordinatorNotAvailableError(BrokerResponseError): +class CoordinatorNotAvailableError(BrokerResponseError): errno = 15 - message = 'CONSUMER_COORDINATOR_NOT_AVAILABLE' - description = ('The broker returns this error code for group coordinator' - ' requests, offset commits, and most group management' + message = 'COORDINATOR_NOT_AVAILABLE' + description = ('The broker returns this error code for consumer and transaction' ' requests if the offsets topic has not yet been created, or' - ' if the group coordinator is not active.') + ' if the group/txn coordinator is not active.') retriable = True -class NotCoordinatorForGroupError(BrokerResponseError): +class NotCoordinatorError(BrokerResponseError): errno = 16 - message = 'NOT_COORDINATOR_FOR_CONSUMER' - description = ('The broker returns this error code if it receives an offset' - ' fetch or commit request for a group that it is not a' - ' coordinator for.') + message = 'NOT_COORDINATOR' + description = ('The broker returns this error code if it is not the correct' + ' coordinator for the specified consumer or transaction group') retriable = True diff --git a/test/test_admin_integration.py b/test/test_admin_integration.py index 83b6ccaf2..f95f367e8 100644 --- a/test/test_admin_integration.py +++ b/test/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, GroupCoordinatorNotAvailableError, NonEmptyGroupError, + BrokerResponseError, KafkaError, NoError, CoordinatorNotAvailableError, NonEmptyGroupError, GroupIdNotFoundError, OffsetOutOfRangeError, UnknownTopicOrPartitionError) @@ -150,7 +150,7 @@ def test_describe_configs_invalid_broker_id_raises(kafka_admin_client): def test_describe_consumer_group_does_not_exist(kafka_admin_client): """Tests that the describe consumer group call fails if the group coordinator is not available """ - with pytest.raises(GroupCoordinatorNotAvailableError): + with pytest.raises(CoordinatorNotAvailableError): kafka_admin_client.describe_consumer_groups(['test']) diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 1d1a6df50..00a929399 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -444,7 +444,7 @@ def test_send_offset_commit_request_fail(mocker, patched_coord, offsets): # No coordinator ret = patched_coord._send_offset_commit_request(offsets) assert ret.failed() - assert isinstance(ret.exception, Errors.GroupCoordinatorNotAvailableError) + assert isinstance(ret.exception, Errors.CoordinatorNotAvailableError) @pytest.mark.parametrize('api_version,req_type', [ @@ -497,11 +497,11 @@ def test_send_offset_commit_request_success(mocker, patched_coord, offsets): (OffsetCommitResponse[0]([('foobar', [(0, 28), (1, 28)])]), Errors.InvalidCommitOffsetSizeError, False), (OffsetCommitResponse[0]([('foobar', [(0, 14), (1, 14)])]), - Errors.GroupLoadInProgressError, False), + Errors.CoordinatorLoadInProgressError, False), (OffsetCommitResponse[0]([('foobar', [(0, 15), (1, 15)])]), - Errors.GroupCoordinatorNotAvailableError, True), + Errors.CoordinatorNotAvailableError, True), (OffsetCommitResponse[0]([('foobar', [(0, 16), (1, 16)])]), - Errors.NotCoordinatorForGroupError, True), + Errors.NotCoordinatorError, True), (OffsetCommitResponse[0]([('foobar', [(0, 7), (1, 7)])]), Errors.RequestTimedOutError, True), (OffsetCommitResponse[0]([('foobar', [(0, 25), (1, 25)])]), @@ -557,7 +557,7 @@ def test_send_offset_fetch_request_fail(mocker, patched_coord, partitions): # No coordinator ret = patched_coord._send_offset_fetch_request(partitions) assert ret.failed() - assert isinstance(ret.exception, Errors.GroupCoordinatorNotAvailableError) + assert isinstance(ret.exception, Errors.CoordinatorNotAvailableError) @pytest.mark.parametrize('api_version,req_type', [ @@ -606,9 +606,9 @@ def test_send_offset_fetch_request_success(patched_coord, partitions): @pytest.mark.parametrize('response,error,dead', [ (OffsetFetchResponse[0]([('foobar', [(0, 123, '', 14), (1, 234, '', 14)])]), - Errors.GroupLoadInProgressError, False), + Errors.CoordinatorLoadInProgressError, False), (OffsetFetchResponse[0]([('foobar', [(0, 123, '', 16), (1, 234, '', 16)])]), - Errors.NotCoordinatorForGroupError, True), + Errors.NotCoordinatorError, True), (OffsetFetchResponse[0]([('foobar', [(0, 123, '', 25), (1, 234, '', 25)])]), Errors.UnknownMemberIdError, False), (OffsetFetchResponse[0]([('foobar', [(0, 123, '', 22), (1, 234, '', 22)])]), From 98d7137e30633d53cc97f036d9723365a8df0014 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Apr 2025 18:12:57 -0700 Subject: [PATCH 117/205] Rename CorruptRecordException -> CorruptRecordError --- kafka/consumer/fetcher.py | 6 +++--- kafka/errors.py | 4 ++-- kafka/record/default_records.py | 12 ++++++------ kafka/record/legacy_records.py | 4 ++-- kafka/record/memory_records.py | 4 ++-- test/record/test_records.py | 4 ++-- 6 files changed, 17 insertions(+), 17 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index ed0c50a5d..96bf3b79b 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -341,7 +341,7 @@ def fetched_records(self, max_records=None, update_offsets=True): Raises: OffsetOutOfRangeError: if no subscription offset_reset_strategy - CorruptRecordException: if message crc validation fails (check_crcs + CorruptRecordError: if message crc validation fails (check_crcs must be set to True) RecordTooLargeError: if a message is larger than the currently configured max_partition_fetch_bytes @@ -925,7 +925,7 @@ def _unpack_records(self, tp, records, key_deserializer, value_deserializer): last_batch = batch if self.check_crcs and not batch.validate_crc(): - raise Errors.CorruptRecordException( + raise Errors.CorruptRecordError( "Record batch for partition %s at offset %s failed crc check" % ( self.topic_partition, batch.base_offset)) @@ -963,7 +963,7 @@ def _unpack_records(self, tp, records, key_deserializer, value_deserializer): for record in batch: if self.check_crcs and not record.validate_crc(): - raise Errors.CorruptRecordException( + raise Errors.CorruptRecordError( "Record for partition %s at offset %s failed crc check" % ( self.topic_partition, record.offset)) key_size = len(record.key) if record.key is not None else -1 diff --git a/kafka/errors.py b/kafka/errors.py index 76a93568e..22dcb2eca 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -120,14 +120,14 @@ class OffsetOutOfRangeError(BrokerResponseError): ' maintained by the server for the given topic/partition.') -class CorruptRecordException(BrokerResponseError): +class CorruptRecordError(BrokerResponseError): errno = 2 message = 'CORRUPT_MESSAGE' description = ('This message has failed its CRC checksum, exceeds the' ' valid size, or is otherwise corrupt.') # Backward compatibility -InvalidMessageError = CorruptRecordException +CorruptRecordException = CorruptRecordError class UnknownTopicOrPartitionError(BrokerResponseError): diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index 2158c48cb..c8305c88e 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -60,7 +60,7 @@ from kafka.record.util import ( decode_varint, encode_varint, calc_crc32c, size_of_varint ) -from kafka.errors import CorruptRecordException, UnsupportedCodecError +from kafka.errors import CorruptRecordError, UnsupportedCodecError from kafka.codec import ( gzip_encode, snappy_encode, lz4_encode, zstd_encode, gzip_decode, snappy_decode, lz4_decode, zstd_decode @@ -288,14 +288,14 @@ def _read_msg( header_count, pos = decode_varint(buffer, pos) if header_count < 0: - raise CorruptRecordException("Found invalid number of record " + raise CorruptRecordError("Found invalid number of record " "headers {}".format(header_count)) headers = [] while header_count: # Header key is of type String, that can't be None h_key_len, pos = decode_varint(buffer, pos) if h_key_len < 0: - raise CorruptRecordException( + raise CorruptRecordError( "Invalid negative header key size {}".format(h_key_len)) h_key = buffer[pos: pos + h_key_len].decode("utf-8") pos += h_key_len @@ -313,7 +313,7 @@ def _read_msg( # validate whether we have read all header bytes in the current record if pos - start_pos != length: - raise CorruptRecordException( + raise CorruptRecordError( "Invalid record size: expected to read {} bytes in record " "payload, but instead read {}".format(length, pos - start_pos)) self._pos = pos @@ -332,14 +332,14 @@ def __iter__(self): def __next__(self): if self._next_record_index >= self._num_records: if self._pos != len(self._buffer): - raise CorruptRecordException( + raise CorruptRecordError( "{} unconsumed bytes after all records consumed".format( len(self._buffer) - self._pos)) raise StopIteration try: msg = self._read_msg() except (ValueError, IndexError) as err: - raise CorruptRecordException( + raise CorruptRecordError( "Found invalid record structure: {!r}".format(err)) else: self._next_record_index += 1 diff --git a/kafka/record/legacy_records.py b/kafka/record/legacy_records.py index c126374b8..f085978f0 100644 --- a/kafka/record/legacy_records.py +++ b/kafka/record/legacy_records.py @@ -52,7 +52,7 @@ gzip_decode, snappy_decode, lz4_decode, lz4_decode_old_kafka, ) import kafka.codec as codecs -from kafka.errors import CorruptRecordException, UnsupportedCodecError +from kafka.errors import CorruptRecordError, UnsupportedCodecError class LegacyRecordBase(object): @@ -191,7 +191,7 @@ def _decompress(self, key_offset): value_size = struct.unpack_from(">i", self._buffer, pos)[0] pos += self.VALUE_LENGTH if value_size == -1: - raise CorruptRecordException("Value of compressed message is None") + raise CorruptRecordError("Value of compressed message is None") else: data = self._buffer[pos:pos + value_size] diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index a803047ea..77e38b9ed 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -22,7 +22,7 @@ import struct -from kafka.errors import CorruptRecordException, IllegalStateError, UnsupportedVersionError +from kafka.errors import CorruptRecordError, IllegalStateError, UnsupportedVersionError from kafka.record.abc import ABCRecords from kafka.record.legacy_records import LegacyRecordBatch, LegacyRecordBatchBuilder from kafka.record.default_records import DefaultRecordBatch, DefaultRecordBatchBuilder @@ -99,7 +99,7 @@ def next_batch(self, _min_slice=MIN_SLICE, if next_slice is None: return None if len(next_slice) < _min_slice: - raise CorruptRecordException( + raise CorruptRecordError( "Record size is less than the minimum record overhead " "({})".format(_min_slice - self.LOG_OVERHEAD)) self._cache_next() diff --git a/test/record/test_records.py b/test/record/test_records.py index dc9c95ff8..65010d88f 100644 --- a/test/record/test_records.py +++ b/test/record/test_records.py @@ -2,7 +2,7 @@ from __future__ import unicode_literals import pytest from kafka.record import MemoryRecords, MemoryRecordsBuilder -from kafka.errors import CorruptRecordException +from kafka.errors import CorruptRecordError from test.testutil import maybe_skip_unsupported_compression @@ -174,7 +174,7 @@ def test_memory_records_corrupt(): b"\x00\x00\x00\x03" # Length=3 b"\xfe\xb0\x1d", # Some random bytes ) - with pytest.raises(CorruptRecordException): + with pytest.raises(CorruptRecordError): records.next_batch() From c964f8fea127039d5d0df55e511f446c6b72b200 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Apr 2025 18:13:34 -0700 Subject: [PATCH 118/205] Dont raise KeyError on incomplete.remove(batch) --- kafka/producer/record_accumulator.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 60fa0a323..6490f48aa 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -588,11 +588,14 @@ def __init__(self): def add(self, batch): with self._lock: - return self._incomplete.add(batch) + self._incomplete.add(batch) def remove(self, batch): with self._lock: - return self._incomplete.remove(batch) + try: + self._incomplete.remove(batch) + except KeyError: + pass def all(self): with self._lock: From cf1a9994e6556953856d3e0cc3baf4616ec01458 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 5 Apr 2025 12:21:17 -0700 Subject: [PATCH 119/205] Cluster.add_group_coordinator -> add_coordinator + support txn type --- kafka/cluster.py | 27 ++++++++++++++------------- kafka/coordinator/base.py | 2 +- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index 9d2115859..c92d1d05b 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -47,7 +47,7 @@ def __init__(self, **configs): self._brokers = {} # node_id -> BrokerMetadata self._partitions = {} # topic -> partition -> PartitionMetadata self._broker_partitions = collections.defaultdict(set) # node_id -> {TopicPartition...} - self._groups = {} # group_name -> node_id + self._coordinators = {} # (coord_type, coord_key) -> node_id self._last_refresh_ms = 0 self._last_successful_refresh_ms = 0 self._need_update = True @@ -167,7 +167,7 @@ def coordinator_for_group(self, group): node_id (int or str) for group coordinator, -1 if coordinator unknown None if the group does not exist. """ - return self._groups.get(group) + return self._coordinators.get(('group', group)) def ttl(self): """Milliseconds until metadata should be refreshed""" @@ -364,24 +364,25 @@ def remove_listener(self, listener): """Remove a previously added listener callback""" self._listeners.remove(listener) - def add_group_coordinator(self, group, response): - """Update with metadata for a group coordinator + def add_coordinator(self, response, coord_type, coord_key): + """Update with metadata for a group or txn coordinator Arguments: - group (str): name of group from FindCoordinatorRequest response (FindCoordinatorResponse): broker response + coord_type (str): 'group' or 'transaction' + coord_key (str): consumer_group or transactional_id Returns: string: coordinator node_id if metadata is updated, None on error """ - log.debug("Updating coordinator for %s: %s", group, response) + log.debug("Updating coordinator for %s/%s: %s", coord_type, coord_key, response) error_type = Errors.for_code(response.error_code) if error_type is not Errors.NoError: log.error("FindCoordinatorResponse error: %s", error_type) - self._groups[group] = -1 + self._coordinators[(coord_type, coord_key)] = -1 return - # Use a coordinator-specific node id so that group requests + # Use a coordinator-specific node id so that requests # get a dedicated connection node_id = 'coordinator-{}'.format(response.coordinator_id) coordinator = BrokerMetadata( @@ -390,9 +391,9 @@ def add_group_coordinator(self, group, response): response.port, None) - log.info("Group coordinator for %s is %s", group, coordinator) + log.info("Coordinator for %s/%s is %s", coord_type, coord_key, coordinator) self._coordinator_brokers[node_id] = coordinator - self._groups[group] = node_id + self._coordinators[(coord_type, coord_key)] = node_id return node_id def with_partitions(self, partitions_to_add): @@ -401,7 +402,7 @@ def with_partitions(self, partitions_to_add): new_metadata._brokers = copy.deepcopy(self._brokers) new_metadata._partitions = copy.deepcopy(self._partitions) new_metadata._broker_partitions = copy.deepcopy(self._broker_partitions) - new_metadata._groups = copy.deepcopy(self._groups) + new_metadata._coordinators = copy.deepcopy(self._coordinators) new_metadata.internal_topics = copy.deepcopy(self.internal_topics) new_metadata.unauthorized_topics = copy.deepcopy(self.unauthorized_topics) @@ -415,5 +416,5 @@ def with_partitions(self, partitions_to_add): return new_metadata def __str__(self): - return 'ClusterMetadata(brokers: %d, topics: %d, groups: %d)' % \ - (len(self._brokers), len(self._partitions), len(self._groups)) + return 'ClusterMetadata(brokers: %d, topics: %d, coordinators: %d)' % \ + (len(self._brokers), len(self._partitions), len(self._coordinators)) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 4f413c768..ad644aa52 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -718,7 +718,7 @@ def _handle_group_coordinator_response(self, future, response): error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: with self._lock: - coordinator_id = self._client.cluster.add_group_coordinator(self.group_id, response) + coordinator_id = self._client.cluster.add_coordinator(response, 'group', self.group_id) if not coordinator_id: # This could happen if coordinator metadata is different # than broker metadata From f5e4fa677975d38a586338a805ac12dd4b6cea39 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 7 Apr 2025 12:40:19 -0700 Subject: [PATCH 120/205] Use SaslAuthenticationFailedError in kafka.conn connection failure Drop unused AuthenticationFailedError --- kafka/conn.py | 2 +- kafka/errors.py | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 1febb479a..85a9658d4 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -813,7 +813,7 @@ def _sasl_authenticate(self, future): log.info('%s: %s', self, self._sasl_mechanism.auth_details()) return future.success(True) else: - return future.failure(Errors.AuthenticationFailedError('Failed to authenticate via SASL %s' % self.config['sasl_mechanism'])) + return future.failure(Errors.SaslAuthenticationFailedError('Failed to authenticate via SASL %s' % self.config['sasl_mechanism'])) def blacked_out(self): """ diff --git a/kafka/errors.py b/kafka/errors.py index 22dcb2eca..ea17d6ae2 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -85,10 +85,6 @@ class AuthenticationMethodNotSupported(KafkaError): pass -class AuthenticationFailedError(KafkaError): - retriable = False - - class BrokerResponseError(KafkaError): errno = None message = None From 3962d67bf8fc83d7e0a48ae9215563093cbe74a3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 4 Apr 2025 17:03:05 -0700 Subject: [PATCH 121/205] Add protocol defs for AddPartitionsToTxn, EndTxn, and AddOffsetsToTxn --- kafka/protocol/add_offsets_to_txn.py | 59 +++++++++++++++++++++++ kafka/protocol/add_partitions_to_txn.py | 63 +++++++++++++++++++++++++ kafka/protocol/end_txn.py | 58 +++++++++++++++++++++++ 3 files changed, 180 insertions(+) create mode 100644 kafka/protocol/add_offsets_to_txn.py create mode 100644 kafka/protocol/add_partitions_to_txn.py create mode 100644 kafka/protocol/end_txn.py diff --git a/kafka/protocol/add_offsets_to_txn.py b/kafka/protocol/add_offsets_to_txn.py new file mode 100644 index 000000000..fa2509330 --- /dev/null +++ b/kafka/protocol/add_offsets_to_txn.py @@ -0,0 +1,59 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Int16, Int32, Int64, Schema, String + + +class AddOffsetsToTxnResponse_v0(Response): + API_KEY = 25 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ) + + +class AddOffsetsToTxnResponse_v1(Response): + API_KEY = 25 + API_VERSION = 1 + SCHEMA = AddOffsetsToTxnResponse_v0.SCHEMA + + +class AddOffsetsToTxnResponse_v2(Response): + API_KEY = 25 + API_VERSION = 2 + SCHEMA = AddOffsetsToTxnResponse_v1.SCHEMA + + +class AddOffsetsToTxnRequest_v0(Request): + API_KEY = 25 + API_VERSION = 0 + RESPONSE_TYPE = AddOffsetsToTxnResponse_v0 + SCHEMA = Schema( + ('transactional_id', String('utf-8')), + ('producer_id', Int64), + ('producer_epoch', Int16), + ('group_id', String('utf-8')), + ) + + +class AddOffsetsToTxnRequest_v1(Request): + API_KEY = 25 + API_VERSION = 1 + RESPONSE_TYPE = AddOffsetsToTxnResponse_v1 + SCHEMA = AddOffsetsToTxnRequest_v0.SCHEMA + + +class AddOffsetsToTxnRequest_v2(Request): + API_KEY = 25 + API_VERSION = 2 + RESPONSE_TYPE = AddOffsetsToTxnResponse_v2 + SCHEMA = AddOffsetsToTxnRequest_v1.SCHEMA + + +AddOffsetsToTxnRequest = [ + AddOffsetsToTxnRequest_v0, AddOffsetsToTxnRequest_v1, AddOffsetsToTxnRequest_v2, +] +AddOffsetsToTxnResponse = [ + AddOffsetsToTxnResponse_v0, AddOffsetsToTxnResponse_v1, AddOffsetsToTxnResponse_v2, +] diff --git a/kafka/protocol/add_partitions_to_txn.py b/kafka/protocol/add_partitions_to_txn.py new file mode 100644 index 000000000..fdf28f4ae --- /dev/null +++ b/kafka/protocol/add_partitions_to_txn.py @@ -0,0 +1,63 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Int16, Int32, Int64, Schema, String + + +class AddPartitionsToTxnResponse_v0(Response): + API_KEY = 24 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('results', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16)))))) + + +class AddPartitionsToTxnResponse_v1(Response): + API_KEY = 24 + API_VERSION = 1 + SCHEMA = AddPartitionsToTxnResponse_v0.SCHEMA + + +class AddPartitionsToTxnResponse_v2(Response): + API_KEY = 24 + API_VERSION = 2 + SCHEMA = AddPartitionsToTxnResponse_v1.SCHEMA + + +class AddPartitionsToTxnRequest_v0(Request): + API_KEY = 24 + API_VERSION = 0 + RESPONSE_TYPE = AddPartitionsToTxnResponse_v0 + SCHEMA = Schema( + ('transactional_id', String('utf-8')), + ('producer_id', Int64), + ('producer_epoch', Int16), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array(Int32))))) + + +class AddPartitionsToTxnRequest_v1(Request): + API_KEY = 24 + API_VERSION = 1 + RESPONSE_TYPE = AddPartitionsToTxnResponse_v1 + SCHEMA = AddPartitionsToTxnRequest_v0.SCHEMA + + +class AddPartitionsToTxnRequest_v2(Request): + API_KEY = 24 + API_VERSION = 2 + RESPONSE_TYPE = AddPartitionsToTxnResponse_v2 + SCHEMA = AddPartitionsToTxnRequest_v1.SCHEMA + + +AddPartitionsToTxnRequest = [ + AddPartitionsToTxnRequest_v0, AddPartitionsToTxnRequest_v1, AddPartitionsToTxnRequest_v2, +] +AddPartitionsToTxnResponse = [ + AddPartitionsToTxnResponse_v0, AddPartitionsToTxnResponse_v1, AddPartitionsToTxnResponse_v2, +] diff --git a/kafka/protocol/end_txn.py b/kafka/protocol/end_txn.py new file mode 100644 index 000000000..96d6cc514 --- /dev/null +++ b/kafka/protocol/end_txn.py @@ -0,0 +1,58 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Boolean, Int16, Int32, Int64, Schema, String + + +class EndTxnResponse_v0(Response): + API_KEY = 26 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('error_code', Int16), + ) + + +class EndTxnResponse_v1(Response): + API_KEY = 26 + API_VERSION = 1 + SCHEMA = EndTxnResponse_v0.SCHEMA + + +class EndTxnResponse_v2(Response): + API_KEY = 26 + API_VERSION = 2 + SCHEMA = EndTxnResponse_v1.SCHEMA + + +class EndTxnRequest_v0(Request): + API_KEY = 26 + API_VERSION = 0 + RESPONSE_TYPE = EndTxnResponse_v0 + SCHEMA = Schema( + ('transactional_id', String('utf-8')), + ('producer_id', Int64), + ('producer_epoch', Int16), + ('committed', Boolean)) + + +class EndTxnRequest_v1(Request): + API_KEY = 26 + API_VERSION = 1 + RESPONSE_TYPE = EndTxnResponse_v1 + SCHEMA = EndTxnRequest_v0.SCHEMA + + +class EndTxnRequest_v2(Request): + API_KEY = 26 + API_VERSION = 2 + RESPONSE_TYPE = EndTxnResponse_v2 + SCHEMA = EndTxnRequest_v1.SCHEMA + + +EndTxnRequest = [ + EndTxnRequest_v0, EndTxnRequest_v1, EndTxnRequest_v2, +] +EndTxnResponse = [ + EndTxnResponse_v0, EndTxnResponse_v1, EndTxnResponse_v2, +] From 9c2dfab87abc5be2a77bf20503f924981bc6ae48 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 8 Apr 2025 10:57:03 -0700 Subject: [PATCH 122/205] Expand Sender test coverage (#2586) --- test/test_sender.py | 192 +++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 183 insertions(+), 9 deletions(-) diff --git a/test/test_sender.py b/test/test_sender.py index eedc43d25..a1a775b59 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -1,15 +1,23 @@ # pylint: skip-file from __future__ import absolute_import -import pytest +import collections import io +import time + +import pytest +from unittest.mock import call + +from kafka.vendor import six from kafka.client_async import KafkaClient +import kafka.errors as Errors 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.transaction_state import TransactionState from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition @@ -20,8 +28,18 @@ def accumulator(): @pytest.fixture -def sender(client, accumulator, metrics, mocker): - return Sender(client, client.cluster, accumulator, metrics=metrics) +def sender(client, accumulator): + return Sender(client, client.cluster, accumulator) + + +def producer_batch(topic='foo', partition=0, magic=2): + tp = TopicPartition(topic, partition) + records = MemoryRecordsBuilder( + magic=magic, compression_type=0, batch_size=100000) + batch = ProducerBatch(tp, records) + batch.try_append(0, None, b'msg', []) + batch.records.close() + return batch @pytest.mark.parametrize(("api_version", "produce_version"), [ @@ -30,13 +48,169 @@ def sender(client, accumulator, metrics, mocker): ((0, 9), 1), ((0, 8, 0), 0) ]) -def test_produce_request(sender, mocker, api_version, produce_version): +def test_produce_request(sender, api_version, produce_version): sender._client._api_versions = BROKER_API_VERSIONS[api_version] - tp = TopicPartition('foo', 0) magic = KafkaProducer.max_usable_produce_magic(api_version) - records = MemoryRecordsBuilder( - magic=1, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records) - records.close() + batch = producer_batch(magic=magic) produce_request = sender._produce_request(0, 0, 0, [batch]) assert isinstance(produce_request, ProduceRequest[produce_version]) + + +@pytest.mark.parametrize(("api_version", "produce_version"), [ + ((2, 1), 7), +]) +def test_create_produce_requests(sender, api_version, produce_version): + sender._client._api_versions = BROKER_API_VERSIONS[api_version] + tp = TopicPartition('foo', 0) + magic = KafkaProducer.max_usable_produce_magic(api_version) + batches_by_node = collections.defaultdict(list) + for node in range(3): + for _ in range(5): + batches_by_node[node].append(producer_batch(magic=magic)) + produce_requests_by_node = sender._create_produce_requests(batches_by_node) + assert len(produce_requests_by_node) == 3 + for node in range(3): + assert isinstance(produce_requests_by_node[node], ProduceRequest[produce_version]) + + +def test_complete_batch_success(sender): + batch = producer_batch() + assert not batch.produce_future.is_done + + # No error, base_offset 0 + sender._complete_batch(batch, None, 0, timestamp_ms=123, log_start_offset=456) + assert batch.is_done + assert batch.produce_future.is_done + assert batch.produce_future.succeeded() + assert batch.produce_future.value == (0, 123, 456) + + +def test_complete_batch_transaction(sender): + sender._transaction_state = TransactionState() + batch = producer_batch() + assert sender._transaction_state.sequence_number(batch.topic_partition) == 0 + assert sender._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id + + # No error, base_offset 0 + sender._complete_batch(batch, None, 0) + assert batch.is_done + assert sender._transaction_state.sequence_number(batch.topic_partition) == batch.record_count + + +@pytest.mark.parametrize(("error", "refresh_metadata"), [ + (Errors.KafkaConnectionError, True), + (Errors.CorruptRecordError, False), + (Errors.UnknownTopicOrPartitionError, True), + (Errors.NotLeaderForPartitionError, True), + (Errors.MessageSizeTooLargeError, False), + (Errors.InvalidTopicError, False), + (Errors.RecordListTooLargeError, False), + (Errors.NotEnoughReplicasError, False), + (Errors.NotEnoughReplicasAfterAppendError, False), + (Errors.InvalidRequiredAcksError, False), + (Errors.TopicAuthorizationFailedError, False), + (Errors.UnsupportedForMessageFormatError, False), + (Errors.InvalidProducerEpochError, False), + (Errors.ClusterAuthorizationFailedError, False), + (Errors.TransactionalIdAuthorizationFailedError, False), +]) +def test_complete_batch_error(sender, error, refresh_metadata): + sender._client.cluster._last_successful_refresh_ms = (time.time() - 10) * 1000 + sender._client.cluster._need_update = False + assert sender._client.cluster.ttl() > 0 + batch = producer_batch() + sender._complete_batch(batch, error, -1) + 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) + + +@pytest.mark.parametrize(("error", "retry"), [ + (Errors.KafkaConnectionError, True), + (Errors.CorruptRecordError, False), + (Errors.UnknownTopicOrPartitionError, True), + (Errors.NotLeaderForPartitionError, True), + (Errors.MessageSizeTooLargeError, False), + (Errors.InvalidTopicError, False), + (Errors.RecordListTooLargeError, False), + (Errors.NotEnoughReplicasError, True), + (Errors.NotEnoughReplicasAfterAppendError, True), + (Errors.InvalidRequiredAcksError, False), + (Errors.TopicAuthorizationFailedError, False), + (Errors.UnsupportedForMessageFormatError, False), + (Errors.InvalidProducerEpochError, False), + (Errors.ClusterAuthorizationFailedError, False), + (Errors.TransactionalIdAuthorizationFailedError, False), +]) +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) + 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) + assert batch.is_done + assert isinstance(batch.produce_future.exception, error) + else: + assert batch.is_done + assert isinstance(batch.produce_future.exception, error) + + +def test_complete_batch_producer_id_changed_no_retry(sender, accumulator, mocker): + sender._transaction_state = TransactionState() + 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) + 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_state.producer_id_and_epoch.producer_id + sender._complete_batch(batch, error, -1) + assert batch.is_done + assert isinstance(batch.produce_future.exception, error) + + +def test_fail_batch(sender, accumulator, mocker): + sender._transaction_state = TransactionState() + mocker.patch.object(TransactionState, 'reset_producer_id') + batch = producer_batch() + mocker.patch.object(batch, 'done') + assert sender._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id + error = Exception('error') + sender._fail_batch(batch, base_offset=0, timestamp_ms=None, exception=error, log_start_offset=None) + sender._transaction_state.reset_producer_id.assert_called_once() + batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error, log_start_offset=None) + + +def test_handle_produce_response(): + pass + + +def test_failed_produce(sender, mocker): + mocker.patch.object(sender, '_complete_batch') + 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), + ]) + + +def test_maybe_wait_for_producer_id(): + pass + + +def test_run_once(): + pass From 614b059c50887e82aa1018dae546363bed4e04f0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 15 Apr 2025 14:52:19 -0700 Subject: [PATCH 123/205] Move integration tests and fixtures to test/integration/; simplify unit fixtures (#2588) --- Makefile | 4 +- test/conftest.py | 179 +----------- test/integration/conftest.py | 168 +++++++++++ test/{ => integration}/fixtures.py | 2 +- .../test_admin_integration.py | 0 test/{ => integration}/test_consumer_group.py | 0 .../test_consumer_integration.py | 0 .../test_producer_integration.py} | 0 .../test_sasl_integration.py | 0 test/test_client_async.py | 271 +++++++++--------- test/test_coordinator.py | 8 - test/test_fetcher.py | 5 - test/test_metrics.py | 20 +- 13 files changed, 315 insertions(+), 342 deletions(-) create mode 100644 test/integration/conftest.py rename test/{ => integration}/fixtures.py (99%) rename test/{ => integration}/test_admin_integration.py (100%) rename test/{ => integration}/test_consumer_group.py (100%) rename test/{ => integration}/test_consumer_integration.py (100%) rename test/{test_producer.py => integration/test_producer_integration.py} (100%) rename test/{ => integration}/test_sasl_integration.py (100%) diff --git a/Makefile b/Makefile index a624b833f..30da9cf91 100644 --- a/Makefile +++ b/Makefile @@ -24,7 +24,7 @@ test: build-integration pytest $(PYTESTS) fixture: build-integration - python -m test.fixtures kafka + python -m test.integration.fixtures kafka cov-local: build-integration pytest --pylint --pylint-rcfile=pylint.rc --pylint-error-types=EF --cov=kafka \ @@ -99,7 +99,7 @@ servers/%/kafka-bin: servers/dist/$$(call kafka_artifact_name,$$*) | servers/dis if [[ "$*" < "1" ]]; then make servers/patch-libs/$*; fi servers/%/api_versions: servers/$$*/kafka-bin - KAFKA_VERSION=$* python -m test.fixtures get_api_versions >$@ + KAFKA_VERSION=$* python -m test.integration.fixtures get_api_versions >$@ servers/%/messages: servers/$$*/kafka-bin cd servers/$*/ && jar xvf kafka-bin/libs/kafka-clients-$*.jar common/message/ diff --git a/test/conftest.py b/test/conftest.py index ba76d6cc5..b65593a86 100644 --- a/test/conftest.py +++ b/test/conftest.py @@ -1,147 +1,17 @@ from __future__ import absolute_import -import os -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.fixtures import KafkaFixture, ZookeeperFixture - - -@pytest.fixture(scope="module") -def zookeeper(): - """Return a Zookeeper fixture""" - if "ZOOKEEPER_URI" in os.environ: - parse = urlparse(os.environ["ZOOKEEPER_URI"]) - (host, port) = (parse.hostname, parse.port) - yield ZookeeperFixture.instance(host=host, port=port, external=True) - else: - zk_instance = ZookeeperFixture.instance() - yield zk_instance - zk_instance.close() - - -@pytest.fixture(scope="module") -def kafka_broker(kafka_broker_factory): - """Return a Kafka broker fixture""" - if "KAFKA_URI" in os.environ: - parse = urlparse(os.environ["KAFKA_URI"]) - (host, port) = (parse.hostname, parse.port) - return KafkaFixture.instance(0, host=host, port=port, external=True) - else: - return kafka_broker_factory() - - -@pytest.fixture(scope="module") -def kafka_broker_factory(): - """Return a Kafka broker fixture factory""" - assert env_kafka_version(), 'KAFKA_VERSION must be specified to run integration tests' - - _brokers = [] - def factory(**broker_params): - params = {} if broker_params is None else broker_params.copy() - params.setdefault('partitions', 4) - node_id = params.pop('node_id', 0) - broker = KafkaFixture.instance(node_id, **params) - _brokers.append(broker) - return broker - - yield factory - - zks = set() - for broker in _brokers: - zks.add(broker.zookeeper) - broker.close() - for zk in zks: - if zk: - zk.close() - - -@pytest.fixture -def kafka_client(kafka_broker, request): - """Return a KafkaClient fixture""" - (client,) = kafka_broker.get_clients(cnt=1, client_id='%s_client' % (request.node.name,)) - yield client - client.close() - - -@pytest.fixture -def kafka_consumer(kafka_consumer_factory): - """Return a KafkaConsumer fixture""" - return kafka_consumer_factory() - - -@pytest.fixture -def kafka_consumer_factory(kafka_broker, topic, request): - """Return a KafkaConsumer factory fixture""" - _consumer = [None] - - def factory(topics=(topic,), **kafka_consumer_params): - params = {} if kafka_consumer_params is None else kafka_consumer_params.copy() - params.setdefault('client_id', 'consumer_%s' % (request.node.name,)) - params.setdefault('auto_offset_reset', 'earliest') - _consumer[0] = next(kafka_broker.get_consumers(cnt=1, topics=list(topics), **params)) - return _consumer[0] - - yield factory - - if _consumer[0]: - _consumer[0].close() - - -@pytest.fixture -def kafka_producer(kafka_producer_factory): - """Return a KafkaProducer fixture""" - yield kafka_producer_factory() - - -@pytest.fixture -def kafka_producer_factory(kafka_broker, request): - """Return a KafkaProduce factory fixture""" - _producer = [None] - - def factory(**kafka_producer_params): - params = {} if kafka_producer_params is None else kafka_producer_params.copy() - params.setdefault('client_id', 'producer_%s' % (request.node.name,)) - _producer[0] = next(kafka_broker.get_producers(cnt=1, **params)) - return _producer[0] - - yield factory - - if _producer[0]: - _producer[0].close() - - -@pytest.fixture -def kafka_admin_client(kafka_admin_client_factory): - """Return a KafkaAdminClient fixture""" - yield kafka_admin_client_factory() - @pytest.fixture -def kafka_admin_client_factory(kafka_broker): - """Return a KafkaAdminClient factory fixture""" - _admin_client = [None] - - def factory(**kafka_admin_client_params): - params = {} if kafka_admin_client_params is None else kafka_admin_client_params.copy() - _admin_client[0] = next(kafka_broker.get_admin_clients(cnt=1, **params)) - return _admin_client[0] - - yield factory - - if _admin_client[0]: - _admin_client[0].close() - +def metrics(): + from kafka.metrics import Metrics -@pytest.fixture -def topic(kafka_broker, request): - """Return a topic fixture""" - topic_name = '%s_%s' % (request.node.name, random_string(10)) - kafka_broker.create_topics([topic_name]) - return topic_name + metrics = Metrics() + try: + yield metrics + finally: + metrics.close() @pytest.fixture @@ -173,41 +43,6 @@ def _set_conn_state(state): return conn -@pytest.fixture() -def send_messages(topic, kafka_producer, request): - """A factory that returns a send_messages function with a pre-populated - topic topic / producer.""" - - def _send_messages(number_range, partition=0, topic=topic, producer=kafka_producer, request=request): - """ - messages is typically `range(0,100)` - partition is an int - """ - messages_and_futures = [] # [(message, produce_future),] - for i in number_range: - # request.node.name provides the test name (including parametrized values) - encoded_msg = '{}-{}-{}'.format(i, request.node.name, uuid.uuid4()).encode('utf-8') - future = kafka_producer.send(topic, value=encoded_msg, partition=partition) - messages_and_futures.append((encoded_msg, future)) - kafka_producer.flush() - for (msg, f) in messages_and_futures: - assert f.succeeded() - return [msg for (msg, f) in messages_and_futures] - - return _send_messages - - -@pytest.fixture -def metrics(): - from kafka.metrics import Metrics - - metrics = Metrics() - try: - yield metrics - finally: - metrics.close() - - @pytest.fixture def client(conn, mocker): from kafka import KafkaClient diff --git a/test/integration/conftest.py b/test/integration/conftest.py new file mode 100644 index 000000000..8af729296 --- /dev/null +++ b/test/integration/conftest.py @@ -0,0 +1,168 @@ +from __future__ import absolute_import + +import os +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 + + +@pytest.fixture(scope="module") +def zookeeper(): + """Return a Zookeeper fixture""" + if "ZOOKEEPER_URI" in os.environ: + parse = urlparse(os.environ["ZOOKEEPER_URI"]) + (host, port) = (parse.hostname, parse.port) + yield ZookeeperFixture.instance(host=host, port=port, external=True) + else: + zk_instance = ZookeeperFixture.instance() + yield zk_instance + zk_instance.close() + + +@pytest.fixture(scope="module") +def kafka_broker(kafka_broker_factory): + """Return a Kafka broker fixture""" + if "KAFKA_URI" in os.environ: + parse = urlparse(os.environ["KAFKA_URI"]) + (host, port) = (parse.hostname, parse.port) + return KafkaFixture.instance(0, host=host, port=port, external=True) + else: + return kafka_broker_factory() + + +@pytest.fixture(scope="module") +def kafka_broker_factory(): + """Return a Kafka broker fixture factory""" + assert env_kafka_version(), 'KAFKA_VERSION must be specified to run integration tests' + + _brokers = [] + def factory(**broker_params): + params = {} if broker_params is None else broker_params.copy() + params.setdefault('partitions', 4) + node_id = params.pop('node_id', 0) + broker = KafkaFixture.instance(node_id, **params) + _brokers.append(broker) + return broker + + yield factory + + zks = set() + for broker in _brokers: + zks.add(broker.zookeeper) + broker.close() + for zk in zks: + if zk: + zk.close() + + +@pytest.fixture +def kafka_client(kafka_broker, request): + """Return a KafkaClient fixture""" + (client,) = kafka_broker.get_clients(cnt=1, client_id='%s_client' % (request.node.name,)) + yield client + client.close() + + +@pytest.fixture +def kafka_consumer(kafka_consumer_factory): + """Return a KafkaConsumer fixture""" + return kafka_consumer_factory() + + +@pytest.fixture +def kafka_consumer_factory(kafka_broker, topic, request): + """Return a KafkaConsumer factory fixture""" + _consumer = [None] + + def factory(topics=(topic,), **kafka_consumer_params): + params = {} if kafka_consumer_params is None else kafka_consumer_params.copy() + params.setdefault('client_id', 'consumer_%s' % (request.node.name,)) + params.setdefault('auto_offset_reset', 'earliest') + _consumer[0] = next(kafka_broker.get_consumers(cnt=1, topics=list(topics), **params)) + return _consumer[0] + + yield factory + + if _consumer[0]: + _consumer[0].close() + + +@pytest.fixture +def kafka_producer(kafka_producer_factory): + """Return a KafkaProducer fixture""" + yield kafka_producer_factory() + + +@pytest.fixture +def kafka_producer_factory(kafka_broker, request): + """Return a KafkaProduce factory fixture""" + _producer = [None] + + def factory(**kafka_producer_params): + params = {} if kafka_producer_params is None else kafka_producer_params.copy() + params.setdefault('client_id', 'producer_%s' % (request.node.name,)) + _producer[0] = next(kafka_broker.get_producers(cnt=1, **params)) + return _producer[0] + + yield factory + + if _producer[0]: + _producer[0].close() + + +@pytest.fixture +def kafka_admin_client(kafka_admin_client_factory): + """Return a KafkaAdminClient fixture""" + yield kafka_admin_client_factory() + + +@pytest.fixture +def kafka_admin_client_factory(kafka_broker): + """Return a KafkaAdminClient factory fixture""" + _admin_client = [None] + + def factory(**kafka_admin_client_params): + params = {} if kafka_admin_client_params is None else kafka_admin_client_params.copy() + _admin_client[0] = next(kafka_broker.get_admin_clients(cnt=1, **params)) + return _admin_client[0] + + yield factory + + if _admin_client[0]: + _admin_client[0].close() + + +@pytest.fixture +def topic(kafka_broker, request): + """Return a topic fixture""" + topic_name = '%s_%s' % (request.node.name, random_string(10)) + kafka_broker.create_topics([topic_name]) + return topic_name + + +@pytest.fixture() +def send_messages(topic, kafka_producer, request): + """A factory that returns a send_messages function with a pre-populated + topic topic / producer.""" + + def _send_messages(number_range, partition=0, topic=topic, producer=kafka_producer, request=request): + """ + messages is typically `range(0,100)` + partition is an int + """ + messages_and_futures = [] # [(message, produce_future),] + for i in number_range: + # request.node.name provides the test name (including parametrized values) + encoded_msg = '{}-{}-{}'.format(i, request.node.name, uuid.uuid4()).encode('utf-8') + future = kafka_producer.send(topic, value=encoded_msg, partition=partition) + messages_and_futures.append((encoded_msg, future)) + kafka_producer.flush() + for (msg, f) in messages_and_futures: + assert f.succeeded() + return [msg for (msg, f) in messages_and_futures] + + return _send_messages diff --git a/test/fixtures.py b/test/integration/fixtures.py similarity index 99% rename from test/fixtures.py rename to test/integration/fixtures.py index 3adb87a97..b9baf5223 100644 --- a/test/fixtures.py +++ b/test/integration/fixtures.py @@ -66,7 +66,7 @@ class Fixture(object): kafka_version = os.environ.get('KAFKA_VERSION', '0.11.0.2') scala_version = os.environ.get("SCALA_VERSION", '2.8.0') project_root = os.environ.get('PROJECT_ROOT', - os.path.abspath(os.path.join(os.path.dirname(__file__), ".."))) + os.path.abspath(os.path.join(os.path.dirname(__file__), "../.."))) kafka_root = os.environ.get("KAFKA_ROOT", os.path.join(project_root, 'servers', kafka_version, "kafka-bin")) diff --git a/test/test_admin_integration.py b/test/integration/test_admin_integration.py similarity index 100% rename from test/test_admin_integration.py rename to test/integration/test_admin_integration.py diff --git a/test/test_consumer_group.py b/test/integration/test_consumer_group.py similarity index 100% rename from test/test_consumer_group.py rename to test/integration/test_consumer_group.py diff --git a/test/test_consumer_integration.py b/test/integration/test_consumer_integration.py similarity index 100% rename from test/test_consumer_integration.py rename to test/integration/test_consumer_integration.py diff --git a/test/test_producer.py b/test/integration/test_producer_integration.py similarity index 100% rename from test/test_producer.py rename to test/integration/test_producer_integration.py diff --git a/test/test_sasl_integration.py b/test/integration/test_sasl_integration.py similarity index 100% rename from test/test_sasl_integration.py rename to test/integration/test_sasl_integration.py diff --git a/test/test_client_async.py b/test/test_client_async.py index 276926116..acc400f9c 100644 --- a/test/test_client_async.py +++ b/test/test_client_async.py @@ -23,12 +23,29 @@ @pytest.fixture -def cli(mocker, conn): +def client_poll_mocked(mocker): + cli = KafkaClient(request_timeout_ms=9999999, + reconnect_backoff_ms=2222, + connections_max_idle_ms=float('inf'), + api_version=(0, 9)) + mocker.patch.object(cli, '_poll') + ttl = mocker.patch.object(cli.cluster, 'ttl') + ttl.return_value = 0 + try: + yield cli + finally: + cli._close() + + +@pytest.fixture +def client_selector_mocked(mocker, conn): client = KafkaClient(api_version=(0, 9)) mocker.patch.object(client, '_selector') client.poll(future=client.cluster.request_update()) - return client - + try: + yield client + finally: + client._close() def test_bootstrap(mocker, conn): conn.state = ConnectionStates.CONNECTED @@ -49,185 +66,181 @@ def test_bootstrap(mocker, conn): BrokerMetadata(1, 'bar', 34, None)]) -def test_can_connect(cli, conn): +def test_can_connect(client_selector_mocked, conn): # Node is not in broker metadata - can't connect - assert not cli._can_connect(2) + assert not client_selector_mocked._can_connect(2) # Node is in broker metadata but not in _conns - assert 0 not in cli._conns - assert cli._can_connect(0) + assert 0 not in client_selector_mocked._conns + assert client_selector_mocked._can_connect(0) # Node is connected, can't reconnect - assert cli._init_connect(0) is True - assert not cli._can_connect(0) + assert client_selector_mocked._init_connect(0) is True + assert not client_selector_mocked._can_connect(0) # Node is disconnected, can connect - cli._conns[0].state = ConnectionStates.DISCONNECTED - assert cli._can_connect(0) + client_selector_mocked._conns[0].state = ConnectionStates.DISCONNECTED + assert client_selector_mocked._can_connect(0) # Node is disconnected, but blacked out conn.blacked_out.return_value = True - assert not cli._can_connect(0) + assert not client_selector_mocked._can_connect(0) -def test_init_connect(cli, conn): +def test_init_connect(client_selector_mocked, conn): # Node not in metadata, return False - assert not cli._init_connect(2) + assert not client_selector_mocked._init_connect(2) # New node_id creates a conn object - assert 0 not in cli._conns + assert 0 not in client_selector_mocked._conns conn.state = ConnectionStates.DISCONNECTED conn.connect.side_effect = lambda: conn._set_conn_state(ConnectionStates.CONNECTING) - assert cli._init_connect(0) is True - assert cli._conns[0] is conn + assert client_selector_mocked._init_connect(0) is True + assert client_selector_mocked._conns[0] is conn -def test_conn_state_change(mocker, cli, conn): - sel = cli._selector +def test_conn_state_change(client_selector_mocked, conn): + sel = client_selector_mocked._selector node_id = 0 - cli._conns[node_id] = conn + client_selector_mocked._conns[node_id] = conn conn.state = ConnectionStates.CONNECTING sock = conn._sock - cli._conn_state_change(node_id, sock, conn) - assert node_id in cli._connecting + client_selector_mocked._conn_state_change(node_id, sock, conn) + assert node_id in client_selector_mocked._connecting sel.register.assert_called_with(sock, selectors.EVENT_WRITE, conn) conn.state = ConnectionStates.CONNECTED - cli._conn_state_change(node_id, sock, conn) - assert node_id not in cli._connecting + client_selector_mocked._conn_state_change(node_id, sock, conn) + assert node_id not in client_selector_mocked._connecting sel.modify.assert_called_with(sock, selectors.EVENT_READ, conn) # Failure to connect should trigger metadata update - assert cli.cluster._need_update is False + assert client_selector_mocked.cluster._need_update is False conn.state = ConnectionStates.DISCONNECTED - cli._conn_state_change(node_id, sock, conn) - assert node_id not in cli._connecting - assert cli.cluster._need_update is True + client_selector_mocked._conn_state_change(node_id, sock, conn) + assert node_id not in client_selector_mocked._connecting + assert client_selector_mocked.cluster._need_update is True sel.unregister.assert_called_with(sock) conn.state = ConnectionStates.CONNECTING - cli._conn_state_change(node_id, sock, conn) - assert node_id in cli._connecting + client_selector_mocked._conn_state_change(node_id, sock, conn) + assert node_id in client_selector_mocked._connecting conn.state = ConnectionStates.DISCONNECTED - cli._conn_state_change(node_id, sock, conn) - assert node_id not in cli._connecting + client_selector_mocked._conn_state_change(node_id, sock, conn) + assert node_id not in client_selector_mocked._connecting -def test_ready(mocker, cli, conn): - maybe_connect = mocker.patch.object(cli, 'maybe_connect') +def test_ready(mocker, client_selector_mocked, conn): + maybe_connect = mocker.patch.object(client_selector_mocked, 'maybe_connect') node_id = 1 - cli.ready(node_id) + client_selector_mocked.ready(node_id) maybe_connect.assert_called_with(node_id) -def test_is_ready(mocker, cli, conn): - cli._init_connect(0) - cli._init_connect(1) +def test_is_ready(client_selector_mocked, conn): + client_selector_mocked._init_connect(0) + client_selector_mocked._init_connect(1) # metadata refresh blocks ready nodes - assert cli.is_ready(0) - assert cli.is_ready(1) - cli._metadata_refresh_in_progress = True - assert not cli.is_ready(0) - assert not cli.is_ready(1) + assert client_selector_mocked.is_ready(0) + assert client_selector_mocked.is_ready(1) + client_selector_mocked._metadata_refresh_in_progress = True + assert not client_selector_mocked.is_ready(0) + assert not client_selector_mocked.is_ready(1) # requesting metadata update also blocks ready nodes - cli._metadata_refresh_in_progress = False - assert cli.is_ready(0) - assert cli.is_ready(1) - cli.cluster.request_update() - cli.cluster.config['retry_backoff_ms'] = 0 - assert not cli._metadata_refresh_in_progress - assert not cli.is_ready(0) - assert not cli.is_ready(1) - cli.cluster._need_update = False + client_selector_mocked._metadata_refresh_in_progress = False + assert client_selector_mocked.is_ready(0) + assert client_selector_mocked.is_ready(1) + client_selector_mocked.cluster.request_update() + client_selector_mocked.cluster.config['retry_backoff_ms'] = 0 + assert not client_selector_mocked._metadata_refresh_in_progress + assert not client_selector_mocked.is_ready(0) + assert not client_selector_mocked.is_ready(1) + client_selector_mocked.cluster._need_update = False # if connection can't send more, not ready - assert cli.is_ready(0) + assert client_selector_mocked.is_ready(0) conn.can_send_more.return_value = False - assert not cli.is_ready(0) + assert not client_selector_mocked.is_ready(0) conn.can_send_more.return_value = True # disconnected nodes, not ready - assert cli.is_ready(0) + assert client_selector_mocked.is_ready(0) conn.state = ConnectionStates.DISCONNECTED - assert not cli.is_ready(0) + assert not client_selector_mocked.is_ready(0) -def test_close(mocker, cli, conn): - mocker.patch.object(cli, '_selector') - +def test_close(client_selector_mocked, conn): call_count = conn.close.call_count # Unknown node - silent - cli.close(2) + client_selector_mocked.close(2) call_count += 0 assert conn.close.call_count == call_count # Single node close - cli._init_connect(0) + client_selector_mocked._init_connect(0) assert conn.close.call_count == call_count - cli.close(0) + client_selector_mocked.close(0) call_count += 1 assert conn.close.call_count == call_count # All node close - cli._init_connect(1) - cli.close() + client_selector_mocked._init_connect(1) + client_selector_mocked.close() # +2 close: node 1, node bootstrap (node 0 already closed) call_count += 2 assert conn.close.call_count == call_count -def test_is_disconnected(cli, conn): +def test_is_disconnected(client_selector_mocked, conn): # False if not connected yet conn.state = ConnectionStates.DISCONNECTED - assert not cli.is_disconnected(0) + assert not client_selector_mocked.is_disconnected(0) - cli._init_connect(0) - assert cli.is_disconnected(0) + client_selector_mocked._init_connect(0) + assert client_selector_mocked.is_disconnected(0) conn.state = ConnectionStates.CONNECTING - assert not cli.is_disconnected(0) + assert not client_selector_mocked.is_disconnected(0) conn.state = ConnectionStates.CONNECTED - assert not cli.is_disconnected(0) + assert not client_selector_mocked.is_disconnected(0) -def test_send(cli, conn): +def test_send(client_selector_mocked, conn): # Send to unknown node => raises AssertionError try: - cli.send(2, None) + client_selector_mocked.send(2, None) assert False, 'Exception not raised' except AssertionError: pass # Send to disconnected node => NodeNotReady conn.state = ConnectionStates.DISCONNECTED - f = cli.send(0, None) + f = client_selector_mocked.send(0, None) assert f.failed() assert isinstance(f.exception, Errors.NodeNotReadyError) conn.state = ConnectionStates.CONNECTED - cli._init_connect(0) + client_selector_mocked._init_connect(0) # ProduceRequest w/ 0 required_acks -> no response request = ProduceRequest[0](0, 0, []) assert request.expect_response() is False - ret = cli.send(0, request) + ret = client_selector_mocked.send(0, request) conn.send.assert_called_with(request, blocking=False, request_timeout_ms=None) assert isinstance(ret, Future) request = MetadataRequest[0]([]) - cli.send(0, request) + client_selector_mocked.send(0, request) conn.send.assert_called_with(request, blocking=False, request_timeout_ms=None) -def test_poll(mocker): - metadata = mocker.patch.object(KafkaClient, '_maybe_refresh_metadata') - ifr_request_timeout = mocker.patch.object(KafkaClient, '_next_ifr_request_timeout_ms') - _poll = mocker.patch.object(KafkaClient, '_poll') - cli = KafkaClient(api_version=(0, 9)) +def test_poll(mocker, client_poll_mocked): + metadata = mocker.patch.object(client_poll_mocked, '_maybe_refresh_metadata') + ifr_request_timeout = mocker.patch.object(client_poll_mocked, '_next_ifr_request_timeout_ms') now = time.time() t = mocker.patch('time.time') t.return_value = now @@ -235,18 +248,18 @@ def test_poll(mocker): # metadata timeout wins ifr_request_timeout.return_value = float('inf') metadata.return_value = 1000 - cli.poll() - _poll.assert_called_with(1.0) + client_poll_mocked.poll() + client_poll_mocked._poll.assert_called_with(1.0) # user timeout wins - cli.poll(timeout_ms=250) - _poll.assert_called_with(0.25) + client_poll_mocked.poll(timeout_ms=250) + client_poll_mocked._poll.assert_called_with(0.25) # ifr request timeout wins ifr_request_timeout.return_value = 30000 metadata.return_value = 1000000 - cli.poll() - _poll.assert_called_with(30.0) + client_poll_mocked.poll() + client_poll_mocked._poll.assert_called_with(30.0) def test__poll(): @@ -287,80 +300,66 @@ def test_set_topics(mocker): request_update.assert_not_called() -@pytest.fixture -def client(mocker): - _poll = mocker.patch.object(KafkaClient, '_poll') - - cli = KafkaClient(request_timeout_ms=9999999, - reconnect_backoff_ms=2222, - connections_max_idle_ms=float('inf'), - api_version=(0, 9)) - - ttl = mocker.patch.object(cli.cluster, 'ttl') - ttl.return_value = 0 - return cli - - -def test_maybe_refresh_metadata_ttl(mocker, client): - client.cluster.ttl.return_value = 1234 +def test_maybe_refresh_metadata_ttl(client_poll_mocked): + client_poll_mocked.cluster.ttl.return_value = 1234 - client.poll(timeout_ms=12345678) - client._poll.assert_called_with(1.234) + client_poll_mocked.poll(timeout_ms=12345678) + client_poll_mocked._poll.assert_called_with(1.234) -def test_maybe_refresh_metadata_backoff(mocker, client): - mocker.patch.object(client, 'least_loaded_node', return_value=None) - mocker.patch.object(client, 'least_loaded_node_refresh_ms', return_value=4321) +def test_maybe_refresh_metadata_backoff(mocker, client_poll_mocked): + mocker.patch.object(client_poll_mocked, 'least_loaded_node', return_value=None) + mocker.patch.object(client_poll_mocked, 'least_loaded_node_refresh_ms', return_value=4321) now = time.time() t = mocker.patch('time.time') t.return_value = now - client.poll(timeout_ms=12345678) - client._poll.assert_called_with(4.321) + client_poll_mocked.poll(timeout_ms=12345678) + client_poll_mocked._poll.assert_called_with(4.321) -def test_maybe_refresh_metadata_in_progress(mocker, client): - client._metadata_refresh_in_progress = True +def test_maybe_refresh_metadata_in_progress(client_poll_mocked): + client_poll_mocked._metadata_refresh_in_progress = True - client.poll(timeout_ms=12345678) - client._poll.assert_called_with(9999.999) # request_timeout_ms + client_poll_mocked.poll(timeout_ms=12345678) + client_poll_mocked._poll.assert_called_with(9999.999) # request_timeout_ms -def test_maybe_refresh_metadata_update(mocker, client): - mocker.patch.object(client, 'least_loaded_node', return_value='foobar') - mocker.patch.object(client, '_can_send_request', return_value=True) - send = mocker.patch.object(client, 'send') - client.cluster.need_all_topic_metadata = True +def test_maybe_refresh_metadata_update(mocker, client_poll_mocked): + mocker.patch.object(client_poll_mocked, 'least_loaded_node', return_value='foobar') + mocker.patch.object(client_poll_mocked, '_can_send_request', return_value=True) + send = mocker.patch.object(client_poll_mocked, 'send') + client_poll_mocked.cluster.need_all_topic_metadata = True - client.poll(timeout_ms=12345678) - client._poll.assert_called_with(9999.999) # request_timeout_ms - assert client._metadata_refresh_in_progress + client_poll_mocked.poll(timeout_ms=12345678) + client_poll_mocked._poll.assert_called_with(9999.999) # request_timeout_ms + assert client_poll_mocked._metadata_refresh_in_progress request = MetadataRequest[0]([]) send.assert_called_once_with('foobar', request, wakeup=False) -def test_maybe_refresh_metadata_cant_send(mocker, client): - mocker.patch.object(client, 'least_loaded_node', return_value='foobar') - mocker.patch.object(client, '_can_send_request', return_value=False) - mocker.patch.object(client, '_can_connect', return_value=True) - mocker.patch.object(client, '_init_connect', return_value=True) +def test_maybe_refresh_metadata_cant_send(mocker, client_poll_mocked): + mocker.patch.object(client_poll_mocked, 'least_loaded_node', return_value='foobar') + mocker.patch.object(client_poll_mocked, '_can_send_request', return_value=False) + mocker.patch.object(client_poll_mocked, '_can_connect', return_value=True) + mocker.patch.object(client_poll_mocked, '_init_connect', return_value=True) now = time.time() t = mocker.patch('time.time') t.return_value = now # first poll attempts connection - client.poll() - client._poll.assert_called() - client._init_connect.assert_called_once_with('foobar') + client_poll_mocked.poll() + client_poll_mocked._poll.assert_called() + client_poll_mocked._init_connect.assert_called_once_with('foobar') # poll while connecting should not attempt a new connection - client._connecting.add('foobar') - client._can_connect.reset_mock() - client.poll() - client._poll.assert_called() - assert not client._can_connect.called - assert not client._metadata_refresh_in_progress + client_poll_mocked._connecting.add('foobar') + client_poll_mocked._can_connect.reset_mock() + client_poll_mocked.poll() + client_poll_mocked._poll.assert_called() + assert not client_poll_mocked._can_connect.called + assert not client_poll_mocked._metadata_refresh_in_progress def test_schedule(): diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 00a929399..8c114c90f 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -24,14 +24,6 @@ from kafka.structs import OffsetAndMetadata, TopicPartition from kafka.util import WeakMethod -@pytest.fixture -def client(conn, mocker): - cli = KafkaClient(api_version=(0, 9)) - mocker.patch.object(cli, '_init_connect', return_value=True) - try: - yield cli - finally: - cli._close() @pytest.fixture def coordinator(client, metrics, mocker): diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 3fc0c55ae..80bd0e42d 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -26,11 +26,6 @@ from kafka.structs import OffsetAndMetadata, OffsetAndTimestamp, TopicPartition -@pytest.fixture -def client(): - return KafkaClient(bootstrap_servers=(), api_version=(0, 9)) - - @pytest.fixture def subscription_state(): return SubscriptionState() diff --git a/test/test_metrics.py b/test/test_metrics.py index 308ea5831..07c0e838a 100644 --- a/test/test_metrics.py +++ b/test/test_metrics.py @@ -19,23 +19,6 @@ def time_keeper(): return TimeKeeper() -@pytest.fixture -def config(): - return MetricConfig() - - -@pytest.fixture -def reporter(): - return DictReporter() - - -@pytest.fixture -def metrics(request, config, reporter): - metrics = Metrics(config, [reporter], enable_expiration=True) - yield metrics - metrics.close() - - def test_MetricName(): # The Java test only cover the differences between the deprecated # constructors, so I'm skipping them but doing some other basic testing. @@ -82,8 +65,9 @@ def test_MetricName(): assert name.tags == tags -def test_simple_stats(mocker, time_keeper, config, metrics): +def test_simple_stats(mocker, time_keeper, metrics): mocker.patch('time.time', side_effect=time_keeper.time) + config = metrics._config measurable = ConstantMeasurable() From 4122c1f024bda6e35fd3dc54b574e9df42be1c54 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 15 Apr 2025 16:46:49 -0700 Subject: [PATCH 124/205] KIP-98: Transactional Producer (#2587) --- kafka/conn.py | 3 +- kafka/producer/kafka.py | 116 ++- kafka/producer/record_accumulator.py | 65 +- kafka/producer/sender.py | 243 ++++-- kafka/producer/transaction_manager.py | 812 ++++++++++++++++++ kafka/producer/transaction_state.py | 96 --- kafka/record/default_records.py | 36 +- kafka/record/memory_records.py | 29 +- test/integration/test_producer_integration.py | 60 +- test/test_producer.py | 23 + test/test_record_accumulator.py | 2 +- test/test_sender.py | 50 +- 12 files changed, 1318 insertions(+), 217 deletions(-) create mode 100644 kafka/producer/transaction_manager.py delete mode 100644 kafka/producer/transaction_state.py create mode 100644 test/test_producer.py diff --git a/kafka/conn.py b/kafka/conn.py index 85a9658d4..31e1f8be9 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -934,7 +934,8 @@ def close(self, error=None): if self.state is ConnectionStates.DISCONNECTED: return log.log(logging.ERROR if error else logging.INFO, '%s: Closing connection. %s', self, error or '') - self._update_reconnect_backoff() + if error: + self._update_reconnect_backoff() self._api_versions_future = None self._sasl_auth_future = None self._init_sasl_mechanism() diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 320a1657f..1468cec55 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -19,7 +19,7 @@ from kafka.producer.future import FutureRecordMetadata, FutureProduceResult from kafka.producer.record_accumulator import AtomicInteger, RecordAccumulator from kafka.producer.sender import Sender -from kafka.producer.transaction_state import TransactionState +from kafka.producer.transaction_manager import TransactionManager from kafka.record.default_records import DefaultRecordBatchBuilder from kafka.record.legacy_records import LegacyRecordBatchBuilder from kafka.serializer import Serializer @@ -318,6 +318,8 @@ class KafkaProducer(object): 'key_serializer': None, 'value_serializer': None, 'enable_idempotence': False, + 'transactional_id': None, + 'transaction_timeout_ms': 60000, 'acks': 1, 'bootstrap_topics_filter': set(), 'compression_type': None, @@ -444,9 +446,30 @@ def __init__(self, **configs): assert checker(), "Libraries for {} compression codec not found".format(ct) self.config['compression_attrs'] = compression_attrs - self._transaction_state = None + self._metadata = client.cluster + self._transaction_manager = None + self._init_transactions_result = None + if 'enable_idempotence' in user_provided_configs and not self.config['enable_idempotence'] and self.config['transactional_id']: + raise Errors.KafkaConfigurationError("Cannot set transactional_id without enable_idempotence.") + + if self.config['transactional_id']: + self.config['enable_idempotence'] = True + if self.config['enable_idempotence']: - self._transaction_state = TransactionState() + assert self.config['api_version'] >= (0, 11), "Transactional/Idempotent producer requires >= Kafka 0.11 Brokers" + + self._transaction_manager = TransactionManager( + transactional_id=self.config['transactional_id'], + transaction_timeout_ms=self.config['transaction_timeout_ms'], + retry_backoff_ms=self.config['retry_backoff_ms'], + api_version=self.config['api_version'], + metadata=self._metadata, + ) + if self._transaction_manager.is_transactional(): + log.info("Instantiated a transactional producer.") + else: + log.info("Instantiated an idempotent producer.") + if 'retries' not in user_provided_configs: log.info("Overriding the default 'retries' config to 3 since the idempotent producer is enabled.") self.config['retries'] = 3 @@ -470,15 +493,14 @@ def __init__(self, **configs): message_version = self.max_usable_produce_magic(self.config['api_version']) self._accumulator = RecordAccumulator( - transaction_state=self._transaction_state, + transaction_manager=self._transaction_manager, message_version=message_version, **self.config) - self._metadata = client.cluster guarantee_message_order = bool(self.config['max_in_flight_requests_per_connection'] == 1) self._sender = Sender(client, self._metadata, self._accumulator, metrics=self._metrics, - transaction_state=self._transaction_state, + transaction_manager=self._transaction_manager, guarantee_message_order=guarantee_message_order, **self.config) self._sender.daemon = True @@ -610,6 +632,84 @@ def _estimate_size_in_bytes(self, key, value, headers=[]): return LegacyRecordBatchBuilder.estimate_size_in_bytes( magic, self.config['compression_type'], key, value) + def init_transactions(self): + """ + Needs to be called before any other methods when the transactional.id is set in the configuration. + + This method does the following: + 1. Ensures any transactions initiated by previous instances of the producer with the same + transactional_id are completed. If the previous instance had failed with a transaction in + progress, it will be aborted. If the last transaction had begun completion, + but not yet finished, this method awaits its completion. + 2. Gets the internal producer id and epoch, used in all future transactional + messages issued by the producer. + + Note that this method will raise KafkaTimeoutError if the transactional state cannot + be initialized before expiration of `max_block_ms`. + + Retrying after a KafkaTimeoutError will continue to wait for the prior request to succeed or fail. + Retrying after any other exception will start a new initialization attempt. + Retrying after a successful initialization will do nothing. + + Raises: + IllegalStateError: if no transactional_id has been configured + AuthorizationError: fatal error indicating that the configured + transactional_id is not authorized. + KafkaError: if the producer has encountered a previous fatal error or for any other unexpected error + KafkaTimeoutError: if the time taken for initialize the transaction has surpassed `max.block.ms`. + """ + if not self._transaction_manager: + raise Errors.IllegalStateError("Cannot call init_transactions without setting a transactional_id.") + if self._init_transactions_result is None: + self._init_transactions_result = self._transaction_manager.initialize_transactions() + self._sender.wakeup() + + try: + if not self._init_transactions_result.wait(timeout_ms=self.config['max_block_ms']): + raise Errors.KafkaTimeoutError("Timeout expired while initializing transactional state in %s ms." % (self.config['max_block_ms'],)) + finally: + if self._init_transactions_result.failed: + self._init_transactions_result = None + + def begin_transaction(self): + """ Should be called before the start of each new transaction. + + Note that prior to the first invocation of this method, + you must invoke `init_transactions()` exactly one time. + + Raises: + ProducerFencedError if another producer is with the same + transactional_id is active. + """ + # Set the transactional bit in the producer. + if not self._transaction_manager: + raise Errors.IllegalStateError("Cannot use transactional methods without enabling transactions") + self._transaction_manager.begin_transaction() + + def commit_transaction(self): + """ Commits the ongoing transaction. + + Raises: ProducerFencedError if another producer with the same + transactional_id is active. + """ + if not self._transaction_manager: + raise Errors.IllegalStateError("Cannot commit transaction since transactions are not enabled") + result = self._transaction_manager.begin_commit() + self._sender.wakeup() + result.wait() + + def abort_transaction(self): + """ Aborts the ongoing transaction. + + Raises: ProducerFencedError if another producer with the same + transactional_id is active. + """ + if not self._transaction_manager: + raise Errors.IllegalStateError("Cannot abort transaction since transactions are not enabled.") + result = self._transaction_manager.begin_abort() + self._sender.wakeup() + result.wait() + def send(self, topic, value=None, key=None, headers=None, partition=None, timestamp_ms=None): """Publish a message to a topic. @@ -687,6 +787,10 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest tp = TopicPartition(topic, partition) log.debug("Sending (key=%r value=%r headers=%r) to %s", key, value, headers, tp) + + if self._transaction_manager and self._transaction_manager.is_transactional(): + self._transaction_manager.maybe_add_partition_to_transaction(tp) + result = self._accumulator.append(tp, timestamp_ms, key_bytes, value_bytes, headers) future, batch_is_full, new_batch_created = result diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 6490f48aa..83802ef96 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -56,6 +56,14 @@ def record_count(self): 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: @@ -170,7 +178,7 @@ class RecordAccumulator(object): 'compression_attrs': 0, 'linger_ms': 0, 'retry_backoff_ms': 100, - 'transaction_state': None, + 'transaction_manager': None, 'message_version': 0, } @@ -181,7 +189,7 @@ def __init__(self, **configs): self.config[key] = configs.pop(key) self._closed = False - self._transaction_state = self.config['transaction_state'] + self._transaction_manager = self.config['transaction_manager'] self._flushes_in_progress = AtomicInteger() self._appends_in_progress = AtomicInteger() self._batches = collections.defaultdict(collections.deque) # TopicPartition: [ProducerBatch] @@ -244,7 +252,7 @@ def append(self, tp, timestamp_ms, key, value, headers): batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False - if self._transaction_state and self.config['message_version'] < 2: + if self._transaction_manager and self.config['message_version'] < 2: raise Errors.UnsupportedVersionError("Attempting to use idempotence with a broker which" " does not support the required message format (v2)." " The broker must be version 0.11 or later.") @@ -418,8 +426,8 @@ def ready(self, cluster, now=None): return ready_nodes, next_ready_check, unknown_leaders_exist - def has_unsent(self): - """Return whether there is any unsent record in the accumulator.""" + def has_undrained(self): + """Check whether there are any batches which haven't been drained""" for tp in list(self._batches.keys()): with self._tp_locks[tp]: dq = self._batches[tp] @@ -479,8 +487,10 @@ def drain(self, cluster, nodes, max_size, now=None): break else: producer_id_and_epoch = None - if self._transaction_state: - producer_id_and_epoch = self._transaction_state.producer_id_and_epoch + if self._transaction_manager: + if not self._transaction_manager.is_send_to_partition_allowed(tp): + break + producer_id_and_epoch = self._transaction_manager.producer_id_and_epoch if not producer_id_and_epoch.is_valid: # we cannot send the batch until we have refreshed the PID log.debug("Waiting to send ready batches because transaction producer id is not valid") @@ -493,11 +503,16 @@ def drain(self, cluster, nodes, max_size, now=None): # the previous attempt may actually have been accepted, and if we change # the pid and sequence here, this attempt will also be accepted, causing # a duplicate. - sequence_number = self._transaction_state.sequence_number(batch.topic_partition) + sequence_number = self._transaction_manager.sequence_number(batch.topic_partition) log.debug("Dest: %s: %s producer_id=%s epoch=%s sequence=%s", node_id, batch.topic_partition, producer_id_and_epoch.producer_id, producer_id_and_epoch.epoch, sequence_number) - batch.records.set_producer_state(producer_id_and_epoch.producer_id, producer_id_and_epoch.epoch, sequence_number) + batch.records.set_producer_state( + producer_id_and_epoch.producer_id, + producer_id_and_epoch.epoch, + sequence_number, + self._transaction_manager.is_transactional() + ) batch.records.close() size += batch.records.size_in_bytes() ready.append(batch) @@ -544,6 +559,10 @@ def await_flush_completion(self, timeout=None): finally: self._flushes_in_progress.decrement() + @property + def has_incomplete(self): + return bool(self._incomplete) + def abort_incomplete_batches(self): """ This function is only called when sender is closed forcefully. It will fail all the @@ -553,27 +572,41 @@ def abort_incomplete_batches(self): # 1. Avoid losing batches. # 2. Free up memory in case appending threads are blocked on buffer full. # This is a tight loop but should be able to get through very quickly. + error = Errors.IllegalStateError("Producer is closed forcefully.") while True: - self._abort_batches() + self._abort_batches(error) if not self._appends_in_progress.get(): break # After this point, no thread will append any messages because they will see the close # flag set. We need to do the last abort after no thread was appending in case the there was a new # batch appended by the last appending thread. - self._abort_batches() + self._abort_batches(error) self._batches.clear() - def _abort_batches(self): + def _abort_batches(self, error): """Go through incomplete batches and abort them.""" - error = Errors.IllegalStateError("Producer is closed forcefully.") for batch in self._incomplete.all(): tp = batch.topic_partition # Close the batch before aborting with self._tp_locks[tp]: batch.records.close() + self._batches[tp].remove(batch) batch.done(exception=error) self.deallocate(batch) + def abort_undrained_batches(self, error): + for batch in self._incomplete.all(): + tp = batch.topic_partition + with self._tp_locks[tp]: + aborted = False + if not batch.is_done: + aborted = True + batch.records.close() + self._batches[tp].remove(batch) + if aborted: + batch.done(exception=error) + self.deallocate(batch) + def close(self): """Close this accumulator and force all the record buffers to be drained.""" self._closed = True @@ -600,3 +633,9 @@ def remove(self, batch): def all(self): with self._lock: return list(self._incomplete) + + def __bool__(self): + return bool(self._incomplete) + + + __nonzero__ = __bool__ diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 96a50cbbc..707d46bf3 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -11,6 +11,7 @@ from kafka import errors as Errors from kafka.metrics.measurable import AnonMeasurable from kafka.metrics.stats import Avg, Max, Rate +from kafka.producer.transaction_manager import ProducerIdAndEpoch from kafka.protocol.init_producer_id import InitProducerIdRequest from kafka.protocol.produce import ProduceRequest from kafka.structs import TopicPartition @@ -33,7 +34,7 @@ class Sender(threading.Thread): 'retry_backoff_ms': 100, 'metrics': None, 'guarantee_message_order': False, - 'transaction_state': None, + 'transaction_manager': None, 'transactional_id': None, 'transaction_timeout_ms': 60000, 'client_id': 'kafka-python-' + __version__, @@ -57,7 +58,7 @@ def __init__(self, client, metadata, accumulator, **configs): self._sensors = SenderMetrics(self.config['metrics'], self._client, self._metadata) else: self._sensors = None - self._transaction_state = self.config['transaction_state'] + self._transaction_manager = self.config['transaction_manager'] def run(self): """The main run loop for the sender thread.""" @@ -77,7 +78,7 @@ def run(self): # requests in the accumulator or waiting for acknowledgment, # wait until these are completed. while (not self._force_close - and (self._accumulator.has_unsent() + and (self._accumulator.has_undrained() or self._client.in_flight_request_count() > 0)): try: self.run_once() @@ -101,8 +102,36 @@ def run_once(self): while self._topics_to_add: self._client.add_topic(self._topics_to_add.pop()) - self._maybe_wait_for_producer_id() + if self._transaction_manager: + try: + if not self._transaction_manager.is_transactional(): + # this is an idempotent producer, so make sure we have a producer id + self._maybe_wait_for_producer_id() + elif self._transaction_manager.has_in_flight_transactional_request() or self._maybe_send_transactional_request(): + # as long as there are outstanding transactional requests, we simply wait for them to return + self._client.poll(timeout_ms=self.config['retry_backoff_ms']) + return + + # do not continue sending if the transaction manager is in a failed state or if there + # is no producer id (for the idempotent case). + if self._transaction_manager.has_fatal_error() or not self._transaction_manager.has_producer_id(): + last_error = self._transaction_manager.last_error + if last_error is not None: + self._maybe_abort_batches(last_error) + 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) + + except Errors.SaslAuthenticationFailedError as e: + # This is already logged as error, but propagated here to perform any clean ups. + log.debug("Authentication exception while processing transactional request: %s", e) + self._transaction_manager.authentication_failed(e) + + poll_timeout_ms = self._send_producer_data() + self._client.poll(timeout_ms=poll_timeout_ms) + def _send_producer_data(self): # get the list of partitions with data ready to send result = self._accumulator.ready(self._metadata) ready_nodes, next_ready_check_delay, unknown_leaders_exist = result @@ -136,18 +165,31 @@ def run_once(self): expired_batches = self._accumulator.abort_expired_batches( self.config['request_timeout_ms'], self._metadata) + if expired_batches: + log.debug("Expired %s batches in accumulator", len(expired_batches)) + # Reset the producer_id if an expired batch has previously been sent to the broker. # See the documentation of `TransactionState.reset_producer_id` to understand why # we need to reset the producer id here. - if self._transaction_state and any([batch.in_retry() for batch in expired_batches]): - self._transaction_state.reset_producer_id() - return + if self._transaction_manager and any([batch.in_retry() for batch in expired_batches]): + needs_transaction_state_reset = True + else: + 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) if self._sensors: - for expired_batch in expired_batches: - self._sensors.record_errors(expired_batch.topic_partition.topic, expired_batch.record_count) self._sensors.update_produce_request_metrics(batches_by_node) + if needs_transaction_state_reset: + self._transaction_manager.reset_producer_id() + return 0 + requests = self._create_produce_requests(batches_by_node) # If we have any nodes that are ready to send + have sendable data, # poll with 0 timeout so this can immediately loop and try sending more @@ -160,6 +202,12 @@ def run_once(self): if ready_nodes: log.debug("Nodes with data ready to send: %s", ready_nodes) # trace log.debug("Created %d produce requests: %s", len(requests), requests) # trace + # if some partitions are already ready to be sent, the select time + # would be 0; otherwise if some partition already has some data + # accumulated but not ready yet, the select time will be the time + # difference between now and its linger expiry time; otherwise the + # select time will be the time difference between now and the + # metadata expiry time poll_timeout_ms = 0 for node_id, request in six.iteritems(requests): @@ -170,14 +218,67 @@ def run_once(self): self._handle_produce_response, node_id, time.time(), batches) .add_errback( self._failed_produce, batches, node_id)) + return poll_timeout_ms + + def _maybe_send_transactional_request(self): + if self._transaction_manager.is_completing() and self._accumulator.has_incomplete: + if self._transaction_manager.is_aborting(): + self._accumulator.abort_undrained_batches(Errors.KafkaError("Failing batch since transaction was aborted")) + # There may still be requests left which are being retried. Since we do not know whether they had + # been successfully appended to the broker log, we must resend them until their final status is clear. + # If they had been appended and we did not receive the error, then our sequence number would no longer + # be correct which would lead to an OutOfSequenceNumberError. + if not self._accumulator.flush_in_progress(): + self._accumulator.begin_flush() + + next_request_handler = self._transaction_manager.next_request_handler(self._accumulator.has_incomplete) + if next_request_handler is None: + return False + + log.debug("transactional_id: %s -- Sending transactional request %s", self._transaction_manager.transactional_id, next_request_handler.request) + while not self._force_close: + target_node = None + try: + if next_request_handler.needs_coordinator(): + target_node = self._transaction_manager.coordinator(next_request_handler.coordinator_type) + if target_node is None: + self._transaction_manager.lookup_coordinator_for_request(next_request_handler) + break + elif not self._client.await_ready(target_node, timeout_ms=self.config['request_timeout_ms']): + self._transaction_manager.lookup_coordinator_for_request(next_request_handler) + target_node = None + break + else: + target_node = self._client.least_loaded_node() + if target_node is not None and not self._client.await_ready(target_node, timeout_ms=self.config['request_timeout_ms']): + target_node = None + + if target_node is not None: + if next_request_handler.is_retry: + time.sleep(self.config['retry_backoff_ms'] / 1000) + txn_correlation_id = self._transaction_manager.next_in_flight_request_correlation_id() + future = self._client.send(target_node, next_request_handler.request) + future.add_both(next_request_handler.on_complete, txn_correlation_id) + return True + + except Exception as e: + log.warn("Got an exception when trying to find a node to send a transactional request to. Going to back off and retry", e) + if next_request_handler.needs_coordinator(): + self._transaction_manager.lookup_coordinator_for_request(next_request_handler) + break - # if some partitions are already ready to be sent, the select time - # would be 0; otherwise if some partition already has some data - # accumulated but not ready yet, the select time will be the time - # difference between now and its linger expiry time; otherwise the - # select time will be the time difference between now and the - # metadata expiry time - self._client.poll(timeout_ms=poll_timeout_ms) + time.sleep(self.config['retry_backoff_ms'] / 1000) + self._metadata.request_update() + + if target_node is None: + self._transaction_manager.retry(next_request_handler) + + return True + + def _maybe_abort_batches(self, exc): + if self._accumulator.has_incomplete: + log.error("Aborting producer batches due to fatal error: %s", exc) + self._accumulator.abort_batches(exc) def initiate_close(self): """Start closing the sender (won't complete until all data is sent).""" @@ -201,10 +302,7 @@ def add_topic(self, topic): self.wakeup() def _maybe_wait_for_producer_id(self): - if not self._transaction_state: - return - - while not self._transaction_state.has_pid(): + while not self._transaction_manager.has_producer_id(): try: node_id = self._client.least_loaded_node() if node_id is None or not self._client.await_ready(node_id): @@ -220,19 +318,19 @@ def _maybe_wait_for_producer_id(self): response = self._client.send_and_receive(node_id, request) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - self._transaction_state.set_producer_id_and_epoch(response.producer_id, response.producer_epoch) - return + self._transaction_manager.set_producer_id_and_epoch(ProducerIdAndEpoch(response.producer_id, response.producer_epoch)) elif getattr(error_type, 'retriable', False): log.debug("Retriable error from InitProducerId response: %s", error_type.__name__) if getattr(error_type, 'invalid_metadata', False): self._metadata.request_update() else: - log.error("Received a non-retriable error from InitProducerId response: %s", error_type.__name__) + self._transaction_manager.transition_to_fatal_error(error_type()) break except Errors.KafkaConnectionError: log.debug("Broker %s disconnected while awaiting InitProducerId response", node_id) except Errors.RequestTimedOutError: log.debug("InitProducerId request to node %s timed out", node_id) + log.debug("Retry InitProducerIdRequest in %sms.", self.config['retry_backoff_ms']) time.sleep(self.config['retry_backoff_ms'] / 1000) def _failed_produce(self, batches, node_id, error): @@ -271,13 +369,29 @@ def _handle_produce_response(self, node_id, send_time, batches, response): for batch in batches: self._complete_batch(batch, None, -1) - def _fail_batch(self, batch, *args, **kwargs): - if self._transaction_state and self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: - # 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 - # numbers for all existing partitions. - self._transaction_state.reset_producer_id() - batch.done(*args, **kwargs) + def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None, log_start_offset=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("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.", + batch.topic_partition, 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 + # numbers for all existing partitions. + self._transaction_manager.reset_producer_id() + elif isinstance(exception, (Errors.ClusterAuthorizationFailedError, + Errors.TransactionalIdAuthorizationFailedError, + Errors.ProducerFencedError, + Errors.InvalidTxnStateError)): + self._transaction_manager.transition_to_fatal_error(exception) + elif self._transaction_manager.is_transactional(): + self._transaction_manager.transition_to_abortable_error(exception) + + batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, exception=exception, log_start_offset=log_start_offset) self._accumulator.deallocate(batch) if self._sensors: self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) @@ -286,7 +400,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star """Complete or retry the given batch of records. Arguments: - batch (RecordBatch): The record batch + 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 @@ -305,29 +419,25 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star self.config['retries'] - batch.attempts - 1, error) - # If idempotence is enabled only retry the request if the current PID is the same as the pid of the batch. - if not self._transaction_state or self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: + # 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): log.debug("Retrying batch to topic-partition %s. Sequence number: %s", batch.topic_partition, - self._transaction_state.sequence_number(batch.topic_partition) if self._transaction_state else None) + self._transaction_manager.sequence_number(batch.topic_partition) if self._transaction_manager else None) self._accumulator.reenqueue(batch) if self._sensors: self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) else: - log.warning("Attempted to retry sending a batch but the producer id changed from %s to %s. This batch will be dropped" % ( - batch.producer_id, self._transaction_state.producer_id_and_epoch.producer_id)) - self._fail_batch(batch, base_offset=base_offset, timestamp_ms=timestamp_ms, exception=error, log_start_offset=log_start_offset) + log.warning("Attempted to retry sending a batch but the producer id/epoch changed from %s/%s to %s/%s. This batch will be dropped" % ( + 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, log_start_offset=log_start_offset) else: - if error is Errors.OutOfOrderSequenceNumberError and batch.producer_id == self._transaction_state.producer_id_and_epoch.producer_id: - log.error("The broker received an out of order sequence number error for produer_id %s, topic-partition %s" - " at offset %s. This indicates data loss on the broker, and should be investigated.", - batch.producer_id, batch.topic_partition, base_offset) - if error is Errors.TopicAuthorizationFailedError: error = error(batch.topic_partition.topic) # tell the user the result of their request - self._fail_batch(batch, base_offset=base_offset, timestamp_ms=timestamp_ms, exception=error, log_start_offset=log_start_offset) + self._fail_batch(batch, error, base_offset=base_offset, timestamp_ms=timestamp_ms, log_start_offset=log_start_offset) if error is Errors.UnknownTopicOrPartitionError: log.warning("Received unknown topic or partition error in produce request on partition %s." @@ -341,10 +451,10 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, log_start_offset=log_start_offset) self._accumulator.deallocate(batch) - if self._transaction_state and self._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id: - self._transaction_state.increment_sequence_number(batch.topic_partition, batch.record_count) + if self._transaction_manager and self._transaction_manager.producer_id_and_epoch.match(batch): + self._transaction_manager.increment_sequence_number(batch.topic_partition, batch.record_count) log.debug("Incremented sequence number for topic-partition %s to %s", batch.topic_partition, - self._transaction_state.sequence_number(batch.topic_partition)) + self._transaction_manager.sequence_number(batch.topic_partition)) # Unmute the completed partition. if self.config['guarantee_message_order']: @@ -364,16 +474,17 @@ def _create_produce_requests(self, collated): per-node basis. Arguments: - collated: {node_id: [RecordBatch]} + collated: {node_id: [ProducerBatch]} Returns: dict: {node_id: ProduceRequest} (version depends on client api_versions) """ requests = {} for node_id, batches in six.iteritems(collated): - requests[node_id] = self._produce_request( - node_id, self.config['acks'], - self.config['request_timeout_ms'], batches) + if batches: + requests[node_id] = self._produce_request( + node_id, self.config['acks'], + self.config['request_timeout_ms'], batches) return requests def _produce_request(self, node_id, acks, timeout, batches): @@ -391,14 +502,25 @@ def _produce_request(self, node_id, acks, timeout, batches): produce_records_by_partition[topic][partition] = buf version = self._client.api_version(ProduceRequest, max_version=7) - # TODO: support transactional_id - return ProduceRequest[version]( - required_acks=acks, - timeout=timeout, - topics=[(topic, list(partition_info.items())) - for topic, partition_info - in six.iteritems(produce_records_by_partition)], - ) + topic_partition_data = [ + (topic, list(partition_info.items())) + for topic, partition_info in six.iteritems(produce_records_by_partition)] + transactional_id = self._transaction_manager.transactional_id if self._transaction_manager else None + if version >= 3: + return ProduceRequest[version]( + transactional_id=transactional_id, + required_acks=acks, + timeout=timeout, + topics=topic_partition_data, + ) + else: + if transactional_id is not None: + log.warning('Broker does not support ProduceRequest v3+, required for transactional_id') + return ProduceRequest[version]( + required_acks=acks, + timeout=timeout, + topics=topic_partition_data, + ) def wakeup(self): """Wake up the selector associated with this send thread.""" @@ -561,8 +683,9 @@ def update_produce_request_metrics(self, batches_map): records += batch.record_count total_bytes += batch.records.size_in_bytes() - self.records_per_request_sensor.record(records) - self.byte_rate_sensor.record(total_bytes) + if node_batch: + self.records_per_request_sensor.record(records) + self.byte_rate_sensor.record(total_bytes) def record_retries(self, topic, count): self.retry_sensor.record(count) diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py new file mode 100644 index 000000000..f5111c780 --- /dev/null +++ b/kafka/producer/transaction_manager.py @@ -0,0 +1,812 @@ +from __future__ import absolute_import, division + +import abc +import collections +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_partitions_to_txn import AddPartitionsToTxnRequest +from kafka.protocol.end_txn import EndTxnRequest +from kafka.protocol.find_coordinator import FindCoordinatorRequest +from kafka.protocol.init_producer_id import InitProducerIdRequest +from kafka.structs import TopicPartition + + +log = logging.getLogger(__name__) + + +NO_PRODUCER_ID = -1 +NO_PRODUCER_EPOCH = -1 +NO_SEQUENCE = -1 + + +class ProducerIdAndEpoch(object): + __slots__ = ('producer_id', 'epoch') + + def __init__(self, producer_id, epoch): + self.producer_id = producer_id + self.epoch = epoch + + @property + def is_valid(self): + return NO_PRODUCER_ID < self.producer_id + + def match(self, batch): + return self.producer_id == batch.producer_id and self.epoch == batch.producer_epoch + + def __str__(self): + return "ProducerIdAndEpoch(producer_id={}, epoch={})".format(self.producer_id, self.epoch) + + +class TransactionState(IntEnum): + UNINITIALIZED = 0 + INITIALIZING = 1 + READY = 2 + IN_TRANSACTION = 3 + COMMITTING_TRANSACTION = 4 + ABORTING_TRANSACTION = 5 + ABORTABLE_ERROR = 6 + FATAL_ERROR = 7 + + @classmethod + def is_transition_valid(cls, source, target): + if target == cls.INITIALIZING: + return source == cls.UNINITIALIZED + elif target == cls.READY: + return source in (cls.INITIALIZING, cls.COMMITTING_TRANSACTION, cls.ABORTING_TRANSACTION) + elif target == cls.IN_TRANSACTION: + return source == cls.READY + elif target == cls.COMMITTING_TRANSACTION: + return source == cls.IN_TRANSACTION + elif target == cls.ABORTING_TRANSACTION: + return source in (cls.IN_TRANSACTION, cls.ABORTABLE_ERROR) + elif target == cls.ABORTABLE_ERROR: + return source in (cls.IN_TRANSACTION, cls.COMMITTING_TRANSACTION, cls.ABORTABLE_ERROR) + elif target == cls.UNINITIALIZED: + # Disallow transitions to UNITIALIZED + return False + elif target == cls.FATAL_ERROR: + # We can transition to FATAL_ERROR unconditionally. + # FATAL_ERROR is never a valid starting state for any transition. So the only option is to close the + # producer or do purely non transactional requests. + return True + + +class Priority(IntEnum): + # We use the priority to determine the order in which requests need to be sent out. For instance, if we have + # a pending FindCoordinator request, that must always go first. Next, If we need a producer id, that must go second. + # The endTxn request must always go last. + FIND_COORDINATOR = 0 + INIT_PRODUCER_ID = 1 + ADD_PARTITIONS_OR_OFFSETS = 2 + END_TXN = 3 + + +class TransactionManager(object): + """ + A class which maintains state for transactions. Also keeps the state necessary to ensure idempotent production. + """ + NO_INFLIGHT_REQUEST_CORRELATION_ID = -1 + # The retry_backoff_ms is overridden to the following value if the first AddPartitions receives a + # CONCURRENT_TRANSACTIONS error. + ADD_PARTITIONS_RETRY_BACKOFF_MS = 20 + + def __init__(self, transactional_id=None, transaction_timeout_ms=0, retry_backoff_ms=100, api_version=(0, 11), metadata=None): + self._api_version = api_version + self._metadata = metadata + + self._sequence_numbers = collections.defaultdict(lambda: 0) + + self.transactional_id = transactional_id + self.transaction_timeout_ms = transaction_timeout_ms + self._transaction_coordinator = None + self._consumer_group_coordinator = None + self._new_partitions_in_transaction = set() + self._pending_partitions_in_transaction = set() + self._partitions_in_transaction = set() + + self._current_state = TransactionState.UNINITIALIZED + self._last_error = None + self.producer_id_and_epoch = ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH) + + self._transaction_started = False + + self._pending_requests = [] # priority queue via heapq + self._pending_requests_sort_id = 0 + self._in_flight_request_correlation_id = self.NO_INFLIGHT_REQUEST_CORRELATION_ID + + # This is used by the TxnRequestHandlers to control how long to back off before a given request is retried. + # For instance, this value is lowered by the AddPartitionsToTxnHandler when it receives a CONCURRENT_TRANSACTIONS + # error for the first AddPartitionsRequest in a transaction. + self.retry_backoff_ms = retry_backoff_ms + self._lock = threading.Condition() + + def initialize_transactions(self): + with self._lock: + self._ensure_transactional() + self._transition_to(TransactionState.INITIALIZING) + self.set_producer_id_and_epoch(ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH)) + self._sequence_numbers.clear() + handler = InitProducerIdHandler(self, self.transactional_id, self.transaction_timeout_ms) + self._enqueue_request(handler) + return handler.result + + def begin_transaction(self): + with self._lock: + self._ensure_transactional() + self._maybe_fail_with_error() + self._transition_to(TransactionState.IN_TRANSACTION) + + def begin_commit(self): + with self._lock: + self._ensure_transactional() + self._maybe_fail_with_error() + self._transition_to(TransactionState.COMMITTING_TRANSACTION) + return self._begin_completing_transaction(True) + + def begin_abort(self): + with self._lock: + self._ensure_transactional() + if self._current_state != TransactionState.ABORTABLE_ERROR: + self._maybe_fail_with_error() + self._transition_to(TransactionState.ABORTING_TRANSACTION) + + # We're aborting the transaction, so there should be no need to add new partitions + self._new_partitions_in_transaction.clear() + return self._begin_completing_transaction(False) + + def _begin_completing_transaction(self, committed): + if self._new_partitions_in_transaction: + self._enqueue_request(self._add_partitions_to_transaction_handler()) + handler = EndTxnHandler(self, self.transactional_id, self.producer_id_and_epoch.producer_id, self.producer_id_and_epoch.epoch, committed) + self._enqueue_request(handler) + return handler.result + + def maybe_add_partition_to_transaction(self, topic_partition): + with self._lock: + self._fail_if_not_ready_for_send() + + if self.is_partition_added(topic_partition) or self.is_partition_pending_add(topic_partition): + return + + log.debug("Begin adding new partition %s to transaction", topic_partition) + self._new_partitions_in_transaction.add(topic_partition) + + def _fail_if_not_ready_for_send(self): + with self._lock: + if self.has_error(): + raise Errors.KafkaError( + "Cannot perform send because at least one previous transactional or" + " idempotent request has failed with errors.", self._last_error) + + if self.is_transactional(): + if not self.has_producer_id(): + raise Errors.IllegalStateError( + "Cannot perform a 'send' before completing a call to initTransactions" + " when transactions are enabled.") + + if self._current_state != TransactionState.IN_TRANSACTION: + raise Errors.IllegalStateError("Cannot call send in state %s" % (self._current_state.name,)) + + def is_send_to_partition_allowed(self, tp): + with self._lock: + if self.has_fatal_error(): + return False + return not self.is_transactional() or tp in self._partitions_in_transaction + + def has_producer_id(self, producer_id=None): + if producer_id is None: + return self.producer_id_and_epoch.is_valid + else: + return self.producer_id_and_epoch.producer_id == producer_id + + def is_transactional(self): + return self.transactional_id is not None + + def has_partitions_to_add(self): + with self._lock: + return bool(self._new_partitions_in_transaction) or bool(self._pending_partitions_in_transaction) + + def is_completing(self): + with self._lock: + return self._current_state in ( + TransactionState.COMMITTING_TRANSACTION, + TransactionState.ABORTING_TRANSACTION) + + @property + def last_error(self): + return self._last_error + + def has_error(self): + with self._lock: + return self._current_state in ( + TransactionState.ABORTABLE_ERROR, + TransactionState.FATAL_ERROR) + + def is_aborting(self): + with self._lock: + return self._current_state == TransactionState.ABORTING_TRANSACTION + + def transition_to_abortable_error(self, exc): + with self._lock: + if self._current_state == TransactionState.ABORTING_TRANSACTION: + log.debug("Skipping transition to abortable error state since the transaction is already being " + " aborted. Underlying exception: ", exc) + return + self._transition_to(TransactionState.ABORTABLE_ERROR, error=exc) + + def transition_to_fatal_error(self, exc): + with self._lock: + self._transition_to(TransactionState.FATAL_ERROR, error=exc) + + def is_partition_added(self, partition): + with self._lock: + return partition in self._partitions_in_transaction + + def is_partition_pending_add(self, partition): + return partition in self._new_partitions_in_transaction or partition in self._pending_partitions_in_transaction + + def has_producer_id_and_epoch(self, producer_id, producer_epoch): + return ( + self.producer_id_and_epoch.producer_id == producer_id and + self.producer_id_and_epoch.epoch == producer_epoch + ) + + def set_producer_id_and_epoch(self, producer_id_and_epoch): + if not isinstance(producer_id_and_epoch, ProducerIdAndEpoch): + raise TypeError("ProducerAndIdEpoch type required") + log.info("ProducerId set to %s with epoch %s", + producer_id_and_epoch.producer_id, producer_id_and_epoch.epoch) + self.producer_id_and_epoch = producer_id_and_epoch + + def reset_producer_id(self): + """ + This method is used when the producer needs to reset its internal state because of an irrecoverable exception + from the broker. + + We need to reset the producer id and associated state when we have sent a batch to the broker, but we either get + a non-retriable exception or we run out of retries, or the batch expired in the producer queue after it was already + sent to the broker. + + In all of these cases, we don't know whether batch was actually committed on the broker, and hence whether the + sequence number was actually updated. If we don't reset the producer state, we risk the chance that all future + messages will return an OutOfOrderSequenceNumberError. + + Note that we can't reset the producer state for the transactional producer as this would mean bumping the epoch + for the same producer id. This might involve aborting the ongoing transaction during the initProducerIdRequest, + and the user would not have any way of knowing this happened. So for the transactional producer, + it's best to return the produce error to the user and let them abort the transaction and close the producer explicitly. + """ + with self._lock: + if self.is_transactional: + raise Errors.IllegalStateError( + "Cannot reset producer state for a transactional producer." + " You must either abort the ongoing transaction or" + " reinitialize the transactional producer instead") + self.set_producer_id_and_epoch(ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH)) + self._sequence_numbers.clear() + + def sequence_number(self, tp): + with self._lock: + return self._sequence_numbers[tp] + + def increment_sequence_number(self, tp, increment): + with self._lock: + if tp not in self._sequence_numbers: + raise Errors.IllegalStateError("Attempt to increment sequence number for a partition with no current sequence.") + # Sequence number wraps at java max int + base = self._sequence_numbers[tp] + if base > (2147483647 - increment): + self._sequence_numbers[tp] = increment - (2147483647 - base) - 1 + else: + self._sequence_numbers[tp] += increment + + def next_request_handler(self, has_incomplete_batches): + with self._lock: + if self._new_partitions_in_transaction: + self._enqueue_request(self._add_partitions_to_transaction_handler()) + + if not self._pending_requests: + return None + + _, _, next_request_handler = self._pending_requests[0] + # Do not send the EndTxn until all batches have been flushed + if isinstance(next_request_handler, EndTxnHandler) and has_incomplete_batches: + return None + + heapq.heappop(self._pending_requests) + if self._maybe_terminate_request_with_error(next_request_handler): + log.debug("Not sending transactional request %s because we are in an error state", + next_request_handler.request) + return None + + if isinstance(next_request_handler, EndTxnHandler) and not self._transaction_started: + next_request_handler.result.done() + if self._current_state != TransactionState.FATAL_ERROR: + log.debug("Not sending EndTxn for completed transaction since no partitions" + " or offsets were successfully added") + self._complete_transaction() + try: + _, _, next_request_handler = heapq.heappop(self._pending_requests) + except IndexError: + next_request_handler = None + + if next_request_handler: + log.debug("Request %s dequeued for sending", next_request_handler.request) + + return next_request_handler + + def retry(self, request): + with self._lock: + request.set_retry() + self._enqueue_request(request) + + def authentication_failed(self, exc): + with self._lock: + for _, _, request in self._pending_requests: + request.fatal_error(exc) + + def coordinator(self, coord_type): + if coord_type == 'group': + return self._consumer_group_coordinator + elif coord_type == 'transaction': + return self._transaction_coordinator + else: + raise Errors.IllegalStateError("Received an invalid coordinator type: %s" % (coord_type,)) + + def lookup_coordinator_for_request(self, request): + self._lookup_coordinator(request.coordinator_type, request.coordinator_key) + + def next_in_flight_request_correlation_id(self): + self._in_flight_request_correlation_id += 1 + return self._in_flight_request_correlation_id + + def clear_in_flight_transactional_request_correlation_id(self): + self._in_flight_request_correlation_id = self.NO_INFLIGHT_REQUEST_CORRELATION_ID + + def has_in_flight_transactional_request(self): + return self._in_flight_request_correlation_id != self.NO_INFLIGHT_REQUEST_CORRELATION_ID + + def has_fatal_error(self): + return self._current_state == TransactionState.FATAL_ERROR + + def has_abortable_error(self): + return self._current_state == TransactionState.ABORTABLE_ERROR + + # visible for testing + def _test_transaction_contains_partition(self, tp): + with self._lock: + return tp in self._partitions_in_transaction + + # visible for testing + def _test_has_ongoing_transaction(self): + with self._lock: + # transactions are considered ongoing once started until completion or a fatal error + return self._current_state == TransactionState.IN_TRANSACTION or self.is_completing() or self.has_abortable_error() + + # visible for testing + def _test_is_ready(self): + with self._lock: + return self.is_transactional() and self._current_state == TransactionState.READY + + def _transition_to(self, target, error=None): + with self._lock: + if not self._current_state.is_transition_valid(self._current_state, target): + raise Errors.KafkaError("TransactionalId %s: Invalid transition attempted from state %s to state %s" % ( + self.transactional_id, self._current_state.name, target.name)) + + if target in (TransactionState.FATAL_ERROR, TransactionState.ABORTABLE_ERROR): + if error is None: + raise Errors.IllegalArgumentError("Cannot transition to %s with an None exception" % (target.name,)) + self._last_error = error + else: + self._last_error = None + + if self._last_error is not None: + log.debug("Transition from state %s to error state %s (%s)", self._current_state.name, target.name, self._last_error) + else: + log.debug("Transition from state %s to %s", self._current_state, target) + self._current_state = target + + def _ensure_transactional(self): + if not self.is_transactional(): + raise Errors.IllegalStateError("Transactional method invoked on a non-transactional producer.") + + def _maybe_fail_with_error(self): + if self.has_error(): + raise Errors.KafkaError("Cannot execute transactional method because we are in an error state: %s" % (self._last_error,)) + + def _maybe_terminate_request_with_error(self, request_handler): + if self.has_error(): + if self.has_abortable_error() and isinstance(request_handler, FindCoordinatorHandler): + # No harm letting the FindCoordinator request go through if we're expecting to abort + return False + request_handler.fail(self._last_error) + return True + return False + + def _next_pending_requests_sort_id(self): + self._pending_requests_sort_id += 1 + return self._pending_requests_sort_id + + def _enqueue_request(self, request_handler): + log.debug("Enqueuing transactional request %s", request_handler.request) + heapq.heappush( + self._pending_requests, + ( + request_handler.priority, # keep lowest priority at head of queue + self._next_pending_requests_sort_id(), # break ties + request_handler + ) + ) + + def _lookup_coordinator(self, coord_type, coord_key): + with self._lock: + if coord_type == 'group': + self._consumer_group_coordinator = None + elif coord_type == 'transaction': + self._transaction_coordinator = None + else: + raise Errors.IllegalStateError("Invalid coordinator type: %s" % (coord_type,)) + self._enqueue_request(FindCoordinatorHandler(self, coord_type, coord_key)) + + def _complete_transaction(self): + with self._lock: + self._transition_to(TransactionState.READY) + self._transaction_started = False + self._new_partitions_in_transaction.clear() + self._pending_partitions_in_transaction.clear() + self._partitions_in_transaction.clear() + + def _add_partitions_to_transaction_handler(self): + with self._lock: + self._pending_partitions_in_transaction.update(self._new_partitions_in_transaction) + self._new_partitions_in_transaction.clear() + return AddPartitionsToTxnHandler(self, self.transactional_id, self.producer_id_and_epoch.producer_id, self.producer_id_and_epoch.epoch, self._pending_partitions_in_transaction) + + +class TransactionalRequestResult(object): + def __init__(self): + self._latch = threading.Event() + self._error = None + + def done(self, error=None): + self._error = error + self._latch.set() + + def wait(self, timeout_ms=None): + timeout = timeout_ms / 1000 if timeout_ms is not None else None + success = self._latch.wait(timeout) + if self._error: + raise self._error + return success + + @property + def is_done(self): + return self._latch.is_set() + + @property + def succeeded(self): + return self._latch.is_set() and self._error is None + + @property + def failed(self): + return self._latch.is_set() and self._error is not None + + @property + def exception(self): + return self._error + + +@six.add_metaclass(abc.ABCMeta) +class TxnRequestHandler(object): + def __init__(self, transaction_manager, result=None): + self.transaction_manager = transaction_manager + self.retry_backoff_ms = transaction_manager.retry_backoff_ms + self.request = None + self._result = result or TransactionalRequestResult() + self._is_retry = False + + def fatal_error(self, 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._result.done(error=exc) + + def fail(self, exc): + self._result.done(error=exc) + + def reenqueue(self): + with self.transaction_manager._lock: + self._is_retry = True + self.transaction_manager._enqueue_request(self) + + def on_complete(self, correlation_id, response_or_exc): + if correlation_id != self.transaction_manager._in_flight_request_correlation_id: + self.fatal_error(RuntimeError("Detected more than one in-flight transactional request.")) + else: + self.transaction_manager.clear_in_flight_transactional_request_correlation_id() + if isinstance(response_or_exc, Errors.KafkaConnectionError): + log.debug("Disconnected from node. Will retry.") + if self.needs_coordinator(): + self.transaction_manager._lookup_coordinator(self.coordinator_type, self.coordinator_key) + self.reenqueue() + elif isinstance(response_or_exc, Errors.UnsupportedVersionError): + self.fatal_error(response_or_exc) + elif not isinstance(response_or_exc, (Exception, type(None))): + log.debug("Received transactional response %s for request %s", response_or_exc, self.request) + with self.transaction_manager._lock: + self.handle_response(response_or_exc) + else: + self.fatal_error(Errors.KafkaError("Could not execute transactional request for unknown reasons: %s" % response_or_exc)) + + def needs_coordinator(self): + return self.coordinator_type is not None + + @property + def result(self): + return self._result + + @property + def coordinator_type(self): + return 'transaction' + + @property + def coordinator_key(self): + return self.transaction_manager.transactional_id + + def set_retry(self): + self._is_retry = True + + @property + def is_retry(self): + return self._is_retry + + @abc.abstractmethod + def handle_response(self, response): + pass + + @abc.abstractproperty + def priority(self): + pass + + +class InitProducerIdHandler(TxnRequestHandler): + def __init__(self, transaction_manager, transactional_id, transaction_timeout_ms): + super(InitProducerIdHandler, self).__init__(transaction_manager) + + self.transactional_id = transactional_id + if transaction_manager._api_version >= (2, 0): + version = 1 + else: + version = 0 + self.request = InitProducerIdRequest[version]( + transactional_id=transactional_id, + transaction_timeout_ms=transaction_timeout_ms) + + @property + def priority(self): + return Priority.INIT_PRODUCER_ID + + def handle_response(self, response): + error = Errors.for_code(response.error_code) + + if error is Errors.NoError: + self.transaction_manager.set_producer_id_and_epoch(ProducerIdAndEpoch(response.producer_id, response.producer_epoch)) + self.transaction_manager._transition_to(TransactionState.READY) + self._result.done() + elif error in (Errors.NotCoordinatorError, Errors.CoordinatorNotAvailableError): + self.transaction_manager._lookup_coordinator('transaction', self.transactional_id) + self.reenqueue() + elif error in (Errors.CoordinatorLoadInProgressError, Errors.ConcurrentTransactionsError): + self.reenqueue() + elif error is Errors.TransactionalIdAuthorizationFailedError: + self.fatal_error(error()) + else: + self.fatal_error(Errors.KafkaError("Unexpected error in InitProducerIdResponse: %s" % (error()))) + +class AddPartitionsToTxnHandler(TxnRequestHandler): + def __init__(self, transaction_manager, transactional_id, producer_id, producer_epoch, topic_partitions): + super(AddPartitionsToTxnHandler, self).__init__(transaction_manager) + + self.transactional_id = transactional_id + if transaction_manager._api_version >= (2, 7): + version = 2 + elif transaction_manager._api_version >= (2, 0): + version = 1 + else: + version = 0 + topic_data = collections.defaultdict(list) + for tp in topic_partitions: + topic_data[tp.topic].append(tp.partition) + self.request = AddPartitionsToTxnRequest[version]( + transactional_id=transactional_id, + producer_id=producer_id, + producer_epoch=producer_epoch, + topics=list(topic_data.items())) + + @property + def priority(self): + return Priority.ADD_PARTITIONS_OR_OFFSETS + + def handle_response(self, response): + has_partition_errors = False + unauthorized_topics = set() + self.retry_backoff_ms = self.transaction_manager.retry_backoff_ms + + results = {TopicPartition(topic, partition): Errors.for_code(error_code) + for topic, partition_data in response.results + for partition, error_code in partition_data} + + for tp, error in six.iteritems(results): + if error is Errors.NoError: + continue + elif error in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): + self.transaction_manager._lookup_coordinator('transaction', self.transactiona_id) + self.reenqueue() + return + elif error is Errors.ConcurrentTransactionsError: + self.maybe_override_retry_backoff_ms() + self.reenqueue() + return + elif error in (Errors.CoordinatorLoadInProgressError, Errors.UnknownTopicOrPartitionError): + self.reenqueue() + return + elif error is Errors.InvalidProducerEpochError: + self.fatal_error(error()) + return + elif error is Errors.TransactionalIdAuthorizationFailedError: + self.fatal_error(error()) + return + elif error in (Errors.InvalidProducerIdMappingError, Errors.InvalidTxnStateError): + self.fatal_error(Errors.KafkaError(error())) + return + elif error is Errors.TopicAuthorizationFailedError: + unauthorized_topics.add(tp.topic) + elif error is Errors.OperationNotAttemptedError: + log.debug("Did not attempt to add partition %s to transaction because other partitions in the" + " batch had errors.", tp) + has_partition_errors = True + else: + log.error("Could not add partition %s due to unexpected error %s", tp, error()) + has_partition_errors = True + + partitions = set(results) + + # Remove the partitions from the pending set regardless of the result. We use the presence + # of partitions in the pending set to know when it is not safe to send batches. However, if + # the partitions failed to be added and we enter an error state, we expect the batches to be + # aborted anyway. In this case, we must be able to continue sending the batches which are in + # retry for partitions that were successfully added. + self.transaction_manager._pending_partitions_in_transaction -= partitions + + if unauthorized_topics: + self.abortable_error(Errors.TopicAuthorizationError(unauthorized_topics)) + elif has_partition_errors: + self.abortable_error(Errors.KafkaError("Could not add partitions to transaction due to errors: %s" % (results))) + else: + log.debug("Successfully added partitions %s to transaction", partitions) + self.transaction_manager._partitions_in_transaction.update(partitions) + self.transaction_manager._transaction_started = True + self._result.done() + + def maybe_override_retry_backoff_ms(self): + # We only want to reduce the backoff when retrying the first AddPartition which errored out due to a + # CONCURRENT_TRANSACTIONS error since this means that the previous transaction is still completing and + # we don't want to wait too long before trying to start the new one. + # + # This is only a temporary fix, the long term solution is being tracked in + # https://issues.apache.org/jira/browse/KAFKA-5482 + if not self.transaction_manager._partitions_in_transaction: + self.retry_backoff_ms = min(self.transaction_manager.ADD_PARTITIONS_RETRY_BACKOFF_MS, self.retry_backoff_ms) + + +class FindCoordinatorHandler(TxnRequestHandler): + def __init__(self, transaction_manager, coord_type, coord_key): + super(FindCoordinatorHandler, self).__init__(transaction_manager) + + self._coord_type = coord_type + self._coord_key = coord_key + if transaction_manager._api_version >= (2, 0): + version = 2 + else: + version = 1 + if coord_type == 'group': + coord_type_int8 = 0 + elif coord_type == 'transaction': + coord_type_int8 = 1 + else: + raise ValueError("Unrecognized coordinator type: %s" % (coord_type,)) + self.request = FindCoordinatorRequest[version]( + coordinator_key=coord_key, + coordinator_type=coord_type_int8, + ) + + @property + def priority(self): + return Priority.FIND_COORDINATOR + + @property + def coordinator_type(self): + return None + + @property + def coordinator_key(self): + return None + + def handle_response(self, response): + error = Errors.for_code(response.error_code) + + if error is Errors.NoError: + coordinator_id = self.transaction_manager._metadata.add_coordinator( + response, self._coord_type, self._coord_key) + if self._coord_type == 'group': + self.transaction_manager._consumer_group_coordinator = coordinator_id + elif self._coord_type == 'transaction': + self.transaction_manager._transaction_coordinator = coordinator_id + self._result.done() + elif error is Errors.CoordinatorNotAvailableError: + self.reenqueue() + elif error is Errors.TransactionalIdAuthorizationFailedError: + self.fatal_error(error()) + elif error is Errors.GroupAuthorizationFailedError: + self.abortable_error(Errors.GroupAuthorizationError(self._coord_key)) + else: + self.fatal_error(Errors.KafkaError( + "Could not find a coordinator with type %s with key %s due to" + " unexpected error: %s" % (self._coord_type, self._coord_key, error()))) + + +class EndTxnHandler(TxnRequestHandler): + def __init__(self, transaction_manager, transactional_id, producer_id, producer_epoch, committed): + super(EndTxnHandler, self).__init__(transaction_manager) + + self.transactional_id = transactional_id + if self.transaction_manager._api_version >= (2, 7): + version = 2 + elif self.transaction_manager._api_version >= (2, 0): + version = 1 + else: + version = 0 + self.request = EndTxnRequest[version]( + transactional_id=transactional_id, + producer_id=producer_id, + producer_epoch=producer_epoch, + committed=committed) + + @property + def priority(self): + return Priority.END_TXN + + def handle_response(self, response): + error = Errors.for_code(response.error_code) + + if error is Errors.NoError: + self.transaction_manager._complete_transaction() + self._result.done() + elif error in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): + self.transaction_manager._lookup_coordinator('transaction', self.transactional_id) + self.reenqueue() + elif error in (Errors.CoordinatorLoadInProgressError, Errors.ConcurrentTransactionsError): + self.reenqueue() + elif error is Errors.InvalidProducerEpochError: + self.fatal_error(error()) + elif error is Errors.TransactionalIdAuthorizationFailedError: + self.fatal_error(error()) + elif error is Errors.InvalidTxnStateError: + self.fatal_error(error()) + else: + self.fatal_error(Errors.KafkaError("Unhandled error in EndTxnResponse: %s" % (error()))) diff --git a/kafka/producer/transaction_state.py b/kafka/producer/transaction_state.py deleted file mode 100644 index 05cdc5766..000000000 --- a/kafka/producer/transaction_state.py +++ /dev/null @@ -1,96 +0,0 @@ -from __future__ import absolute_import, division - -import collections -import threading -import time - -from kafka.errors import IllegalStateError - - -NO_PRODUCER_ID = -1 -NO_PRODUCER_EPOCH = -1 - - -class ProducerIdAndEpoch(object): - __slots__ = ('producer_id', 'epoch') - - def __init__(self, producer_id, epoch): - self.producer_id = producer_id - self.epoch = epoch - - @property - def is_valid(self): - return NO_PRODUCER_ID < self.producer_id - - def __str__(self): - return "ProducerIdAndEpoch(producer_id={}, epoch={})".format(self.producer_id, self.epoch) - -class TransactionState(object): - __slots__ = ('producer_id_and_epoch', '_sequence_numbers', '_lock') - - def __init__(self): - self.producer_id_and_epoch = ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH) - self._sequence_numbers = collections.defaultdict(lambda: 0) - self._lock = threading.Condition() - - def has_pid(self): - return self.producer_id_and_epoch.is_valid - - - def await_producer_id_and_epoch(self, max_wait_time_ms): - """ - A blocking call to get the pid and epoch for the producer. If the PID and epoch has not been set, this method - will block for at most maxWaitTimeMs. It is expected that this method be called from application thread - contexts (ie. through Producer.send). The PID it self will be retrieved in the background thread. - - Arguments: - max_wait_time_ms (numeric): The maximum time to block. - - Returns: - ProducerIdAndEpoch object. Callers must check the 'is_valid' property of the returned object to ensure that a - valid pid and epoch is actually returned. - """ - with self._lock: - start = time.time() - elapsed = 0 - while not self.has_pid() and elapsed < max_wait_time_ms: - self._lock.wait(max_wait_time_ms / 1000) - elapsed = time.time() - start - return self.producer_id_and_epoch - - def set_producer_id_and_epoch(self, producer_id, epoch): - """ - Set the pid and epoch atomically. This method will signal any callers blocked on the `pidAndEpoch` method - once the pid is set. This method will be called on the background thread when the broker responds with the pid. - """ - with self._lock: - self.producer_id_and_epoch = ProducerIdAndEpoch(producer_id, epoch) - if self.producer_id_and_epoch.is_valid: - self._lock.notify_all() - - def reset_producer_id(self): - """ - This method is used when the producer needs to reset it's internal state because of an irrecoverable exception - from the broker. - - We need to reset the producer id and associated state when we have sent a batch to the broker, but we either get - a non-retriable exception or we run out of retries, or the batch expired in the producer queue after it was already - sent to the broker. - - In all of these cases, we don't know whether batch was actually committed on the broker, and hence whether the - sequence number was actually updated. If we don't reset the producer state, we risk the chance that all future - messages will return an OutOfOrderSequenceException. - """ - with self._lock: - self.producer_id_and_epoch = ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH) - self._sequence_numbers.clear() - - def sequence_number(self, tp): - with self._lock: - return self._sequence_numbers[tp] - - def increment_sequence_number(self, tp, increment): - with self._lock: - if tp not in self._sequence_numbers: - raise IllegalStateError("Attempt to increment sequence number for a partition with no current sequence.") - self._sequence_numbers[tp] += increment diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index c8305c88e..91d4a9d62 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -210,6 +210,10 @@ def producer_epoch(self): def base_sequence(self): return self._header_data[11] + @property + def has_sequence(self): + return self._header_data[11] != -1 # NO_SEQUENCE + @property def last_sequence(self): if self.base_sequence == self.NO_SEQUENCE: @@ -356,6 +360,17 @@ def validate_crc(self): verify_crc = calc_crc32c(data_view.tobytes()) return crc == verify_crc + def __str__(self): + return ( + "DefaultRecordBatch(magic={}, base_offset={}, last_offset_delta={}," + " first_timestamp={}, max_timestamp={}," + " is_transactional={}, producer_id={}, producer_epoch={}, base_sequence={}," + " records_count={})".format( + self.magic, self.base_offset, self.last_offset_delta, + self.first_timestamp, self.max_timestamp, + self.is_transactional, self.producer_id, self.producer_epoch, self.base_sequence, + self.records_count)) + class DefaultRecord(ABCRecord): @@ -493,15 +508,23 @@ def __init__( self._buffer = bytearray(self.HEADER_STRUCT.size) - def set_producer_state(self, producer_id, producer_epoch, base_sequence): + def set_producer_state(self, producer_id, producer_epoch, base_sequence, is_transactional): + assert not is_transactional or producer_id != -1, "Cannot write transactional messages without a valid producer ID" + assert producer_id == -1 or producer_epoch != -1, "Invalid negative producer epoch" + assert producer_id == -1 or base_sequence != -1, "Invalid negative sequence number" self._producer_id = producer_id self._producer_epoch = producer_epoch self._base_sequence = base_sequence + self._is_transactional = is_transactional @property def producer_id(self): return self._producer_id + @property + def producer_epoch(self): + return self._producer_epoch + def _get_attributes(self, include_compression_type=True): attrs = 0 if include_compression_type: @@ -706,6 +729,17 @@ def estimate_size_in_bytes(cls, key, value, headers): cls.size_of(key, value, headers) ) + def __str__(self): + return ( + "DefaultRecordBatchBuilder(magic={}, base_offset={}, last_offset_delta={}," + " first_timestamp={}, max_timestamp={}," + " is_transactional={}, producer_id={}, producer_epoch={}, base_sequence={}," + " records_count={})".format( + self._magic, 0, self._last_offset, + self._first_timestamp or 0, self._max_timestamp or 0, + self._is_transactional, self._producer_id, self._producer_epoch, self._base_sequence, + self._num_records)) + class DefaultRecordMetadata(object): diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index 77e38b9ed..4bf3115c8 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -113,18 +113,26 @@ def next_batch(self, _min_slice=MIN_SLICE, class MemoryRecordsBuilder(object): __slots__ = ("_builder", "_batch_size", "_buffer", "_next_offset", "_closed", - "_magic", "_bytes_written", "_producer_id") + "_magic", "_bytes_written", "_producer_id", "_producer_epoch") - def __init__(self, magic, compression_type, batch_size, offset=0): + def __init__(self, magic, compression_type, batch_size, offset=0, + transactional=False, producer_id=-1, producer_epoch=-1, base_sequence=-1): assert magic in [0, 1, 2], "Not supported magic" assert compression_type in [0, 1, 2, 3, 4], "Not valid compression type" if magic >= 2: + assert not transactional or producer_id != -1, "Cannot write transactional messages without a valid producer ID" + assert producer_id == -1 or producer_epoch != -1, "Invalid negative producer epoch" + assert producer_id == -1 or base_sequence != -1, "Invalid negative sequence number used" + self._builder = DefaultRecordBatchBuilder( magic=magic, compression_type=compression_type, - is_transactional=False, producer_id=-1, producer_epoch=-1, - base_sequence=-1, batch_size=batch_size) - self._producer_id = -1 + is_transactional=transactional, producer_id=producer_id, + producer_epoch=producer_epoch, base_sequence=base_sequence, + batch_size=batch_size) + self._producer_id = producer_id + self._producer_epoch = producer_epoch else: + assert not transactional and producer_id == -1, "Idempotent messages are not supported for magic %s" % (magic,) self._builder = LegacyRecordBatchBuilder( magic=magic, compression_type=compression_type, batch_size=batch_size) @@ -158,7 +166,7 @@ def append(self, timestamp, key, value, headers=[]): self._next_offset += 1 return metadata - def set_producer_state(self, producer_id, producer_epoch, base_sequence): + def set_producer_state(self, producer_id, producer_epoch, base_sequence, is_transactional): if self._magic < 2: raise UnsupportedVersionError('Producer State requires Message format v2+') elif self._closed: @@ -167,15 +175,17 @@ def set_producer_state(self, producer_id, producer_epoch, base_sequence): # be re queued. In this case, we should not attempt to set the state again, since changing the pid and sequence # once a batch has been sent to the broker risks introducing duplicates. raise IllegalStateError("Trying to set producer state of an already closed batch. This indicates a bug on the client.") - self._builder.set_producer_state(producer_id, producer_epoch, base_sequence) + self._builder.set_producer_state(producer_id, producer_epoch, base_sequence, is_transactional) self._producer_id = producer_id @property def producer_id(self): - if self._magic < 2: - raise UnsupportedVersionError('Producer State requires Message format v2+') return self._producer_id + @property + def producer_epoch(self): + return self._producer_epoch + def close(self): # This method may be called multiple times on the same batch # i.e., on retries @@ -187,6 +197,7 @@ def close(self): self._buffer = bytes(self._builder.build()) if self._magic == 2: self._producer_id = self._builder.producer_id + self._producer_epoch = self._builder.producer_epoch self._builder = None self._closed = True diff --git a/test/integration/test_producer_integration.py b/test/integration/test_producer_integration.py index 303832b9f..0739d8eba 100644 --- a/test/integration/test_producer_integration.py +++ b/test/integration/test_producer_integration.py @@ -1,8 +1,8 @@ +from __future__ import absolute_import + from contextlib import contextmanager -import gc import platform import time -import threading import pytest @@ -16,7 +16,7 @@ def producer_factory(**kwargs): try: yield producer finally: - producer.close(timeout=0) + producer.close(timeout=1) @contextmanager @@ -25,7 +25,7 @@ def consumer_factory(**kwargs): try: yield consumer finally: - consumer.close(timeout_ms=0) + consumer.close(timeout_ms=100) @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") @@ -77,18 +77,6 @@ def test_end_to_end(kafka_broker, compression): assert msgs == set(['msg %d' % (i,) for i in range(messages)]) -@pytest.mark.skipif(platform.python_implementation() != 'CPython', - reason='Test relies on CPython-specific gc policies') -def test_kafka_producer_gc_cleanup(): - gc.collect() - threads = threading.active_count() - producer = KafkaProducer(api_version='0.9') # set api_version explicitly to avoid auto-detection - assert threading.active_count() == threads + 1 - del(producer) - gc.collect() - assert threading.active_count() == threads - - @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") @pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd']) def test_kafka_producer_proper_record_metadata(kafka_broker, compression): @@ -145,3 +133,43 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression): partition=0) record = future.get(timeout=5) assert abs(record.timestamp - send_time) <= 1000 # Allow 1s deviation + + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason="Idempotent producer requires broker >=0.11") +def test_idempotent_producer(kafka_broker): + connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) + with producer_factory(bootstrap_servers=connect_str, enable_idempotence=True) as producer: + for _ in range(10): + producer.send('idempotent_test_topic', value=b'idempotent_msg').get(timeout=1) + + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason="Idempotent producer requires broker >=0.11") +def test_transactional_producer(kafka_broker): + connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) + with producer_factory(bootstrap_servers=connect_str, transactional_id='testing') as producer: + producer.init_transactions() + producer.begin_transaction() + producer.send('transactional_test_topic', partition=0, value=b'msg1').get() + producer.send('transactional_test_topic', partition=0, value=b'msg2').get() + producer.abort_transaction() + producer.begin_transaction() + producer.send('transactional_test_topic', partition=0, value=b'msg3').get() + producer.send('transactional_test_topic', partition=0, value=b'msg4').get() + producer.commit_transaction() + + messages = set() + consumer_opts = { + 'bootstrap_servers': connect_str, + 'group_id': None, + 'consumer_timeout_ms': 10000, + 'auto_offset_reset': 'earliest', + 'isolation_level': 'read_committed', + } + with consumer_factory(**consumer_opts) as consumer: + consumer.assign([TopicPartition('transactional_test_topic', 0)]) + for msg in consumer: + assert msg.value in {b'msg3', b'msg4'} + messages.add(msg.value) + if messages == {b'msg3', b'msg4'}: + break + assert messages == {b'msg3', b'msg4'} diff --git a/test/test_producer.py b/test/test_producer.py new file mode 100644 index 000000000..569df79f9 --- /dev/null +++ b/test/test_producer.py @@ -0,0 +1,23 @@ +from __future__ import absolute_import + +import gc +import platform +import threading + +import pytest + +from kafka import KafkaProducer + +@pytest.mark.skipif(platform.python_implementation() != 'CPython', + reason='Test relies on CPython-specific gc policies') +def test_kafka_producer_gc_cleanup(): + gc.collect() + threads = threading.active_count() + producer = KafkaProducer(api_version=(2, 1)) # set api_version explicitly to avoid auto-detection + assert threading.active_count() == threads + 1 + del(producer) + gc.collect() + assert threading.active_count() == threads + + + diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py index babff5617..42f980712 100644 --- a/test/test_record_accumulator.py +++ b/test/test_record_accumulator.py @@ -17,7 +17,7 @@ def test_producer_batch_producer_id(): 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) + batch.records.set_producer_state(123, 456, 789, False) assert batch.producer_id == 123 records.close() assert batch.producer_id == 123 diff --git a/test/test_sender.py b/test/test_sender.py index a1a775b59..ba20759a5 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -11,13 +11,14 @@ from kafka.vendor import six from kafka.client_async import KafkaClient +from kafka.cluster import ClusterMetadata import kafka.errors as Errors 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.transaction_state import TransactionState +from kafka.producer.transaction_manager import TransactionManager from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition @@ -42,6 +43,16 @@ def producer_batch(topic='foo', partition=0, magic=2): return batch +@pytest.fixture +def transaction_manager(): + return TransactionManager( + transactional_id=None, + transaction_timeout_ms=60000, + retry_backoff_ms=100, + api_version=(2, 1), + metadata=ClusterMetadata()) + + @pytest.mark.parametrize(("api_version", "produce_version"), [ ((2, 1), 7), ((0, 10, 0), 2), @@ -85,16 +96,16 @@ def test_complete_batch_success(sender): assert batch.produce_future.value == (0, 123, 456) -def test_complete_batch_transaction(sender): - sender._transaction_state = TransactionState() +def test_complete_batch_transaction(sender, transaction_manager): + sender._transaction_manager = transaction_manager batch = producer_batch() - assert sender._transaction_state.sequence_number(batch.topic_partition) == 0 - assert sender._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id + assert sender._transaction_manager.sequence_number(batch.topic_partition) == 0 + assert sender._transaction_manager.producer_id_and_epoch.producer_id == batch.producer_id # No error, base_offset 0 sender._complete_batch(batch, None, 0) assert batch.is_done - assert sender._transaction_state.sequence_number(batch.topic_partition) == batch.record_count + assert sender._transaction_manager.sequence_number(batch.topic_partition) == batch.record_count @pytest.mark.parametrize(("error", "refresh_metadata"), [ @@ -164,8 +175,8 @@ def test_complete_batch_retry(sender, accumulator, mocker, error, retry): assert isinstance(batch.produce_future.exception, error) -def test_complete_batch_producer_id_changed_no_retry(sender, accumulator, mocker): - sender._transaction_state = TransactionState() +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') @@ -175,21 +186,32 @@ def test_complete_batch_producer_id_changed_no_retry(sender, accumulator, mocker 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_state.producer_id_and_epoch.producer_id + assert batch.producer_id != sender._transaction_manager.producer_id_and_epoch.producer_id sender._complete_batch(batch, error, -1) assert batch.is_done assert isinstance(batch.produce_future.exception, error) -def test_fail_batch(sender, accumulator, mocker): - sender._transaction_state = TransactionState() - mocker.patch.object(TransactionState, 'reset_producer_id') +def test_fail_batch(sender, accumulator, transaction_manager, mocker): + sender._transaction_manager = transaction_manager batch = producer_batch() mocker.patch.object(batch, 'done') - assert sender._transaction_state.producer_id_and_epoch.producer_id == batch.producer_id + 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, log_start_offset=None) - sender._transaction_state.reset_producer_id.assert_called_once() + batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error, log_start_offset=None) + + +def test_out_of_order_sequence_number_reset_producer_id(sender, accumulator, transaction_manager, mocker): + sender._transaction_manager = transaction_manager + assert transaction_manager.transactional_id is None # this test is for idempotent producer only + mocker.patch.object(TransactionManager, 'reset_producer_id') + 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, log_start_offset=None) + sender._transaction_manager.reset_producer_id.assert_called_once() batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error, log_start_offset=None) From 7a0e9f61458b924d910b373ab51f4cefee5ed56e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 08:03:20 -0700 Subject: [PATCH 125/205] py2 test fixups --- test/integration/__init__.py | 0 test/test_sender.py | 5 ++++- 2 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 test/integration/__init__.py diff --git a/test/integration/__init__.py b/test/integration/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/test/test_sender.py b/test/test_sender.py index ba20759a5..ee057ff3a 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -6,7 +6,10 @@ import time import pytest -from unittest.mock import call +try: + from unittest.mock import call +except ImportError: + from mock import call from kafka.vendor import six From e2c3b80fd03655741cf115a18049624686ccc862 Mon Sep 17 00:00:00 2001 From: Emmanuel Ferdman Date: Wed, 16 Apr 2025 20:25:53 +0300 Subject: [PATCH 126/205] Resolve datetime deprecation warnings (#2589) --- test/sasl/test_msk.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/test/sasl/test_msk.py b/test/sasl/test_msk.py index 297ca84ce..e9f1325f3 100644 --- a/test/sasl/test_msk.py +++ b/test/sasl/test_msk.py @@ -1,5 +1,6 @@ import datetime import json +import sys from kafka.sasl.msk import AwsMskIamClient @@ -10,7 +11,10 @@ def client_factory(token=None): - now = datetime.datetime.utcfromtimestamp(1629321911) + if sys.version_info >= (3, 3): + now = datetime.datetime.fromtimestamp(1629321911, datetime.timezone.utc) + else: + now = datetime.datetime.utcfromtimestamp(1629321911) with mock.patch('kafka.sasl.msk.datetime') as mock_dt: mock_dt.datetime.utcnow = mock.Mock(return_value=now) return AwsMskIamClient( From 369478aa4a8a73f56390473abf1f1cd3341c08ef Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 14:23:11 -0700 Subject: [PATCH 127/205] Remove old/unused errors; reorder; KafkaTimeout -> retriable --- kafka/errors.py | 124 ++++++++++++++++++++---------------------------- 1 file changed, 52 insertions(+), 72 deletions(-) diff --git a/kafka/errors.py b/kafka/errors.py index ea17d6ae2..900dcd5e2 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -16,21 +16,44 @@ def __str__(self): super(KafkaError, self).__str__()) +class Cancelled(KafkaError): + retriable = True + + +class CommitFailedError(KafkaError): + def __init__(self, *args, **kwargs): + super(CommitFailedError, self).__init__( + """Commit cannot be completed since the group has already + rebalanced and assigned the partitions to another member. + This means that the time between subsequent calls to poll() + was longer than the configured max_poll_interval_ms, which + typically implies that the poll loop is spending too much + time message processing. You can address this either by + increasing the rebalance timeout with max_poll_interval_ms, + or by reducing the maximum size of batches returned in poll() + with max_poll_records. + """, *args, **kwargs) + + +class IllegalArgumentError(KafkaError): + pass + + class IllegalStateError(KafkaError): pass -class IllegalArgumentError(KafkaError): +class IncompatibleBrokerVersion(KafkaError): pass -class NoBrokersAvailable(KafkaError): - retriable = True - invalid_metadata = True +class KafkaConfigurationError(KafkaError): + pass -class NodeNotReadyError(KafkaError): +class KafkaConnectionError(KafkaError): retriable = True + invalid_metadata = True class KafkaProtocolError(KafkaError): @@ -41,47 +64,41 @@ class CorrelationIdError(KafkaProtocolError): retriable = True -class Cancelled(KafkaError): +class KafkaTimeoutError(KafkaError): retriable = True -class TooManyInFlightRequests(KafkaError): +class MetadataEmptyBrokerList(KafkaError): retriable = True -class StaleMetadata(KafkaError): +class NoBrokersAvailable(KafkaError): retriable = True invalid_metadata = True -class MetadataEmptyBrokerList(KafkaError): +class NodeNotReadyError(KafkaError): retriable = True -class UnrecognizedBrokerVersion(KafkaError): +class QuotaViolationError(KafkaError): pass -class IncompatibleBrokerVersion(KafkaError): - pass +class StaleMetadata(KafkaError): + retriable = True + invalid_metadata = True -class CommitFailedError(KafkaError): - def __init__(self, *args, **kwargs): - super(CommitFailedError, self).__init__( - """Commit cannot be completed since the group has already - rebalanced and assigned the partitions to another member. - This means that the time between subsequent calls to poll() - was longer than the configured max_poll_interval_ms, which - typically implies that the poll loop is spending too much - time message processing. You can address this either by - increasing the rebalance timeout with max_poll_interval_ms, - or by reducing the maximum size of batches returned in poll() - with max_poll_records. - """, *args, **kwargs) +class TooManyInFlightRequests(KafkaError): + retriable = True + +class UnrecognizedBrokerVersion(KafkaError): + pass -class AuthenticationMethodNotSupported(KafkaError): + +class UnsupportedCodecError(KafkaError): pass @@ -97,6 +114,10 @@ def __str__(self): super(BrokerResponseError, self).__str__()) +class AuthorizationError(BrokerResponseError): + pass + + class NoError(BrokerResponseError): errno = 0 message = 'NO_ERROR' @@ -332,21 +353,21 @@ class InvalidCommitOffsetSizeError(BrokerResponseError): ' because of oversize metadata.') -class TopicAuthorizationFailedError(BrokerResponseError): +class TopicAuthorizationFailedError(AuthorizationError): errno = 29 message = 'TOPIC_AUTHORIZATION_FAILED' description = ('Returned by the broker when the client is not authorized to' ' access the requested topic.') -class GroupAuthorizationFailedError(BrokerResponseError): +class GroupAuthorizationFailedError(AuthorizationError): errno = 30 message = 'GROUP_AUTHORIZATION_FAILED' description = ('Returned by the broker when the client is not authorized to' ' access a particular groupId.') -class ClusterAuthorizationFailedError(BrokerResponseError): +class ClusterAuthorizationFailedError(AuthorizationError): errno = 31 message = 'CLUSTER_AUTHORIZATION_FAILED' description = ('Returned by the broker when the client is not authorized to' @@ -493,7 +514,7 @@ class TransactionCoordinatorFencedError(BrokerResponseError): retriable = False -class TransactionalIdAuthorizationFailedError(BrokerResponseError): +class TransactionalIdAuthorizationFailedError(AuthorizationError): errno = 53 message = 'TRANSACTIONAL_ID_AUTHORIZATION_FAILED' description = 'Transactional Id authorization failed.' @@ -578,7 +599,7 @@ class DelegationTokenRequestNotAllowedError(BrokerResponseError): retriable = False -class DelegationTokenAuthorizationFailedError(BrokerResponseError): +class DelegationTokenAuthorizationFailedError(AuthorizationError): errno = 65 message = 'DELEGATION_TOKEN_AUTHORIZATION_FAILED' description = 'Delegation Token authorization failed.' @@ -1027,47 +1048,6 @@ class VoterNotFoundError(BrokerResponseError): retriable = False -class KafkaUnavailableError(KafkaError): - pass - - -class KafkaTimeoutError(KafkaError): - pass - - -class FailedPayloadsError(KafkaError): - def __init__(self, payload, *args): - super(FailedPayloadsError, self).__init__(*args) - self.payload = payload - - -class KafkaConnectionError(KafkaError): - retriable = True - invalid_metadata = True - - -class ProtocolError(KafkaError): - pass - - -class UnsupportedCodecError(KafkaError): - pass - - -class KafkaConfigurationError(KafkaError): - pass - - -class QuotaViolationError(KafkaError): - pass - - -class AsyncProducerQueueFull(KafkaError): - def __init__(self, failed_msgs, *args): - super(AsyncProducerQueueFull, self).__init__(*args) - self.failed_msgs = failed_msgs - - def _iter_broker_errors(): for name, obj in inspect.getmembers(sys.modules[__name__]): if inspect.isclass(obj) and issubclass(obj, BrokerResponseError) and obj != BrokerResponseError: From 315f9d848f0263b473bd0b12b4a866b87e55c999 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 16:41:16 -0700 Subject: [PATCH 128/205] Fixup retry/sleep after successful init producer id in idempotent producer --- kafka/producer/sender.py | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 707d46bf3..9c845cfca 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -319,6 +319,7 @@ def _maybe_wait_for_producer_id(self): error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: self._transaction_manager.set_producer_id_and_epoch(ProducerIdAndEpoch(response.producer_id, response.producer_epoch)) + break elif getattr(error_type, 'retriable', False): log.debug("Retriable error from InitProducerId response: %s", error_type.__name__) if getattr(error_type, 'invalid_metadata', False): From a87f92203e24846a926b4967ac1adfac949cb7e3 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 16:53:53 -0700 Subject: [PATCH 129/205] KIP-98: Add offsets support to transactional KafkaProducer (#2590) --- kafka/producer/kafka.py | 30 +++ kafka/producer/transaction_manager.py | 198 ++++++++++++++++-- kafka/protocol/txn_offset_commit.py | 78 +++++++ test/integration/test_producer_integration.py | 36 +++- 4 files changed, 324 insertions(+), 18 deletions(-) create mode 100644 kafka/protocol/txn_offset_commit.py diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 1468cec55..d3d9699bd 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -686,6 +686,36 @@ def begin_transaction(self): raise Errors.IllegalStateError("Cannot use transactional methods without enabling transactions") self._transaction_manager.begin_transaction() + def send_offsets_to_transaction(self, offsets, consumer_group_id): + """ + Sends a list of consumed offsets to the consumer group coordinator, and also marks + those offsets as part of the current transaction. These offsets will be considered + consumed only if the transaction is committed successfully. + + This method should be used when you need to batch consumed and produced messages + together, typically in a consume-transform-produce pattern. + + Arguments: + offsets ({TopicPartition: OffsetAndMetadata}): map of topic-partition -> offsets to commit + as part of current transaction. + consumer_group_id (str): Name of consumer group for offsets commit. + + Raises: + IllegalStateError: if no transactional_id, or transaction has not been started. + ProducerFencedError: fatal error indicating another producer with the same transactional_id is active. + UnsupportedVersionError: fatal error indicating the broker does not support transactions (i.e. if < 0.11). + UnsupportedForMessageFormatError: fatal error indicating the message format used for the offsets + topic on the broker does not support transactions. + AuthorizationError: fatal error indicating that the configured transactional_id is not authorized. + KafkaErro:r if the producer has encountered a previous fatal or abortable error, or for any + other unexpected error + """ + if not self._transaction_manager: + raise Errors.IllegalStateError("Cannot use transactional methods without enabling transactions") + result = self._transaction_manager.send_offsets_to_transaction(offsets, consumer_group_id) + self._sender.wakeup() + result.wait() + def commit_transaction(self): """ Commits the ongoing transaction. diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index f5111c780..7191fb0c7 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -16,10 +16,12 @@ 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 from kafka.protocol.end_txn import EndTxnRequest from kafka.protocol.find_coordinator import FindCoordinatorRequest from kafka.protocol.init_producer_id import InitProducerIdRequest +from kafka.protocol.txn_offset_commit import TxnOffsetCommitRequest from kafka.structs import TopicPartition @@ -115,6 +117,7 @@ def __init__(self, transactional_id=None, transaction_timeout_ms=0, retry_backof self._new_partitions_in_transaction = set() self._pending_partitions_in_transaction = set() self._partitions_in_transaction = set() + self._pending_txn_offset_commits = dict() self._current_state = TransactionState.UNINITIALIZED self._last_error = None @@ -138,7 +141,7 @@ def initialize_transactions(self): self._transition_to(TransactionState.INITIALIZING) self.set_producer_id_and_epoch(ProducerIdAndEpoch(NO_PRODUCER_ID, NO_PRODUCER_EPOCH)) self._sequence_numbers.clear() - handler = InitProducerIdHandler(self, self.transactional_id, self.transaction_timeout_ms) + handler = InitProducerIdHandler(self, self.transaction_timeout_ms) self._enqueue_request(handler) return handler.result @@ -169,10 +172,22 @@ def begin_abort(self): def _begin_completing_transaction(self, committed): if self._new_partitions_in_transaction: self._enqueue_request(self._add_partitions_to_transaction_handler()) - handler = EndTxnHandler(self, self.transactional_id, self.producer_id_and_epoch.producer_id, self.producer_id_and_epoch.epoch, committed) + handler = EndTxnHandler(self, committed) self._enqueue_request(handler) return handler.result + def send_offsets_to_transaction(self, offsets, consumer_group_id): + with self._lock: + self._ensure_transactional() + self._maybe_fail_with_error() + if self._current_state != TransactionState.IN_TRANSACTION: + raise Errors.KafkaError("Cannot send offsets to transaction because the producer is not in an active transaction") + + log.debug("Begin adding offsets %s for consumer group %s to transaction", offsets, consumer_group_id) + handler = AddOffsetsToTxnHandler(self, consumer_group_id, offsets) + self._enqueue_request(handler) + return handler.result + def maybe_add_partition_to_transaction(self, topic_partition): with self._lock: self._fail_if_not_ready_for_send() @@ -389,6 +404,10 @@ def _test_transaction_contains_partition(self, tp): with self._lock: return tp in self._partitions_in_transaction + # visible for testing + def _test_has_pending_offset_commits(self): + return bool(self._pending_txn_offset_commits) + # visible for testing def _test_has_ongoing_transaction(self): with self._lock: @@ -473,7 +492,7 @@ def _add_partitions_to_transaction_handler(self): with self._lock: self._pending_partitions_in_transaction.update(self._new_partitions_in_transaction) self._new_partitions_in_transaction.clear() - return AddPartitionsToTxnHandler(self, self.transactional_id, self.producer_id_and_epoch.producer_id, self.producer_id_and_epoch.epoch, self._pending_partitions_in_transaction) + return AddPartitionsToTxnHandler(self, self._pending_partitions_in_transaction) class TransactionalRequestResult(object): @@ -518,6 +537,18 @@ def __init__(self, transaction_manager, result=None): self._result = result or TransactionalRequestResult() self._is_retry = False + @property + def transactional_id(self): + return self.transaction_manager.transactional_id + + @property + def producer_id(self): + return self.transaction_manager.producer_id_and_epoch.producer_id + + @property + 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._result.done(error=exc) @@ -585,16 +616,15 @@ def priority(self): class InitProducerIdHandler(TxnRequestHandler): - def __init__(self, transaction_manager, transactional_id, transaction_timeout_ms): + def __init__(self, transaction_manager, transaction_timeout_ms): super(InitProducerIdHandler, self).__init__(transaction_manager) - self.transactional_id = transactional_id if transaction_manager._api_version >= (2, 0): version = 1 else: version = 0 self.request = InitProducerIdRequest[version]( - transactional_id=transactional_id, + transactional_id=self.transactional_id, transaction_timeout_ms=transaction_timeout_ms) @property @@ -619,10 +649,9 @@ def handle_response(self, response): self.fatal_error(Errors.KafkaError("Unexpected error in InitProducerIdResponse: %s" % (error()))) class AddPartitionsToTxnHandler(TxnRequestHandler): - def __init__(self, transaction_manager, transactional_id, producer_id, producer_epoch, topic_partitions): + def __init__(self, transaction_manager, topic_partitions): super(AddPartitionsToTxnHandler, self).__init__(transaction_manager) - self.transactional_id = transactional_id if transaction_manager._api_version >= (2, 7): version = 2 elif transaction_manager._api_version >= (2, 0): @@ -633,9 +662,9 @@ def __init__(self, transaction_manager, transactional_id, producer_id, producer_ for tp in topic_partitions: topic_data[tp.topic].append(tp.partition) self.request = AddPartitionsToTxnRequest[version]( - transactional_id=transactional_id, - producer_id=producer_id, - producer_epoch=producer_epoch, + transactional_id=self.transactional_id, + producer_id=self.producer_id, + producer_epoch=self.producer_epoch, topics=list(topic_data.items())) @property @@ -771,10 +800,9 @@ def handle_response(self, response): class EndTxnHandler(TxnRequestHandler): - def __init__(self, transaction_manager, transactional_id, producer_id, producer_epoch, committed): + def __init__(self, transaction_manager, committed): super(EndTxnHandler, self).__init__(transaction_manager) - self.transactional_id = transactional_id if self.transaction_manager._api_version >= (2, 7): version = 2 elif self.transaction_manager._api_version >= (2, 0): @@ -782,9 +810,9 @@ def __init__(self, transaction_manager, transactional_id, producer_id, producer_ else: version = 0 self.request = EndTxnRequest[version]( - transactional_id=transactional_id, - producer_id=producer_id, - producer_epoch=producer_epoch, + transactional_id=self.transactional_id, + producer_id=self.producer_id, + producer_epoch=self.producer_epoch, committed=committed) @property @@ -810,3 +838,141 @@ def handle_response(self, response): self.fatal_error(error()) else: self.fatal_error(Errors.KafkaError("Unhandled error in EndTxnResponse: %s" % (error()))) + + +class AddOffsetsToTxnHandler(TxnRequestHandler): + def __init__(self, transaction_manager, consumer_group_id, offsets): + super(AddOffsetsToTxnHandler, self).__init__(transaction_manager) + + self.consumer_group_id = consumer_group_id + self.offsets = offsets + if self.transaction_manager._api_version >= (2, 7): + version = 2 + elif self.transaction_manager._api_version >= (2, 0): + version = 1 + else: + version = 0 + self.request = AddOffsetsToTxnRequest[version]( + transactional_id=self.transactional_id, + producer_id=self.producer_id, + producer_epoch=self.producer_epoch, + group_id=consumer_group_id) + + @property + def priority(self): + return Priority.ADD_PARTITIONS_OR_OFFSETS + + def handle_response(self, response): + error = Errors.for_code(response.error_code) + + if error is Errors.NoError: + 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): + 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) + self.transaction_manager._enqueue_request(handler) + self.transaction_manager._transaction_started = True + elif error in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): + self.transaction_manager._lookup_coordinator('transaction', self.transactional_id) + self.reenqueue() + elif error in (Errors.CoordinatorLoadInProgressError, Errors.ConcurrentTransactionsError): + self.reenqueue() + elif error is Errors.InvalidProducerEpochError: + self.fatal_error(error()) + elif error is Errors.TransactionalIdAuthorizationFailedError: + self.fatal_error(error()) + elif error is Errors.GroupAuthorizationFailedError: + self.abortable_error(Errors.GroupAuthorizationError(self.consumer_group_id)) + else: + self.fatal_error(Errors.KafkaError("Unexpected error in AddOffsetsToTxnResponse: %s" % (error()))) + + +class TxnOffsetCommitHandler(TxnRequestHandler): + def __init__(self, transaction_manager, consumer_group_id, offsets, result): + super(TxnOffsetCommitHandler, self).__init__(transaction_manager, result=result) + + self.consumer_group_id = consumer_group_id + self.offsets = offsets + self.request = self._build_request() + + def _build_request(self): + if self.transaction_manager._api_version >= (2, 1): + version = 2 + elif self.transaction_manager._api_version >= (2, 0): + version = 1 + else: + version = 0 + + topic_data = collections.defaultdict(list) + for tp, offset in six.iteritems(self.offsets): + if version >= 2: + partition_data = (tp.partition, offset.offset, offset.leader_epoch, offset.metadata) + else: + partition_data = (tp.partition, offset.offset, offset.metadata) + topic_data[tp.topic].append(partition_data) + + return TxnOffsetCommitRequest[version]( + transactional_id=self.transactional_id, + group_id=self.consumer_group_id, + producer_id=self.producer_id, + producer_epoch=self.producer_epoch, + topics=list(topic_data.items())) + + @property + def priority(self): + return Priority.ADD_PARTITIONS_OR_OFFSETS + + @property + def coordinator_type(self): + return 'group' + + @property + def coordinator_key(self): + return self.consumer_group_id + + def handle_response(self, response): + lookup_coordinator = False + retriable_failure = False + + errors = {TopicPartition(topic, partition): Errors.for_code(error_code) + for topic, partition_data in response.topics + for partition, error_code in partition_data} + + for tp, error in six.iteritems(errors): + if error is Errors.NoError: + log.debug("Successfully added offsets for %s from consumer group %s to transaction.", + tp, self.consumer_group_id) + del self.transaction_manager._pending_txn_offset_commits[tp] + elif error in (errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError, Errors.RequestTimedOutError): + retriable_failure = True + lookup_coordinator = True + elif error is Errors.UnknownTopicOrPartitionError: + retriable_failure = True + elif error is Errors.GroupAuthorizationFailedError: + self.abortable_error(Errors.GroupAuthorizationError(self.consumer_group_id)) + return + elif error in (Errors.TransactionalIdAuthorizationFailedError, + Errors.InvalidProducerEpochError, + Errors.UnsupportedForMessageFormatError): + self.fatal_error(error()) + return + else: + self.fatal_error(Errors.KafkaError("Unexpected error in TxnOffsetCommitResponse: %s" % (error()))) + return + + if lookup_coordinator: + self.transaction_manager._lookup_coordinator('group', self.consumer_group_id) + + if not retriable_failure: + # all attempted partitions were either successful, or there was a fatal failure. + # either way, we are not retrying, so complete the request. + self.result.done() + + # retry the commits which failed with a retriable error. + elif self.transaction_manager._pending_txn_offset_commits: + self.offsets = self.transaction_manager._pending_txn_offset_commits + self.request = self._build_request() + self.reenqueue() diff --git a/kafka/protocol/txn_offset_commit.py b/kafka/protocol/txn_offset_commit.py new file mode 100644 index 000000000..df1b1bd1e --- /dev/null +++ b/kafka/protocol/txn_offset_commit.py @@ -0,0 +1,78 @@ +from __future__ import absolute_import + +from kafka.protocol.api import Request, Response +from kafka.protocol.types import Array, Int16, Int32, Int64, Schema, String + + +class TxnOffsetCommitResponse_v0(Response): + API_KEY = 28 + API_VERSION = 0 + SCHEMA = Schema( + ('throttle_time_ms', Int32), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('error_code', Int16)))))) + + +class TxnOffsetCommitResponse_v1(Response): + API_KEY = 28 + API_VERSION = 1 + SCHEMA = TxnOffsetCommitResponse_v0.SCHEMA + + +class TxnOffsetCommitResponse_v2(Response): + API_KEY = 28 + API_VERSION = 2 + SCHEMA = TxnOffsetCommitResponse_v1.SCHEMA + + +class TxnOffsetCommitRequest_v0(Request): + API_KEY = 28 + API_VERSION = 0 + RESPONSE_TYPE = TxnOffsetCommitResponse_v0 + SCHEMA = Schema( + ('transactional_id', String('utf-8')), + ('group_id', String('utf-8')), + ('producer_id', Int64), + ('producer_epoch', Int16), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('metadata', String('utf-8'))))))) + + +class TxnOffsetCommitRequest_v1(Request): + API_KEY = 28 + API_VERSION = 1 + RESPONSE_TYPE = TxnOffsetCommitResponse_v1 + SCHEMA = TxnOffsetCommitRequest_v0.SCHEMA + + +class TxnOffsetCommitRequest_v2(Request): + API_KEY = 28 + API_VERSION = 2 + RESPONSE_TYPE = TxnOffsetCommitResponse_v2 + SCHEMA = Schema( + ('transactional_id', String('utf-8')), + ('group_id', String('utf-8')), + ('producer_id', Int64), + ('producer_epoch', Int16), + ('topics', Array( + ('topic', String('utf-8')), + ('partitions', Array( + ('partition', Int32), + ('offset', Int64), + ('leader_epoch', Int32), + ('metadata', String('utf-8'))))))) + + +TxnOffsetCommitRequest = [ + TxnOffsetCommitRequest_v0, TxnOffsetCommitRequest_v1, TxnOffsetCommitRequest_v2, +] +TxnOffsetCommitResponse = [ + TxnOffsetCommitResponse_v0, TxnOffsetCommitResponse_v1, TxnOffsetCommitResponse_v2, +] diff --git a/test/integration/test_producer_integration.py b/test/integration/test_producer_integration.py index 0739d8eba..037a82834 100644 --- a/test/integration/test_producer_integration.py +++ b/test/integration/test_producer_integration.py @@ -6,7 +6,7 @@ import pytest -from kafka import KafkaConsumer, KafkaProducer, TopicPartition +from kafka import KafkaAdminClient, KafkaConsumer, KafkaProducer, TopicPartition, OffsetAndMetadata from test.testutil import env_kafka_version, random_string, maybe_skip_unsupported_compression @@ -28,6 +28,15 @@ def consumer_factory(**kwargs): consumer.close(timeout_ms=100) +@contextmanager +def admin_factory(**kwargs): + admin = KafkaAdminClient(**kwargs) + try: + yield admin + finally: + admin.close() + + @pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set") @pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd']) def test_end_to_end(kafka_broker, compression): @@ -144,7 +153,7 @@ def test_idempotent_producer(kafka_broker): @pytest.mark.skipif(env_kafka_version() < (0, 11), reason="Idempotent producer requires broker >=0.11") -def test_transactional_producer(kafka_broker): +def test_transactional_producer_messages(kafka_broker): connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) with producer_factory(bootstrap_servers=connect_str, transactional_id='testing') as producer: producer.init_transactions() @@ -173,3 +182,26 @@ def test_transactional_producer(kafka_broker): if messages == {b'msg3', b'msg4'}: break assert messages == {b'msg3', b'msg4'} + + +@pytest.mark.skipif(env_kafka_version() < (0, 11), reason="Idempotent producer requires broker >=0.11") +def test_transactional_producer_offsets(kafka_broker): + connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)]) + # Setting leader_epoch only supported in 2.1+ + if env_kafka_version() >= (2, 1): + leader_epoch = 0 + else: + leader_epoch = -1 + offsets = {TopicPartition('transactional_test_topic', 0): OffsetAndMetadata(0, 'metadata', leader_epoch)} + with producer_factory(bootstrap_servers=connect_str, transactional_id='testing') as producer: + producer.init_transactions() + producer.begin_transaction() + producer.send_offsets_to_transaction(offsets, 'txn-test-group') + producer.commit_transaction() + + producer.begin_transaction() + producer.send_offsets_to_transaction({TopicPartition('transactional_test_topic', 1): OffsetAndMetadata(1, 'bad', 1)}, 'txn-test-group') + producer.abort_transaction() + + with admin_factory(bootstrap_servers=connect_str) as admin: + assert admin.list_consumer_group_offsets('txn-test-group') == offsets From c5cbe840ec0ff0f32b6ece08a1297ce29eca2e5f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 17:58:44 -0700 Subject: [PATCH 130/205] fixup reset_producer_id is_transactional() check --- kafka/producer/transaction_manager.py | 5 ++++- test/test_producer.py | 16 ++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index 7191fb0c7..e2dd4e020 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -47,6 +47,9 @@ def is_valid(self): def match(self, batch): return self.producer_id == batch.producer_id and self.epoch == batch.producer_epoch + def __eq__(self, other): + return isinstance(other, ProducerIdAndEpoch) and self.producer_id == other.producer_id and self.epoch == other.epoch + def __str__(self): return "ProducerIdAndEpoch(producer_id={}, epoch={})".format(self.producer_id, self.epoch) @@ -304,7 +307,7 @@ def reset_producer_id(self): it's best to return the produce error to the user and let them abort the transaction and close the producer explicitly. """ with self._lock: - if self.is_transactional: + if self.is_transactional(): raise Errors.IllegalStateError( "Cannot reset producer state for a transactional producer." " You must either abort the ongoing transaction or" diff --git a/test/test_producer.py b/test/test_producer.py index 569df79f9..8a8c48324 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -7,6 +7,9 @@ import pytest from kafka import KafkaProducer +from kafka.cluster import ClusterMetadata +from kafka.producer.transaction_manager import TransactionManager, ProducerIdAndEpoch + @pytest.mark.skipif(platform.python_implementation() != 'CPython', reason='Test relies on CPython-specific gc policies') @@ -20,4 +23,17 @@ def test_kafka_producer_gc_cleanup(): assert threading.active_count() == threads +def test_idempotent_producer_reset_producer_id(): + transaction_manager = TransactionManager( + transactional_id=None, + transaction_timeout_ms=1000, + retry_backoff_ms=100, + api_version=(0, 11), + metadata=ClusterMetadata(), + ) + test_producer_id_and_epoch = ProducerIdAndEpoch(123, 456) + transaction_manager.set_producer_id_and_epoch(test_producer_id_and_epoch) + assert transaction_manager.producer_id_and_epoch == test_producer_id_and_epoch + transaction_manager.reset_producer_id() + assert transaction_manager.producer_id_and_epoch == ProducerIdAndEpoch(-1, -1) From 99c08e66fc363538ff4b8c87bd2702612be62990 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 18:26:45 -0700 Subject: [PATCH 131/205] Prefix producer logs w/ client id and transactional id (#2591) --- kafka/producer/kafka.py | 50 +++++++++++++------------ kafka/producer/sender.py | 80 +++++++++++++++++++++------------------- 2 files changed, 69 insertions(+), 61 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index d3d9699bd..1535dcedb 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -380,7 +380,6 @@ class KafkaProducer(object): } def __init__(self, **configs): - log.debug("Starting the Kafka producer") # trace self.config = copy.copy(self.DEFAULT_CONFIG) user_provided_configs = set(configs.keys()) for key in self.config: @@ -409,8 +408,10 @@ def __init__(self, **configs): self.config['api_version'] = None else: self.config['api_version'] = tuple(map(int, deprecated.split('.'))) - log.warning('use api_version=%s [tuple] -- "%s" as str is deprecated', - str(self.config['api_version']), deprecated) + log.warning('%s: use api_version=%s [tuple] -- "%s" as str is deprecated', + self, str(self.config['api_version']), deprecated) + + log.debug("%s: Starting Kafka producer", self) # Configure metrics if self.config['metrics_enabled']: @@ -466,18 +467,18 @@ def __init__(self, **configs): metadata=self._metadata, ) if self._transaction_manager.is_transactional(): - log.info("Instantiated a transactional producer.") + log.info("%s: Instantiated a transactional producer.", self) else: - log.info("Instantiated an idempotent producer.") + log.info("%s: Instantiated an idempotent producer.", self) if 'retries' not in user_provided_configs: - log.info("Overriding the default 'retries' config to 3 since the idempotent producer is enabled.") + log.info("%s: Overriding the default 'retries' config to 3 since the idempotent producer is enabled.", self) self.config['retries'] = 3 elif self.config['retries'] == 0: raise Errors.KafkaConfigurationError("Must set 'retries' to non-zero when using the idempotent producer.") if 'max_in_flight_requests_per_connection' not in user_provided_configs: - log.info("Overriding the default 'max_in_flight_requests_per_connection' to 1 since idempontence is enabled.") + log.info("%s: Overriding the default 'max_in_flight_requests_per_connection' to 1 since idempontence is enabled.", self) self.config['max_in_flight_requests_per_connection'] = 1 elif self.config['max_in_flight_requests_per_connection'] != 1: raise Errors.KafkaConfigurationError("Must set 'max_in_flight_requests_per_connection' to 1 in order" @@ -485,7 +486,7 @@ def __init__(self, **configs): " Otherwise we cannot guarantee idempotence.") if 'acks' not in user_provided_configs: - log.info("Overriding the default 'acks' config to 'all' since idempotence is enabled") + log.info("%s: Overriding the default 'acks' config to 'all' since idempotence is enabled", self) self.config['acks'] = -1 elif self.config['acks'] != -1: raise Errors.KafkaConfigurationError("Must set 'acks' config to 'all' in order to use the idempotent" @@ -509,7 +510,7 @@ def __init__(self, **configs): self._cleanup = self._cleanup_factory() atexit.register(self._cleanup) - log.debug("Kafka producer started") + log.debug("%s: Kafka producer started", self) def bootstrap_connected(self): """Return True if the bootstrap is connected.""" @@ -564,7 +565,7 @@ def __getattr__(self, name): self._unregister_cleanup() if not hasattr(self, '_closed') or self._closed: - log.info('Kafka producer closed') + log.info('%s: Kafka producer closed', self) return if timeout is None: # threading.TIMEOUT_MAX is available in Python3.3+ @@ -574,16 +575,16 @@ def __getattr__(self, name): else: assert timeout >= 0 - log.info("Closing the Kafka producer with %s secs timeout.", timeout) + log.info("%s: Closing the Kafka producer with %s secs timeout.", self, timeout) self.flush(timeout) invoked_from_callback = bool(threading.current_thread() is self._sender) if timeout > 0: if invoked_from_callback: - log.warning("Overriding close timeout %s secs to 0 in order to" + log.warning("%s: Overriding close timeout %s secs to 0 in order to" " prevent useless blocking due to self-join. This" " means you have incorrectly invoked close with a" " non-zero timeout from the producer call-back.", - timeout) + self, timeout) else: # Try to close gracefully. if self._sender is not None: @@ -591,9 +592,9 @@ def __getattr__(self, name): self._sender.join(timeout) if self._sender is not None and self._sender.is_alive(): - log.info("Proceeding to force close the producer since pending" + log.info("%s: Proceeding to force close the producer since pending" " requests could not be completed within timeout %s.", - timeout) + self, timeout) self._sender.force_close() if self._metrics: @@ -607,7 +608,7 @@ def __getattr__(self, name): except AttributeError: pass self._closed = True - log.debug("The Kafka producer has closed.") + log.debug("%s: The Kafka producer has closed.", self) def partitions_for(self, topic): """Returns set of all known partitions for the topic.""" @@ -816,7 +817,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest self._ensure_valid_record_size(message_size) tp = TopicPartition(topic, partition) - log.debug("Sending (key=%r value=%r headers=%r) to %s", key, value, headers, tp) + log.debug("%s: Sending (key=%r value=%r headers=%r) to %s", self, key, value, headers, tp) if self._transaction_manager and self._transaction_manager.is_transactional(): self._transaction_manager.maybe_add_partition_to_transaction(tp) @@ -825,8 +826,8 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest key_bytes, value_bytes, headers) future, batch_is_full, new_batch_created = result if batch_is_full or new_batch_created: - log.debug("Waking up the sender since %s is either full or" - " getting a new batch", tp) + log.debug("%s: Waking up the sender since %s is either full or" + " getting a new batch", self, tp) self._sender.wakeup() return future @@ -834,7 +835,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest # for API exceptions return them in the future, # for other exceptions raise directly except Errors.BrokerResponseError as e: - log.error("Exception occurred during message send: %s", e) + log.error("%s: Exception occurred during message send: %s", self, e) return FutureRecordMetadata( FutureProduceResult(TopicPartition(topic, partition)), -1, None, None, @@ -865,7 +866,7 @@ def flush(self, timeout=None): KafkaTimeoutError: failure to flush buffered records within the provided timeout """ - log.debug("Flushing accumulated records in producer.") # trace + log.debug("%s: Flushing accumulated records in producer.", self) self._accumulator.begin_flush() self._sender.wakeup() self._accumulator.await_flush_completion(timeout=timeout) @@ -911,7 +912,7 @@ def _wait_on_metadata(self, topic, max_wait): if not metadata_event: metadata_event = threading.Event() - log.debug("Requesting metadata update for topic %s", topic) + log.debug("%s: Requesting metadata update for topic %s", self, topic) metadata_event.clear() future = self._metadata.request_update() @@ -925,7 +926,7 @@ def _wait_on_metadata(self, topic, max_wait): raise Errors.TopicAuthorizationFailedError(set([topic])) else: elapsed = time.time() - begin - log.debug("_wait_on_metadata woke after %s secs.", elapsed) + log.debug("%s: _wait_on_metadata woke after %s secs.", self, elapsed) def _serialize(self, f, topic, data): if not f: @@ -972,3 +973,6 @@ def metrics(self, raw=False): metrics[k.group][k.name] = {} metrics[k.group][k.name] = v.value() return metrics + + def __str__(self): + return "" % (self.config['client_id'], self.config['transactional_id']) diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 9c845cfca..3637be416 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -62,17 +62,17 @@ def __init__(self, client, metadata, accumulator, **configs): def run(self): """The main run loop for the sender thread.""" - log.debug("Starting Kafka producer I/O thread.") + log.debug("%s: Starting Kafka producer I/O thread.", self) # main loop, runs until close is called while self._running: try: self.run_once() except Exception: - log.exception("Uncaught error in kafka producer I/O thread") + log.exception("%s: Uncaught error in kafka producer I/O thread", self) - log.debug("Beginning shutdown of Kafka producer I/O thread, sending" - " remaining records.") + log.debug("%s: Beginning shutdown of Kafka producer I/O thread, sending" + " remaining records.", self) # okay we stopped accepting requests but there may still be # requests in the accumulator or waiting for acknowledgment, @@ -83,7 +83,7 @@ def run(self): try: self.run_once() except Exception: - log.exception("Uncaught error in kafka producer I/O thread") + log.exception("%s: Uncaught error in kafka producer I/O thread", self) if self._force_close: # We need to fail all the incomplete batches and wake up the @@ -93,9 +93,9 @@ def run(self): try: self._client.close() except Exception: - log.exception("Failed to close network client") + log.exception("%s: Failed to close network client", self) - log.debug("Shutdown of Kafka producer I/O thread has completed.") + log.debug("%s: Shutdown of Kafka producer I/O thread has completed.", self) def run_once(self): """Run a single iteration of sending.""" @@ -125,7 +125,7 @@ def run_once(self): except Errors.SaslAuthenticationFailedError as e: # This is already logged as error, but propagated here to perform any clean ups. - log.debug("Authentication exception while processing transactional request: %s", e) + log.debug("%s: Authentication exception while processing transactional request: %s", self, e) self._transaction_manager.authentication_failed(e) poll_timeout_ms = self._send_producer_data() @@ -139,7 +139,7 @@ def _send_producer_data(self): # if there are any partitions whose leaders are not known yet, force # metadata update if unknown_leaders_exist: - log.debug('Unknown leaders exist, requesting metadata update') + log.debug('%s: Unknown leaders exist, requesting metadata update', self) self._metadata.request_update() # remove any nodes we aren't ready to send to @@ -147,7 +147,7 @@ def _send_producer_data(self): for node in list(ready_nodes): if not self._client.is_ready(node): node_delay_ms = self._client.connection_delay(node) - log.debug('Node %s not ready; delaying produce of accumulated batch (%f ms)', node, node_delay_ms) + log.debug('%s: Node %s not ready; delaying produce of accumulated batch (%f ms)', self, node, node_delay_ms) self._client.maybe_connect(node, wakeup=False) ready_nodes.remove(node) not_ready_timeout_ms = min(not_ready_timeout_ms, node_delay_ms) @@ -166,7 +166,7 @@ def _send_producer_data(self): self.config['request_timeout_ms'], self._metadata) if expired_batches: - log.debug("Expired %s batches in accumulator", len(expired_batches)) + log.debug("%s: Expired %s batches in accumulator", self, len(expired_batches)) # Reset the producer_id if an expired batch has previously been sent to the broker. # See the documentation of `TransactionState.reset_producer_id` to understand why @@ -200,8 +200,8 @@ def _send_producer_data(self): # looping. poll_timeout_ms = min(next_ready_check_delay * 1000, not_ready_timeout_ms) if ready_nodes: - log.debug("Nodes with data ready to send: %s", ready_nodes) # trace - log.debug("Created %d produce requests: %s", len(requests), requests) # trace + log.debug("%s: Nodes with data ready to send: %s", self, ready_nodes) # trace + log.debug("%s: Created %d produce requests: %s", self, len(requests), requests) # trace # if some partitions are already ready to be sent, the select time # would be 0; otherwise if some partition already has some data # accumulated but not ready yet, the select time will be the time @@ -212,7 +212,7 @@ def _send_producer_data(self): for node_id, request in six.iteritems(requests): batches = batches_by_node[node_id] - log.debug('Sending Produce Request: %r', request) + log.debug('%s: Sending Produce Request: %r', self, request) (self._client.send(node_id, request, wakeup=False) .add_callback( self._handle_produce_response, node_id, time.time(), batches) @@ -235,7 +235,7 @@ def _maybe_send_transactional_request(self): if next_request_handler is None: return False - log.debug("transactional_id: %s -- Sending transactional request %s", self._transaction_manager.transactional_id, next_request_handler.request) + log.debug("%s: Sending transactional request %s", self, next_request_handler.request) while not self._force_close: target_node = None try: @@ -262,7 +262,7 @@ def _maybe_send_transactional_request(self): return True except Exception as e: - log.warn("Got an exception when trying to find a node to send a transactional request to. Going to back off and retry", e) + log.warn("%s: Got an exception when trying to find a node to send a transactional request to. Going to back off and retry", self, e) if next_request_handler.needs_coordinator(): self._transaction_manager.lookup_coordinator_for_request(next_request_handler) break @@ -277,7 +277,7 @@ def _maybe_send_transactional_request(self): def _maybe_abort_batches(self, exc): if self._accumulator.has_incomplete: - log.error("Aborting producer batches due to fatal error: %s", exc) + log.error("%s: Aborting producer batches due to fatal error: %s", self, exc) self._accumulator.abort_batches(exc) def initiate_close(self): @@ -306,8 +306,8 @@ def _maybe_wait_for_producer_id(self): try: node_id = self._client.least_loaded_node() if node_id is None or not self._client.await_ready(node_id): - log.debug("Could not find an available broker to send InitProducerIdRequest to." + - " Will back off and try again.") + log.debug("%s, Could not find an available broker to send InitProducerIdRequest to." + + " Will back off and try again.", self) time.sleep(self._client.least_loaded_node_refresh_ms() / 1000) continue version = self._client.api_version(InitProducerIdRequest, max_version=1) @@ -321,28 +321,28 @@ def _maybe_wait_for_producer_id(self): self._transaction_manager.set_producer_id_and_epoch(ProducerIdAndEpoch(response.producer_id, response.producer_epoch)) break elif getattr(error_type, 'retriable', False): - log.debug("Retriable error from InitProducerId response: %s", error_type.__name__) + log.debug("%s: Retriable error from InitProducerId response: %s", self, error_type.__name__) if getattr(error_type, 'invalid_metadata', False): self._metadata.request_update() else: self._transaction_manager.transition_to_fatal_error(error_type()) break except Errors.KafkaConnectionError: - log.debug("Broker %s disconnected while awaiting InitProducerId response", node_id) + log.debug("%s: Broker %s disconnected while awaiting InitProducerId response", self, node_id) except Errors.RequestTimedOutError: - log.debug("InitProducerId request to node %s timed out", node_id) - log.debug("Retry InitProducerIdRequest in %sms.", self.config['retry_backoff_ms']) + log.debug("%s: InitProducerId request to node %s timed out", self, node_id) + log.debug("%s: Retry InitProducerIdRequest in %sms.", self, self.config['retry_backoff_ms']) time.sleep(self.config['retry_backoff_ms'] / 1000) def _failed_produce(self, batches, node_id, error): - log.error("Error sending produce request to node %d: %s", node_id, error) # trace + log.error("%s: Error sending produce request to node %d: %s", self, node_id, error) # trace for batch in batches: self._complete_batch(batch, error, -1) def _handle_produce_response(self, node_id, send_time, batches, response): """Handle a produce response.""" # if we have a response, parse it - log.debug('Parsing produce response: %r', response) + log.debug('%s: Parsing produce response: %r', self, response) if response: batches_by_partition = dict([(batch.topic_partition, batch) for batch in batches]) @@ -376,9 +376,9 @@ def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None, log if isinstance(exception, Errors.OutOfOrderSequenceNumberError) and \ not self._transaction_manager.is_transactional() and \ self._transaction_manager.has_producer_id(batch.producer_id): - log.error("The broker received an out of order sequence number for topic-partition %s" + 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.", - batch.topic_partition, base_offset) + self, batch.topic_partition, 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 @@ -414,24 +414,25 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star if error is not None: if self._can_retry(batch, error): # retry - log.warning("Got error produce response on topic-partition %s," + log.warning("%s: Got error produce response on topic-partition %s," " retrying (%d attempts left). Error: %s", - batch.topic_partition, + self, batch.topic_partition, self.config['retries'] - batch.attempts - 1, error) # 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): - log.debug("Retrying batch to topic-partition %s. Sequence number: %s", - batch.topic_partition, + log.debug("%s: Retrying batch to topic-partition %s. Sequence number: %s", + self, batch.topic_partition, self._transaction_manager.sequence_number(batch.topic_partition) if self._transaction_manager else None) self._accumulator.reenqueue(batch) if self._sensors: self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) else: - log.warning("Attempted to retry sending a batch but the producer id/epoch changed from %s/%s to %s/%s. This batch will be dropped" % ( - batch.producer_id, batch.producer_epoch, - self._transaction_manager.producer_id_and_epoch.producer_id, self._transaction_manager.producer_id_and_epoch.epoch)) + log.warning("%s: Attempted to retry sending a batch but the producer id/epoch changed from %s/%s to %s/%s. This batch will be dropped", + 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, log_start_offset=log_start_offset) else: if error is Errors.TopicAuthorizationFailedError: @@ -441,9 +442,9 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star self._fail_batch(batch, error, base_offset=base_offset, timestamp_ms=timestamp_ms, log_start_offset=log_start_offset) if error is Errors.UnknownTopicOrPartitionError: - log.warning("Received unknown topic or partition error in produce request on partition %s." + log.warning("%s: Received unknown topic or partition error in produce request on partition %s." " The topic/partition may not exist or the user may not have Describe access to it", - batch.topic_partition) + self, batch.topic_partition) if getattr(error, 'invalid_metadata', False): self._metadata.request_update() @@ -454,7 +455,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star if self._transaction_manager and self._transaction_manager.producer_id_and_epoch.match(batch): self._transaction_manager.increment_sequence_number(batch.topic_partition, batch.record_count) - log.debug("Incremented sequence number for topic-partition %s to %s", batch.topic_partition, + log.debug("%s: Incremented sequence number for topic-partition %s to %s", self, batch.topic_partition, self._transaction_manager.sequence_number(batch.topic_partition)) # Unmute the completed partition. @@ -516,7 +517,7 @@ def _produce_request(self, node_id, acks, timeout, batches): ) else: if transactional_id is not None: - log.warning('Broker does not support ProduceRequest v3+, required for transactional_id') + log.warning('%s: Broker does not support ProduceRequest v3+, required for transactional_id', self) return ProduceRequest[version]( required_acks=acks, timeout=timeout, @@ -530,6 +531,9 @@ def wakeup(self): def bootstrap_connected(self): return self._client.bootstrap_connected() + def __str__(self): + return "" % (self.config['client_id'], self.config['transactional_id']) + class SenderMetrics(object): From a2f8c5819e570d6b695b21111b8dfb7da3180adb Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Apr 2025 11:21:17 -0700 Subject: [PATCH 132/205] Avoid self refcount in log messages; test thread close on all pythons --- kafka/producer/kafka.py | 38 ++++++++++++------------- kafka/producer/sender.py | 60 ++++++++++++++++++++-------------------- test/test_producer.py | 8 ++---- 3 files changed, 51 insertions(+), 55 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 1535dcedb..9bb958138 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -409,9 +409,9 @@ def __init__(self, **configs): else: self.config['api_version'] = tuple(map(int, deprecated.split('.'))) log.warning('%s: use api_version=%s [tuple] -- "%s" as str is deprecated', - self, str(self.config['api_version']), deprecated) + str(self), str(self.config['api_version']), deprecated) - log.debug("%s: Starting Kafka producer", self) + log.debug("%s: Starting Kafka producer", str(self)) # Configure metrics if self.config['metrics_enabled']: @@ -467,18 +467,18 @@ def __init__(self, **configs): metadata=self._metadata, ) if self._transaction_manager.is_transactional(): - log.info("%s: Instantiated a transactional producer.", self) + log.info("%s: Instantiated a transactional producer.", str(self)) else: - log.info("%s: Instantiated an idempotent producer.", self) + log.info("%s: Instantiated an idempotent producer.", str(self)) if 'retries' not in user_provided_configs: - log.info("%s: Overriding the default 'retries' config to 3 since the idempotent producer is enabled.", self) + log.info("%s: Overriding the default 'retries' config to 3 since the idempotent producer is enabled.", str(self)) self.config['retries'] = 3 elif self.config['retries'] == 0: raise Errors.KafkaConfigurationError("Must set 'retries' to non-zero when using the idempotent producer.") if 'max_in_flight_requests_per_connection' not in user_provided_configs: - log.info("%s: Overriding the default 'max_in_flight_requests_per_connection' to 1 since idempontence is enabled.", self) + log.info("%s: Overriding the default 'max_in_flight_requests_per_connection' to 1 since idempontence is enabled.", str(self)) self.config['max_in_flight_requests_per_connection'] = 1 elif self.config['max_in_flight_requests_per_connection'] != 1: raise Errors.KafkaConfigurationError("Must set 'max_in_flight_requests_per_connection' to 1 in order" @@ -486,7 +486,7 @@ def __init__(self, **configs): " Otherwise we cannot guarantee idempotence.") if 'acks' not in user_provided_configs: - log.info("%s: Overriding the default 'acks' config to 'all' since idempotence is enabled", self) + log.info("%s: Overriding the default 'acks' config to 'all' since idempotence is enabled", str(self)) self.config['acks'] = -1 elif self.config['acks'] != -1: raise Errors.KafkaConfigurationError("Must set 'acks' config to 'all' in order to use the idempotent" @@ -510,7 +510,7 @@ def __init__(self, **configs): self._cleanup = self._cleanup_factory() atexit.register(self._cleanup) - log.debug("%s: Kafka producer started", self) + log.debug("%s: Kafka producer started", str(self)) def bootstrap_connected(self): """Return True if the bootstrap is connected.""" @@ -565,7 +565,7 @@ def __getattr__(self, name): self._unregister_cleanup() if not hasattr(self, '_closed') or self._closed: - log.info('%s: Kafka producer closed', self) + log.info('%s: Kafka producer closed', str(self)) return if timeout is None: # threading.TIMEOUT_MAX is available in Python3.3+ @@ -575,7 +575,7 @@ def __getattr__(self, name): else: assert timeout >= 0 - log.info("%s: Closing the Kafka producer with %s secs timeout.", self, timeout) + log.info("%s: Closing the Kafka producer with %s secs timeout.", str(self), timeout) self.flush(timeout) invoked_from_callback = bool(threading.current_thread() is self._sender) if timeout > 0: @@ -584,7 +584,7 @@ def __getattr__(self, name): " prevent useless blocking due to self-join. This" " means you have incorrectly invoked close with a" " non-zero timeout from the producer call-back.", - self, timeout) + str(self), timeout) else: # Try to close gracefully. if self._sender is not None: @@ -594,7 +594,7 @@ def __getattr__(self, name): if self._sender is not None and self._sender.is_alive(): log.info("%s: Proceeding to force close the producer since pending" " requests could not be completed within timeout %s.", - self, timeout) + str(self), timeout) self._sender.force_close() if self._metrics: @@ -608,7 +608,7 @@ def __getattr__(self, name): except AttributeError: pass self._closed = True - log.debug("%s: The Kafka producer has closed.", self) + log.debug("%s: The Kafka producer has closed.", str(self)) def partitions_for(self, topic): """Returns set of all known partitions for the topic.""" @@ -817,7 +817,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest self._ensure_valid_record_size(message_size) tp = TopicPartition(topic, partition) - log.debug("%s: Sending (key=%r value=%r headers=%r) to %s", self, key, value, headers, tp) + log.debug("%s: Sending (key=%r value=%r headers=%r) to %s", str(self), key, value, headers, tp) if self._transaction_manager and self._transaction_manager.is_transactional(): self._transaction_manager.maybe_add_partition_to_transaction(tp) @@ -827,7 +827,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest future, batch_is_full, new_batch_created = result if batch_is_full or new_batch_created: log.debug("%s: Waking up the sender since %s is either full or" - " getting a new batch", self, tp) + " getting a new batch", str(self), tp) self._sender.wakeup() return future @@ -835,7 +835,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest # for API exceptions return them in the future, # for other exceptions raise directly except Errors.BrokerResponseError as e: - log.error("%s: Exception occurred during message send: %s", self, e) + log.error("%s: Exception occurred during message send: %s", str(self), e) return FutureRecordMetadata( FutureProduceResult(TopicPartition(topic, partition)), -1, None, None, @@ -866,7 +866,7 @@ def flush(self, timeout=None): KafkaTimeoutError: failure to flush buffered records within the provided timeout """ - log.debug("%s: Flushing accumulated records in producer.", self) + log.debug("%s: Flushing accumulated records in producer.", str(self)) self._accumulator.begin_flush() self._sender.wakeup() self._accumulator.await_flush_completion(timeout=timeout) @@ -912,7 +912,7 @@ def _wait_on_metadata(self, topic, max_wait): if not metadata_event: metadata_event = threading.Event() - log.debug("%s: Requesting metadata update for topic %s", self, topic) + log.debug("%s: Requesting metadata update for topic %s", str(self), topic) metadata_event.clear() future = self._metadata.request_update() @@ -926,7 +926,7 @@ def _wait_on_metadata(self, topic, max_wait): raise Errors.TopicAuthorizationFailedError(set([topic])) else: elapsed = time.time() - begin - log.debug("%s: _wait_on_metadata woke after %s secs.", self, elapsed) + log.debug("%s: _wait_on_metadata woke after %s secs.", str(self), elapsed) def _serialize(self, f, topic, data): if not f: diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 3637be416..0e3806175 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -62,17 +62,17 @@ def __init__(self, client, metadata, accumulator, **configs): def run(self): """The main run loop for the sender thread.""" - log.debug("%s: Starting Kafka producer I/O thread.", self) + log.debug("%s: Starting Kafka producer I/O thread.", str(self)) # main loop, runs until close is called while self._running: try: self.run_once() except Exception: - log.exception("%s: Uncaught error in kafka producer I/O thread", self) + log.exception("%s: Uncaught error in kafka producer I/O thread", str(self)) log.debug("%s: Beginning shutdown of Kafka producer I/O thread, sending" - " remaining records.", self) + " remaining records.", str(self)) # okay we stopped accepting requests but there may still be # requests in the accumulator or waiting for acknowledgment, @@ -83,7 +83,7 @@ def run(self): try: self.run_once() except Exception: - log.exception("%s: Uncaught error in kafka producer I/O thread", self) + log.exception("%s: Uncaught error in kafka producer I/O thread", str(self)) if self._force_close: # We need to fail all the incomplete batches and wake up the @@ -93,9 +93,9 @@ def run(self): try: self._client.close() except Exception: - log.exception("%s: Failed to close network client", self) + log.exception("%s: Failed to close network client", str(self)) - log.debug("%s: Shutdown of Kafka producer I/O thread has completed.", self) + log.debug("%s: Shutdown of Kafka producer I/O thread has completed.", str(self)) def run_once(self): """Run a single iteration of sending.""" @@ -125,7 +125,7 @@ def run_once(self): except Errors.SaslAuthenticationFailedError as e: # This is already logged as error, but propagated here to perform any clean ups. - log.debug("%s: Authentication exception while processing transactional request: %s", self, e) + log.debug("%s: Authentication exception while processing transactional request: %s", str(self), e) self._transaction_manager.authentication_failed(e) poll_timeout_ms = self._send_producer_data() @@ -139,7 +139,7 @@ def _send_producer_data(self): # if there are any partitions whose leaders are not known yet, force # metadata update if unknown_leaders_exist: - log.debug('%s: Unknown leaders exist, requesting metadata update', self) + log.debug('%s: Unknown leaders exist, requesting metadata update', str(self)) self._metadata.request_update() # remove any nodes we aren't ready to send to @@ -147,7 +147,7 @@ def _send_producer_data(self): for node in list(ready_nodes): if not self._client.is_ready(node): node_delay_ms = self._client.connection_delay(node) - log.debug('%s: Node %s not ready; delaying produce of accumulated batch (%f ms)', self, node, node_delay_ms) + log.debug('%s: Node %s not ready; delaying produce of accumulated batch (%f ms)', str(self), node, node_delay_ms) self._client.maybe_connect(node, wakeup=False) ready_nodes.remove(node) not_ready_timeout_ms = min(not_ready_timeout_ms, node_delay_ms) @@ -166,7 +166,7 @@ def _send_producer_data(self): self.config['request_timeout_ms'], self._metadata) if expired_batches: - log.debug("%s: Expired %s batches in accumulator", self, len(expired_batches)) + log.debug("%s: Expired %s batches in accumulator", str(self), len(expired_batches)) # Reset the producer_id if an expired batch has previously been sent to the broker. # See the documentation of `TransactionState.reset_producer_id` to understand why @@ -200,8 +200,8 @@ def _send_producer_data(self): # looping. poll_timeout_ms = min(next_ready_check_delay * 1000, not_ready_timeout_ms) if ready_nodes: - log.debug("%s: Nodes with data ready to send: %s", self, ready_nodes) # trace - log.debug("%s: Created %d produce requests: %s", self, len(requests), requests) # trace + log.debug("%s: Nodes with data ready to send: %s", str(self), ready_nodes) # trace + log.debug("%s: Created %d produce requests: %s", str(self), len(requests), requests) # trace # if some partitions are already ready to be sent, the select time # would be 0; otherwise if some partition already has some data # accumulated but not ready yet, the select time will be the time @@ -212,7 +212,7 @@ def _send_producer_data(self): for node_id, request in six.iteritems(requests): batches = batches_by_node[node_id] - log.debug('%s: Sending Produce Request: %r', self, request) + log.debug('%s: Sending Produce Request: %r', str(self), request) (self._client.send(node_id, request, wakeup=False) .add_callback( self._handle_produce_response, node_id, time.time(), batches) @@ -235,7 +235,7 @@ def _maybe_send_transactional_request(self): if next_request_handler is None: return False - log.debug("%s: Sending transactional request %s", self, next_request_handler.request) + log.debug("%s: Sending transactional request %s", str(self), next_request_handler.request) while not self._force_close: target_node = None try: @@ -262,7 +262,7 @@ def _maybe_send_transactional_request(self): return True except Exception as e: - log.warn("%s: Got an exception when trying to find a node to send a transactional request to. Going to back off and retry", self, e) + log.warn("%s: Got an exception when trying to find a node to send a transactional request to. Going to back off and retry", str(self), e) if next_request_handler.needs_coordinator(): self._transaction_manager.lookup_coordinator_for_request(next_request_handler) break @@ -277,7 +277,7 @@ def _maybe_send_transactional_request(self): def _maybe_abort_batches(self, exc): if self._accumulator.has_incomplete: - log.error("%s: Aborting producer batches due to fatal error: %s", self, exc) + log.error("%s: Aborting producer batches due to fatal error: %s", str(self), exc) self._accumulator.abort_batches(exc) def initiate_close(self): @@ -307,7 +307,7 @@ def _maybe_wait_for_producer_id(self): node_id = self._client.least_loaded_node() if node_id is None or not self._client.await_ready(node_id): log.debug("%s, Could not find an available broker to send InitProducerIdRequest to." + - " Will back off and try again.", self) + " Will back off and try again.", str(self)) time.sleep(self._client.least_loaded_node_refresh_ms() / 1000) continue version = self._client.api_version(InitProducerIdRequest, max_version=1) @@ -321,28 +321,28 @@ def _maybe_wait_for_producer_id(self): self._transaction_manager.set_producer_id_and_epoch(ProducerIdAndEpoch(response.producer_id, response.producer_epoch)) break elif getattr(error_type, 'retriable', False): - log.debug("%s: Retriable error from InitProducerId response: %s", self, error_type.__name__) + log.debug("%s: Retriable error from InitProducerId response: %s", str(self), error_type.__name__) if getattr(error_type, 'invalid_metadata', False): self._metadata.request_update() else: self._transaction_manager.transition_to_fatal_error(error_type()) break except Errors.KafkaConnectionError: - log.debug("%s: Broker %s disconnected while awaiting InitProducerId response", self, node_id) + log.debug("%s: Broker %s disconnected while awaiting InitProducerId response", str(self), node_id) except Errors.RequestTimedOutError: - log.debug("%s: InitProducerId request to node %s timed out", self, node_id) - log.debug("%s: Retry InitProducerIdRequest in %sms.", self, self.config['retry_backoff_ms']) + log.debug("%s: InitProducerId request to node %s timed out", str(self), node_id) + log.debug("%s: Retry InitProducerIdRequest in %sms.", str(self), self.config['retry_backoff_ms']) time.sleep(self.config['retry_backoff_ms'] / 1000) def _failed_produce(self, batches, node_id, error): - log.error("%s: Error sending produce request to node %d: %s", self, node_id, error) # trace + 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) def _handle_produce_response(self, node_id, send_time, batches, response): """Handle a produce response.""" # if we have a response, parse it - log.debug('%s: Parsing produce response: %r', self, response) + log.debug('%s: Parsing produce response: %r', str(self), response) if response: batches_by_partition = dict([(batch.topic_partition, batch) for batch in batches]) @@ -378,7 +378,7 @@ def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None, log 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.", - self, batch.topic_partition, base_offset) + str(self), batch.topic_partition, 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 @@ -416,21 +416,21 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star # retry log.warning("%s: Got error produce response on topic-partition %s," " retrying (%d attempts left). Error: %s", - self, batch.topic_partition, + str(self), batch.topic_partition, self.config['retries'] - batch.attempts - 1, error) # 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): log.debug("%s: Retrying batch to topic-partition %s. Sequence number: %s", - self, batch.topic_partition, + str(self), batch.topic_partition, self._transaction_manager.sequence_number(batch.topic_partition) if self._transaction_manager else None) self._accumulator.reenqueue(batch) if self._sensors: self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) else: log.warning("%s: Attempted to retry sending a batch but the producer id/epoch changed from %s/%s to %s/%s. This batch will be dropped", - self, batch.producer_id, batch.producer_epoch, + 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, log_start_offset=log_start_offset) @@ -444,7 +444,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star if error is Errors.UnknownTopicOrPartitionError: log.warning("%s: Received unknown topic or partition error in produce request on partition %s." " The topic/partition may not exist or the user may not have Describe access to it", - self, batch.topic_partition) + str(self), batch.topic_partition) if getattr(error, 'invalid_metadata', False): self._metadata.request_update() @@ -455,7 +455,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star if self._transaction_manager and self._transaction_manager.producer_id_and_epoch.match(batch): self._transaction_manager.increment_sequence_number(batch.topic_partition, batch.record_count) - log.debug("%s: Incremented sequence number for topic-partition %s to %s", self, batch.topic_partition, + log.debug("%s: Incremented sequence number for topic-partition %s to %s", str(self), batch.topic_partition, self._transaction_manager.sequence_number(batch.topic_partition)) # Unmute the completed partition. @@ -517,7 +517,7 @@ def _produce_request(self, node_id, acks, timeout, batches): ) else: if transactional_id is not None: - log.warning('%s: Broker does not support ProduceRequest v3+, required for transactional_id', self) + log.warning('%s: Broker does not support ProduceRequest v3+, required for transactional_id', str(self)) return ProduceRequest[version]( required_acks=acks, timeout=timeout, diff --git a/test/test_producer.py b/test/test_producer.py index 8a8c48324..e79c682a7 100644 --- a/test/test_producer.py +++ b/test/test_producer.py @@ -11,15 +11,11 @@ from kafka.producer.transaction_manager import TransactionManager, ProducerIdAndEpoch -@pytest.mark.skipif(platform.python_implementation() != 'CPython', - reason='Test relies on CPython-specific gc policies') -def test_kafka_producer_gc_cleanup(): - gc.collect() +def test_kafka_producer_thread_close(): threads = threading.active_count() producer = KafkaProducer(api_version=(2, 1)) # set api_version explicitly to avoid auto-detection assert threading.active_count() == threads + 1 - del(producer) - gc.collect() + producer.close() assert threading.active_count() == threads From e92defe73b96e5907ebaf75d7182f333942b0900 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Apr 2025 11:24:26 -0700 Subject: [PATCH 133/205] Fix client.wakeup() race from producer/sender close --- kafka/client_async.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 30258b7bd..448a995ba 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -1115,7 +1115,7 @@ def api_version(self, operation, max_version=None): return version def wakeup(self): - if self._waking or self._wake_w is None: + if self._closed or self._waking or self._wake_w is None: return with self._wake_lock: try: From 2f262b9f993dfa84edb86cb75d45d1a98bc2a8cc Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 16 Apr 2025 18:48:54 -0700 Subject: [PATCH 134/205] KAFKA-4547: Avoid resetting paused partitions to committed offsets --- kafka/consumer/fetcher.py | 8 +---- kafka/consumer/group.py | 6 ++-- kafka/consumer/subscription_state.py | 14 ++++++-- test/test_fetcher.py | 50 +++++++++++++++++++++++++++- 4 files changed, 64 insertions(+), 14 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 96bf3b79b..19f5c75f1 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -194,13 +194,7 @@ def update_fetch_positions(self, partitions, timeout_ms=None): inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout updating fetch positions') # reset the fetch position to the committed position for tp in partitions: - if not self._subscriptions.is_assigned(tp): - log.warning("partition %s is not assigned - skipping offset" - " update", tp) - continue - elif self._subscriptions.is_fetchable(tp): - log.warning("partition %s is still fetchable -- skipping offset" - " update", tp) + if not self._subscriptions.is_assigned(tp) or self._subscriptions.has_valid_position(tp): continue if self._subscriptions.is_offset_reset_needed(tp): diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 7fff6e795..58fced337 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -1144,9 +1144,9 @@ def _update_fetch_positions(self, partitions, timeout_ms=None): # their own offsets). self._fetcher.reset_offsets_if_needed(partitions, timeout_ms=inner_timeout_ms()) - if not self._subscription.has_all_fetch_positions(): - # if we still don't have offsets for all partitions, then we should either seek - # to the last committed position or reset using the auto reset policy + if not self._subscription.has_all_fetch_positions(partitions): + # if we still don't have offsets for the given partitions, then we should either + # seek to the last committed position or reset using the auto reset policy if (self.config['api_version'] >= (0, 8, 1) and self.config['group_id'] is not None): # first refresh commits for all assigned partitions diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 0ff2ae91b..5c1a65426 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -351,9 +351,11 @@ def has_default_offset_reset_policy(self): def is_offset_reset_needed(self, partition): return self.assignment[partition].awaiting_reset - def has_all_fetch_positions(self): - for state in self.assignment.values(): - if not state.has_valid_position: + def has_all_fetch_positions(self, partitions=None): + if partitions is None: + partitions = self.assigned_partitions() + for tp in partitions: + if not self.has_valid_position(tp): return False return True @@ -364,6 +366,9 @@ def missing_fetch_positions(self): missing.add(partition) return missing + def has_valid_position(self, partition): + return partition in self.assignment and self.assignment[partition].has_valid_position + def is_assigned(self, partition): return partition in self.assignment @@ -387,6 +392,9 @@ def move_partition_to_end(self, partition): state = self.assignment.pop(partition) self.assignment[partition] = state + def position(self, partition): + return self.assignment[partition].position + class TopicPartitionState(object): def __init__(self): diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 80bd0e42d..8d41c0817 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -17,7 +17,7 @@ from kafka.future import Future from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.fetch import FetchRequest, FetchResponse -from kafka.protocol.list_offsets import ListOffsetsResponse +from kafka.protocol.list_offsets import ListOffsetsResponse, OffsetResetStrategy from kafka.errors import ( StaleMetadata, LeaderNotAvailableError, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError @@ -610,3 +610,51 @@ def test_partition_records_compacted_offset(mocker): msgs = records.take() assert len(msgs) == batch_end - fetch_offset - 1 assert msgs[0].offset == fetch_offset + 1 + + +def test_update_fetch_positions_paused(subscription_state, client, mocker): + fetcher = Fetcher(client, subscription_state) + tp = TopicPartition('foo', 0) + subscription_state.assign_from_user([tp]) + subscription_state.pause(tp) # paused partition does not have a valid position + subscription_state.need_offset_reset(tp, OffsetResetStrategy.LATEST) + + mocker.patch.object(fetcher, '_retrieve_offsets', return_value={tp: OffsetAndTimestamp(10, 1, -1)}) + fetcher.update_fetch_positions([tp]) + + assert not subscription_state.is_offset_reset_needed(tp) + assert not subscription_state.is_fetchable(tp) # because tp is paused + assert subscription_state.has_valid_position(tp) + assert subscription_state.position(tp) == OffsetAndMetadata(10, '', -1) + + +def test_update_fetch_positions_paused_without_valid(subscription_state, client, mocker): + fetcher = Fetcher(client, subscription_state) + tp = TopicPartition('foo', 0) + subscription_state.assign_from_user([tp]) + subscription_state.pause(tp) # paused partition does not have a valid position + + mocker.patch.object(fetcher, '_retrieve_offsets', return_value={tp: OffsetAndTimestamp(0, 1, -1)}) + fetcher.update_fetch_positions([tp]) + + assert not subscription_state.is_offset_reset_needed(tp) + assert not subscription_state.is_fetchable(tp) # because tp is paused + assert subscription_state.has_valid_position(tp) + assert subscription_state.position(tp) == OffsetAndMetadata(0, '', -1) + + +def test_update_fetch_positions_paused_with_valid(subscription_state, client, mocker): + fetcher = Fetcher(client, subscription_state) + tp = TopicPartition('foo', 0) + subscription_state.assign_from_user([tp]) + subscription_state.assignment[tp].committed = OffsetAndMetadata(0, '', -1) + subscription_state.seek(tp, 10) + subscription_state.pause(tp) # paused partition already has a valid position + + mocker.patch.object(fetcher, '_retrieve_offsets', return_value={tp: OffsetAndTimestamp(0, 1, -1)}) + fetcher.update_fetch_positions([tp]) + + assert not subscription_state.is_offset_reset_needed(tp) + assert not subscription_state.is_fetchable(tp) # because tp is paused + assert subscription_state.has_valid_position(tp) + assert subscription_state.position(tp) == OffsetAndMetadata(10, '', -1) From d1666680adec02ec88a276edf0fdfa02c3f813f9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Apr 2025 12:01:34 -0700 Subject: [PATCH 135/205] KAFKA-4937: Batch offset fetches in the Consumer --- kafka/consumer/fetcher.py | 50 +++++++++++++++------------- kafka/consumer/group.py | 3 +- kafka/consumer/subscription_state.py | 2 +- test/test_fetcher.py | 18 +++++----- 4 files changed, 39 insertions(+), 34 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 19f5c75f1..aa10fecb3 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -161,11 +161,13 @@ def reset_offsets_if_needed(self, partitions, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms provided """ - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout resetting offsets') + needs_offset_reset = set() for tp in partitions: - # TODO: If there are several offsets to reset, we could submit offset requests in parallel if self._subscriptions.is_assigned(tp) and self._subscriptions.is_offset_reset_needed(tp): - self._reset_offset(tp, timeout_ms=inner_timeout_ms()) + needs_offset_reset.add(tp) + + if needs_offset_reset: + self._reset_offsets(needs_offset_reset, timeout_ms=timeout_ms) def _clean_done_fetch_futures(self): while True: @@ -191,25 +193,28 @@ def update_fetch_positions(self, partitions, timeout_ms=None): partition and no reset policy is available KafkaTimeoutError if timeout_ms provided. """ - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout updating fetch positions') + needs_offset_reset = set() # reset the fetch position to the committed position for tp in partitions: if not self._subscriptions.is_assigned(tp) or self._subscriptions.has_valid_position(tp): continue if self._subscriptions.is_offset_reset_needed(tp): - self._reset_offset(tp, timeout_ms=inner_timeout_ms()) + needs_offset_reset.add(tp) elif self._subscriptions.assignment[tp].committed is None: # there's no committed position, so we need to reset with the # default strategy self._subscriptions.need_offset_reset(tp) - self._reset_offset(tp, timeout_ms=inner_timeout_ms()) + needs_offset_reset.add(tp) else: committed = self._subscriptions.assignment[tp].committed.offset log.debug("Resetting offset for partition %s to the committed" " offset %s", tp, committed) self._subscriptions.seek(tp, committed) + if needs_offset_reset: + self._reset_offsets(needs_offset_reset, timeout_ms=timeout_ms) + def get_offsets_by_times(self, timestamps, timeout_ms): offsets = self._retrieve_offsets(timestamps, timeout_ms) for tp in timestamps: @@ -232,37 +237,36 @@ def beginning_or_end_offset(self, partitions, timestamp, timeout_ms): offsets[tp] = offsets[tp].offset return offsets - def _reset_offset(self, partition, timeout_ms=None): - """Reset offsets for the given partition using the offset reset strategy. + def _reset_offsets(self, partitions, timeout_ms=None): + """Reset offsets for the given partitions using the offset reset strategy. Arguments: - partition (TopicPartition): the partition that needs reset offset + partitions ([TopicPartition]): the partitions that need offsets reset Raises: NoOffsetForPartitionError: if no offset reset strategy is defined KafkaTimeoutError if timeout_ms provided """ - timestamp = self._subscriptions.assignment[partition].reset_strategy - if timestamp is OffsetResetStrategy.EARLIEST: - strategy = 'earliest' - elif timestamp is OffsetResetStrategy.LATEST: - strategy = 'latest' - else: - raise NoOffsetForPartitionError(partition) + offset_resets = dict() + for tp in partitions: + ts = self._subscriptions.assignment[tp].reset_strategy + if not ts: + raise NoOffsetForPartitionError(tp) + offset_resets[tp] = ts - log.debug("Resetting offset for partition %s to offset %s.", - partition, strategy) - offsets = self._retrieve_offsets({partition: timestamp}, timeout_ms=timeout_ms) + offsets = self._retrieve_offsets(offset_resets, timeout_ms=timeout_ms) - if partition in offsets: - offset = offsets[partition].offset + for partition in partitions: + if partition not in offsets: + raise NoOffsetForPartitionError(partition) # we might lose the assignment while fetching the offset, # so check it is still active if self._subscriptions.is_assigned(partition): + offset = offsets[partition].offset + log.debug("Resetting offset for partition %s to offset %s.", + partition, offset) self._subscriptions.seek(partition, offset) - else: - log.debug("Could not find offset for partition %s since it is probably deleted" % (partition,)) def _retrieve_offsets(self, timestamps, timeout_ms=None): """Fetch offset for each partition passed in ``timestamps`` map. diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 58fced337..078f49c39 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -760,7 +760,8 @@ def position(self, partition, timeout_ms=None): assert self._subscription.is_assigned(partition), 'Partition is not assigned' position = self._subscription.assignment[partition].position if position is None: - self._update_fetch_positions([partition], timeout_ms=timeout_ms) + # batch update fetch positions for any partitions without a valid position + self._update_fetch_positions(self._subscription.assigned_partitions(), timeout_ms=timeout_ms) position = self._subscription.assignment[partition].position return position.offset if position else None diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 5c1a65426..0f479a55b 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -402,7 +402,7 @@ def __init__(self): self.has_valid_position = False # whether we have valid position self.paused = False # whether this partition has been paused by the user self.awaiting_reset = False # whether we are awaiting reset - self.reset_strategy = None # the reset strategy if awaitingReset is set + self.reset_strategy = None # the reset strategy if awaiting_reset is set self._position = None # OffsetAndMetadata exposed to the user self.highwater = None self.drop_pending_record_batch = False diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 8d41c0817..14c735ea1 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -109,38 +109,38 @@ def test_create_fetch_requests(fetcher, mocker, api_version, fetch_version): def test_update_fetch_positions(fetcher, topic, mocker): - mocker.patch.object(fetcher, '_reset_offset') + mocker.patch.object(fetcher, '_reset_offsets') partition = TopicPartition(topic, 0) # unassigned partition fetcher.update_fetch_positions([TopicPartition('fizzbuzz', 0)]) - assert fetcher._reset_offset.call_count == 0 + assert fetcher._reset_offsets.call_count == 0 # fetchable partition (has offset, not paused) fetcher.update_fetch_positions([partition]) - assert fetcher._reset_offset.call_count == 0 + assert fetcher._reset_offsets.call_count == 0 # partition needs reset, no committed offset fetcher._subscriptions.need_offset_reset(partition) fetcher._subscriptions.assignment[partition].awaiting_reset = False fetcher.update_fetch_positions([partition]) - fetcher._reset_offset.assert_called_with(partition, timeout_ms=None) + fetcher._reset_offsets.assert_called_with(set([partition]), timeout_ms=None) assert fetcher._subscriptions.assignment[partition].awaiting_reset is True fetcher.update_fetch_positions([partition]) - fetcher._reset_offset.assert_called_with(partition, timeout_ms=None) + fetcher._reset_offsets.assert_called_with(set([partition]), timeout_ms=None) # partition needs reset, has committed offset - fetcher._reset_offset.reset_mock() + fetcher._reset_offsets.reset_mock() fetcher._subscriptions.need_offset_reset(partition) fetcher._subscriptions.assignment[partition].awaiting_reset = False fetcher._subscriptions.assignment[partition].committed = OffsetAndMetadata(123, '', -1) mocker.patch.object(fetcher._subscriptions, 'seek') fetcher.update_fetch_positions([partition]) - assert fetcher._reset_offset.call_count == 0 + assert fetcher._reset_offsets.call_count == 0 fetcher._subscriptions.seek.assert_called_with(partition, 123) -def test__reset_offset(fetcher, mocker): +def test__reset_offsets(fetcher, mocker): tp = TopicPartition("topic", 0) fetcher._subscriptions.subscribe(topics=["topic"]) fetcher._subscriptions.assign_from_subscribed([tp]) @@ -148,7 +148,7 @@ def test__reset_offset(fetcher, mocker): mocked = mocker.patch.object(fetcher, '_retrieve_offsets') mocked.return_value = {tp: OffsetAndTimestamp(1001, None, -1)} - fetcher._reset_offset(tp) + fetcher._reset_offsets([tp]) assert not fetcher._subscriptions.assignment[tp].awaiting_reset assert fetcher._subscriptions.assignment[tp].position.offset == 1001 From 2513d55b2d6960647452328c08f14f03a298f1fd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Apr 2025 13:11:30 -0700 Subject: [PATCH 136/205] KAFKA-5075 - Defer consumer fetcher exception if fetch position has already increased --- kafka/consumer/fetcher.py | 47 +++++++++++++++++------ test/test_fetcher.py | 81 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 117 insertions(+), 11 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index aa10fecb3..ee04038ce 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -43,6 +43,10 @@ "partition_data", "metric_aggregator"]) +ExceptionMetadata = collections.namedtuple("ExceptionMetadata", + ["partition", "fetched_offset", "exception"]) + + class NoOffsetForPartitionError(Errors.KafkaError): pass @@ -131,6 +135,7 @@ def __init__(self, client, subscriptions, **configs): self._isolation_level = ISOLATION_LEVEL_CONFIG[self.config['isolation_level']] self._session_handlers = {} self._nodes_with_pending_fetch_requests = set() + self._next_in_line_exception_metadata = None def send_fetches(self): """Send FetchRequests for all assigned partitions that do not already have @@ -356,20 +361,39 @@ def fetched_records(self, max_records=None, update_offsets=True): max_records = self.config['max_poll_records'] assert max_records > 0 + if self._next_in_line_exception_metadata is not None: + exc_meta = self._next_in_line_exception_metadata + self._next_in_line_exception_metadata = None + tp = exc_meta.partition + if self._subscriptions.is_fetchable(tp) and self._subscriptions.position(tp).offset == exc_meta.fetched_offset: + raise exc_meta.exception + drained = collections.defaultdict(list) records_remaining = max_records + # Needed to construct ExceptionMetadata if any exception is found when processing completed_fetch + fetched_partition = None + fetched_offset = -1 - while records_remaining > 0: - if not self._next_partition_records: - if not self._completed_fetches: - break - completion = self._completed_fetches.popleft() - self._next_partition_records = self._parse_fetched_data(completion) - else: - records_remaining -= self._append(drained, - self._next_partition_records, - records_remaining, - update_offsets) + try: + while records_remaining > 0: + if not self._next_partition_records: + if not self._completed_fetches: + break + completion = self._completed_fetches.popleft() + fetched_partition = completion.topic_partition + fetched_offset = completion.fetched_offset + self._next_partition_records = self._parse_fetched_data(completion) + else: + fetched_partition = self._next_partition_records.topic_partition + fetched_offset = self._next_partition_records.next_fetch_offset + records_remaining -= self._append(drained, + self._next_partition_records, + records_remaining, + update_offsets) + except Exception as e: + if not drained: + raise e + self._next_in_line_exception_metadata = ExceptionMetadata(fetched_partition, fetched_offset, e) return dict(drained), bool(self._completed_fetches) def _append(self, drained, part, max_records, update_offsets): @@ -860,6 +884,7 @@ def _on_partition_records_drain(self, partition_records): def close(self): if self._next_partition_records is not None: self._next_partition_records.drain() + self._next_in_line_exception_metadata = None class PartitionRecords(object): def __init__(self, fetch_offset, tp, records, diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 14c735ea1..2f5467ff9 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -658,3 +658,84 @@ def test_update_fetch_positions_paused_with_valid(subscription_state, client, mo assert not subscription_state.is_fetchable(tp) # because tp is paused assert subscription_state.has_valid_position(tp) assert subscription_state.position(tp) == OffsetAndMetadata(10, '', -1) + + +def test_fetch_position_after_exception(client, mocker): + subscription_state = SubscriptionState(offset_reset_strategy='NONE') + fetcher = Fetcher(client, subscription_state) + + tp0 = TopicPartition('foo', 0) + tp1 = TopicPartition('foo', 1) + # verify the advancement in the next fetch offset equals to the number of fetched records when + # some fetched partitions cause Exception. This ensures that consumer won't lose record upon exception + subscription_state.assign_from_user([tp0, tp1]) + subscription_state.seek(tp0, 1) + subscription_state.seek(tp1, 1) + + assert len(fetcher._fetchable_partitions()) == 2 + + empty_records = _build_record_batch([], offset=1) + three_records = _build_record_batch([(None, b'msg', None) for _ in range(3)], offset=1) + fetcher._completed_fetches.append( + CompletedFetch(tp1, 1, 0, [0, 100, three_records], mocker.MagicMock())) + fetcher._completed_fetches.append( + CompletedFetch(tp0, 1, 0, [1, 100, empty_records], mocker.MagicMock())) + records, partial = fetcher.fetched_records() + + assert len(records) == 1 + assert tp1 in records + assert tp0 not in records + assert len(records[tp1]) == 3 + assert subscription_state.position(tp1).offset == 4 + + exceptions = [] + try: + records, partial = fetcher.fetched_records() + except Errors.OffsetOutOfRangeError as e: + exceptions.append(e) + + assert len(exceptions) == 1 + assert isinstance(exceptions[0], Errors.OffsetOutOfRangeError) + assert exceptions[0].args == ({tp0: 1},) + + +def test_seek_before_exception(client, mocker): + subscription_state = SubscriptionState(offset_reset_strategy='NONE') + fetcher = Fetcher(client, subscription_state, max_poll_records=2) + + tp0 = TopicPartition('foo', 0) + tp1 = TopicPartition('foo', 1) + subscription_state.assign_from_user([tp0]) + subscription_state.seek(tp0, 1) + + assert len(fetcher._fetchable_partitions()) == 1 + + three_records = _build_record_batch([(None, b'msg', None) for _ in range(3)], offset=1) + fetcher._completed_fetches.append( + CompletedFetch(tp0, 1, 0, [0, 100, three_records], mocker.MagicMock())) + records, partial = fetcher.fetched_records() + + assert len(records) == 1 + assert tp0 in records + assert len(records[tp0]) == 2 + assert subscription_state.position(tp0).offset == 3 + + subscription_state.assign_from_user([tp0, tp1]) + subscription_state.seek(tp1, 1) + + assert len(fetcher._fetchable_partitions()) == 1 + + empty_records = _build_record_batch([], offset=1) + fetcher._completed_fetches.append( + CompletedFetch(tp1, 1, 0, [1, 100, empty_records], mocker.MagicMock())) + records, partial = fetcher.fetched_records() + + assert len(records) == 1 + assert tp0 in records + assert len(records[tp0]) == 1 + assert subscription_state.position(tp0).offset == 4 + + subscription_state.seek(tp1, 10) + # Should not throw OffsetOutOfRangeError after the seek + records, partial = fetcher.fetched_records() + assert len(records) == 0 From 9300bcbe90725beb877bed986c99b824172577a4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Apr 2025 14:20:58 -0700 Subject: [PATCH 137/205] KAFKA-5078 - defer fetch record exception if iterator has already moved across a valid record --- kafka/consumer/fetcher.py | 20 +++++++++++++++++++- test/test_fetcher.py | 17 +++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index ee04038ce..14dc8a30d 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -393,6 +393,7 @@ def fetched_records(self, max_records=None, update_offsets=True): except Exception as e: if not drained: raise e + # To be thrown in the next call of this method self._next_in_line_exception_metadata = ExceptionMetadata(fetched_partition, fetched_offset, e) return dict(drained), bool(self._completed_fetches) @@ -910,6 +911,7 @@ def __init__(self, fetch_offset, tp, records, self._maybe_skip_record, self._unpack_records(tp, records, key_deserializer, value_deserializer)) self.on_drain = on_drain + self._next_inline_exception = None def _maybe_skip_record(self, record): # When fetching an offset that is in the middle of a @@ -933,12 +935,28 @@ def __bool__(self): def drain(self): if self.record_iterator is not None: self.record_iterator = None + self._next_inline_exception = None if self.metric_aggregator: self.metric_aggregator.record(self.topic_partition, self.bytes_read, self.records_read) self.on_drain(self) + def _maybe_raise_next_inline_exception(self): + if self._next_inline_exception: + exc, self._next_inline_exception = self._next_inline_exception, None + raise exc + def take(self, n=None): - return list(itertools.islice(self.record_iterator, 0, n)) + self._maybe_raise_next_inline_exception() + records = [] + try: + # Note that records.extend(iter) will extend partially when exception raised mid-stream + records.extend(itertools.islice(self.record_iterator, 0, n)) + except Exception as e: + if not records: + raise e + # To be thrown in the next call of this method + self._next_inline_exception = e + return records def _unpack_records(self, tp, records, key_deserializer, value_deserializer): try: diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 2f5467ff9..ddecc90cc 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -459,6 +459,23 @@ def test__unpack_records(mocker): assert records[2].offset == 2 +def test__unpack_records_corrupted(mocker): + tp = TopicPartition('foo', 0) + messages = [ + (None, b"a", None), + (None, b"b", None), + (None, b"c", None), + ] + memory_records = MemoryRecords(_build_record_batch(messages)) + from kafka.record.default_records import DefaultRecord + mocker.patch.object(DefaultRecord, 'validate_crc', side_effect=[True, True, False]) + part_records = Fetcher.PartitionRecords(0, tp, memory_records) + records = part_records.take(10) + assert len(records) == 2 + with pytest.raises(Errors.CorruptRecordError): + part_records.take(10) + + def test__parse_fetched_data(fetcher, topic, mocker): fetcher.config['check_crcs'] = False tp = TopicPartition(topic, 0) From 1b53598d16cc51e25373e228a55fa2a2bc77f02d Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 17 Apr 2025 14:21:18 -0700 Subject: [PATCH 138/205] Drop unused KafkaClient import from test_fetcher --- test/test_fetcher.py | 1 - 1 file changed, 1 deletion(-) diff --git a/test/test_fetcher.py b/test/test_fetcher.py index ddecc90cc..cc4789e6d 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -8,7 +8,6 @@ import itertools import time -from kafka.client_async import KafkaClient from kafka.consumer.fetcher import ( CompletedFetch, ConsumerRecord, Fetcher ) From 39aa42186702eeeb9387da345bfe9f37737f3858 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 23 Apr 2025 07:57:22 -0700 Subject: [PATCH 139/205] fixup exception message --- kafka/producer/transaction_manager.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index e2dd4e020..f8f9d2f52 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -211,7 +211,7 @@ def _fail_if_not_ready_for_send(self): if self.is_transactional(): if not self.has_producer_id(): raise Errors.IllegalStateError( - "Cannot perform a 'send' before completing a call to initTransactions" + "Cannot perform a 'send' before completing a call to init_transactions" " when transactions are enabled.") if self._current_state != TransactionState.IN_TRANSACTION: From be9dd7c95bf5c23cb68727f85204ee2362c532f9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 23 Apr 2025 08:20:20 -0700 Subject: [PATCH 140/205] KAFKA-6397: Consumer should not block setting positions of unavailable partitions (#2593) --- kafka/cluster.py | 4 + kafka/consumer/fetcher.py | 300 +++++++++--------- kafka/consumer/group.py | 103 +++--- kafka/consumer/subscription_state.py | 110 ++++--- kafka/coordinator/consumer.py | 35 +- kafka/errors.py | 4 + test/integration/test_consumer_integration.py | 5 +- test/test_coordinator.py | 20 +- test/test_fetcher.py | 138 ++++---- 9 files changed, 380 insertions(+), 339 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index c92d1d05b..ae822a401 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -202,6 +202,10 @@ def request_update(self): self._future = Future() return self._future + @property + def need_update(self): + return self._need_update + def topics(self, exclude_internal_topics=True): """Get set of known topics. diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 14dc8a30d..ceca1d9b6 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -67,6 +67,7 @@ class Fetcher(six.Iterator): 'check_crcs': True, 'metrics': None, 'metric_group_prefix': 'consumer', + 'request_timeout_ms': 30000, 'retry_backoff_ms': 100, 'enable_incremental_fetch_sessions': True, 'isolation_level': 'read_uncommitted', @@ -135,6 +136,7 @@ def __init__(self, client, subscriptions, **configs): self._isolation_level = ISOLATION_LEVEL_CONFIG[self.config['isolation_level']] self._session_handlers = {} self._nodes_with_pending_fetch_requests = set() + self._cached_list_offsets_exception = None self._next_in_line_exception_metadata = None def send_fetches(self): @@ -156,24 +158,6 @@ def send_fetches(self): self._clean_done_fetch_futures() return futures - def reset_offsets_if_needed(self, partitions, timeout_ms=None): - """Lookup and set offsets for any partitions which are awaiting an - explicit reset. - - Arguments: - partitions (set of TopicPartitions): the partitions to reset - - Raises: - KafkaTimeoutError if timeout_ms provided - """ - needs_offset_reset = set() - for tp in partitions: - if self._subscriptions.is_assigned(tp) and self._subscriptions.is_offset_reset_needed(tp): - needs_offset_reset.add(tp) - - if needs_offset_reset: - self._reset_offsets(needs_offset_reset, timeout_ms=timeout_ms) - def _clean_done_fetch_futures(self): while True: if not self._fetch_futures: @@ -187,62 +171,7 @@ def in_flight_fetches(self): self._clean_done_fetch_futures() return bool(self._fetch_futures) - def update_fetch_positions(self, partitions, timeout_ms=None): - """Update the fetch positions for the provided partitions. - - Arguments: - partitions (list of TopicPartitions): partitions to update - - Raises: - NoOffsetForPartitionError: if no offset is stored for a given - partition and no reset policy is available - KafkaTimeoutError if timeout_ms provided. - """ - needs_offset_reset = set() - # reset the fetch position to the committed position - for tp in partitions: - if not self._subscriptions.is_assigned(tp) or self._subscriptions.has_valid_position(tp): - continue - - if self._subscriptions.is_offset_reset_needed(tp): - needs_offset_reset.add(tp) - elif self._subscriptions.assignment[tp].committed is None: - # there's no committed position, so we need to reset with the - # default strategy - self._subscriptions.need_offset_reset(tp) - needs_offset_reset.add(tp) - else: - committed = self._subscriptions.assignment[tp].committed.offset - log.debug("Resetting offset for partition %s to the committed" - " offset %s", tp, committed) - self._subscriptions.seek(tp, committed) - - if needs_offset_reset: - self._reset_offsets(needs_offset_reset, timeout_ms=timeout_ms) - - def get_offsets_by_times(self, timestamps, timeout_ms): - offsets = self._retrieve_offsets(timestamps, timeout_ms) - for tp in timestamps: - if tp not in offsets: - offsets[tp] = None - return offsets - - def beginning_offsets(self, partitions, timeout_ms): - return self.beginning_or_end_offset( - partitions, OffsetResetStrategy.EARLIEST, timeout_ms) - - def end_offsets(self, partitions, timeout_ms): - return self.beginning_or_end_offset( - partitions, OffsetResetStrategy.LATEST, timeout_ms) - - def beginning_or_end_offset(self, partitions, timestamp, timeout_ms): - timestamps = dict([(tp, timestamp) for tp in partitions]) - offsets = self._retrieve_offsets(timestamps, timeout_ms) - for tp in timestamps: - offsets[tp] = offsets[tp].offset - return offsets - - def _reset_offsets(self, partitions, timeout_ms=None): + def reset_offsets_if_needed(self): """Reset offsets for the given partitions using the offset reset strategy. Arguments: @@ -252,28 +181,24 @@ def _reset_offsets(self, partitions, timeout_ms=None): NoOffsetForPartitionError: if no offset reset strategy is defined KafkaTimeoutError if timeout_ms provided """ + # Raise exception from previous offset fetch if there is one + exc, self._cached_list_offsets_exception = self._cached_list_offsets_exception, None + if exc: + raise exc + + partitions = self._subscriptions.partitions_needing_reset() + if not partitions: + return + offset_resets = dict() for tp in partitions: ts = self._subscriptions.assignment[tp].reset_strategy - if not ts: - raise NoOffsetForPartitionError(tp) - offset_resets[tp] = ts - - offsets = self._retrieve_offsets(offset_resets, timeout_ms=timeout_ms) + if ts: + offset_resets[tp] = ts - for partition in partitions: - if partition not in offsets: - raise NoOffsetForPartitionError(partition) + self._reset_offsets_async(offset_resets) - # we might lose the assignment while fetching the offset, - # so check it is still active - if self._subscriptions.is_assigned(partition): - offset = offsets[partition].offset - log.debug("Resetting offset for partition %s to offset %s.", - partition, offset) - self._subscriptions.seek(partition, offset) - - def _retrieve_offsets(self, timestamps, timeout_ms=None): + def offsets_by_times(self, timestamps, timeout_ms=None): """Fetch offset for each partition passed in ``timestamps`` map. Blocks until offsets are obtained, a non-retriable exception is raised @@ -283,6 +208,7 @@ def _retrieve_offsets(self, timestamps, timeout_ms=None): timestamps: {TopicPartition: int} dict with timestamps to fetch offsets by. -1 for the latest available, -2 for the earliest available. Otherwise timestamp is treated as epoch milliseconds. + timeout_ms (int, optional): The maximum time in milliseconds to block. Returns: {TopicPartition: OffsetAndTimestamp}: Mapping of partition to @@ -293,11 +219,19 @@ def _retrieve_offsets(self, timestamps, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms provided """ + offsets = self._fetch_offsets_by_times(timestamps, timeout_ms) + for tp in timestamps: + if tp not in offsets: + offsets[tp] = None + return offsets + + def _fetch_offsets_by_times(self, timestamps, timeout_ms=None): if not timestamps: return {} inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout fetching offsets') timestamps = copy.copy(timestamps) + fetched_offsets = dict() while True: if not timestamps: return {} @@ -310,31 +244,42 @@ def _retrieve_offsets(self, timestamps, timeout_ms=None): break if future.succeeded(): - return future.value - if not future.retriable(): + fetched_offsets.update(future.value[0]) + if not future.value[1]: + return fetched_offsets + + timestamps = {tp: timestamps[tp] for tp in future.value[1]} + + elif not future.retriable(): raise future.exception # pylint: disable-msg=raising-bad-type - if future.exception.invalid_metadata: + if future.exception.invalid_metadata or self._client.cluster.need_update: refresh_future = self._client.cluster.request_update() self._client.poll(future=refresh_future, timeout_ms=inner_timeout_ms()) if not future.is_done: break - - # Issue #1780 - # Recheck partition existence after after a successful metadata refresh - if refresh_future.succeeded() and isinstance(future.exception, Errors.StaleMetadata): - log.debug("Stale metadata was raised, and we now have an updated metadata. Rechecking partition existence") - unknown_partition = future.exception.args[0] # TopicPartition from StaleMetadata - if self._client.cluster.leader_for_partition(unknown_partition) is None: - log.debug("Removed partition %s from offsets retrieval" % (unknown_partition, )) - timestamps.pop(unknown_partition) else: time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) raise Errors.KafkaTimeoutError( "Failed to get offsets by timestamps in %s ms" % (timeout_ms,)) + def beginning_offsets(self, partitions, timeout_ms): + return self.beginning_or_end_offset( + partitions, OffsetResetStrategy.EARLIEST, timeout_ms) + + def end_offsets(self, partitions, timeout_ms): + return self.beginning_or_end_offset( + partitions, OffsetResetStrategy.LATEST, timeout_ms) + + def beginning_or_end_offset(self, partitions, timestamp, timeout_ms): + timestamps = dict([(tp, timestamp) for tp in partitions]) + offsets = self._fetch_offsets_by_times(timestamps, timeout_ms) + for tp in timestamps: + offsets[tp] = offsets[tp].offset + return offsets + def fetched_records(self, max_records=None, update_offsets=True): """Returns previously fetched records and updates consumed offsets. @@ -449,6 +394,53 @@ def _append(self, drained, part, max_records, update_offsets): part.drain() return 0 + def _reset_offset_if_needed(self, partition, timestamp, offset): + # we might lose the assignment while fetching the offset, or the user might seek to a different offset, + # so verify it is still assigned and still in need of the requested reset + if not self._subscriptions.is_assigned(partition): + log.debug("Skipping reset of partition %s since it is no longer assigned", partition) + elif not self._subscriptions.is_offset_reset_needed(partition): + log.debug("Skipping reset of partition %s since reset is no longer needed", partition) + elif timestamp and not timestamp == self._subscriptions.assignment[partition].reset_strategy: + log.debug("Skipping reset of partition %s since an alternative reset has been requested", partition) + else: + log.info("Resetting offset for partition %s to offset %s.", partition, offset) + self._subscriptions.seek(partition, 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): + if not self._client.ready(node_id): + continue + partitions = set(timestamps_and_epochs.keys()) + expire_at = time.time() + self.config['request_timeout_ms'] / 1000 + self._subscriptions.set_reset_pending(partitions, expire_at) + + def on_success(result): + fetched_offsets, partitions_to_retry = result + if partitions_to_retry: + 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): + ts, _epoch = timestamps_and_epochs[partition] + self._reset_offset_if_needed(partition, ts, offset.offset) + + def on_failure(error): + self._subscriptions.reset_failed(partitions, time.time() + self.config['retry_backoff_ms'] / 1000) + self._client.cluster.request_update() + + if not getattr(error, 'retriable', False): + if not self._cached_list_offsets_exception: + self._cached_list_offsets_exception = error + else: + log.error("Discarding error in ListOffsetResponse because another error is pending: %s", error) + + future = self._send_list_offsets_request(node_id, timestamps_and_epochs) + future.add_callback(on_success) + future.add_errback(on_failure) + def _send_list_offsets_requests(self, timestamps): """Fetch offsets for each partition in timestamps dict. This may send request to multiple nodes, based on who is Leader for partition. @@ -460,39 +452,22 @@ def _send_list_offsets_requests(self, timestamps): Returns: Future: resolves to a mapping of retrieved offsets """ - timestamps_by_node = collections.defaultdict(dict) - for partition, timestamp in six.iteritems(timestamps): - node_id = self._client.cluster.leader_for_partition(partition) - if node_id is None: - if partition.topic not in self._client.cluster.topics(): - log.warning("Could not lookup offsets for partition %s since no metadata is available for topic. " - "Wait for metadata refresh and try again", partition) - else: - log.warning("Could not lookup offsets for partition %s since no metadata is available for it. " - "Wait for metadata refresh and try again", partition) - self._client.add_topic(partition.topic) - return Future().failure(Errors.StaleMetadata(partition)) - elif node_id == -1: - log.debug("Leader for partition %s unavailable for fetching " - "offset, wait for metadata refresh", partition) - return Future().failure( - Errors.LeaderNotAvailableError(partition)) - else: - leader_epoch = -1 - timestamps_by_node[node_id][partition] = (timestamp, leader_epoch) + timestamps_by_node = self._group_list_offset_requests(timestamps) + if not timestamps_by_node: + return Future().failure(Errors.StaleMetadata()) - # Aggregate results until we have all + # Aggregate results until we have all responses list_offsets_future = Future() - responses = [] - node_count = len(timestamps_by_node) + fetched_offsets = dict() + partitions_to_retry = set() + remaining_responses = [len(timestamps_by_node)] # list for mutable / 2.7 hack - def on_success(value): - responses.append(value) - if len(responses) == node_count: - offsets = {} - for r in responses: - offsets.update(r) - list_offsets_future.success(offsets) + def on_success(remaining_responses, value): + remaining_responses[0] -= 1 # noqa: F823 + fetched_offsets.update(value[0]) + partitions_to_retry.update(value[1]) + if not remaining_responses[0] and not list_offsets_future.is_done: + list_offsets_future.success((fetched_offsets, partitions_to_retry)) def on_fail(err): if not list_offsets_future.is_done: @@ -500,12 +475,31 @@ def on_fail(err): for node_id, timestamps in six.iteritems(timestamps_by_node): _f = self._send_list_offsets_request(node_id, timestamps) - _f.add_callback(on_success) + _f.add_callback(on_success, remaining_responses) _f.add_errback(on_fail) return list_offsets_future + def _group_list_offset_requests(self, timestamps): + timestamps_by_node = collections.defaultdict(dict) + for partition, timestamp in six.iteritems(timestamps): + node_id = self._client.cluster.leader_for_partition(partition) + if node_id is None: + self._client.add_topic(partition.topic) + log.debug("Partition %s is unknown for fetching offset", partition) + self._client.cluster.request_update() + elif node_id == -1: + log.debug("Leader for partition %s unavailable for fetching " + "offset, wait for metadata refresh", partition) + self._client.cluster.request_update() + else: + leader_epoch = -1 + timestamps_by_node[node_id][partition] = (timestamp, leader_epoch) + 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) + 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): if version >= 4: @@ -526,12 +520,12 @@ def _send_list_offsets_request(self, node_id, timestamps_and_epochs): self._isolation_level, list(six.iteritems(by_topic))) - # Client returns a future that only fails on network issues # so create a separate future and attach a callback to update it # based on response error codes future = Future() + log.debug("Sending ListOffsetRequest %s to broker %s", request, node_id) _f = self._client.send(node_id, request) _f.add_callback(self._handle_list_offsets_response, future) _f.add_errback(lambda e: future.failure(e)) @@ -547,7 +541,9 @@ def _handle_list_offsets_response(self, future, response): Raises: AssertionError: if response does not match partition """ - timestamp_offset_map = {} + fetched_offsets = dict() + partitions_to_retry = set() + unauthorized_topics = set() for topic, part_data in response.topics: for partition_info in part_data: partition, error_code = partition_info[:2] @@ -572,10 +568,11 @@ def _handle_list_offsets_response(self, future, response): "Fetched offset %s, timestamp %s, leader_epoch %s", partition, offset, timestamp, leader_epoch) if offset != UNKNOWN_OFFSET: - timestamp_offset_map[partition] = OffsetAndTimestamp(offset, timestamp, leader_epoch) + fetched_offsets[partition] = OffsetAndTimestamp(offset, timestamp, leader_epoch) elif error_type is Errors.UnsupportedForMessageFormatError: - # The message format on the broker side is before 0.10.0, - # we simply put None in the response. + # The message format on the broker side is before 0.10.0, which means it does not + # support timestamps. We treat this case the same as if we weren't able to find an + # offset corresponding to the requested timestamp and leave it out of the result. log.debug("Cannot search by timestamp for partition %s because the" " message format version is before 0.10.0", partition) elif error_type in (Errors.NotLeaderForPartitionError, @@ -583,22 +580,23 @@ def _handle_list_offsets_response(self, future, response): Errors.KafkaStorageError): log.debug("Attempt to fetch offsets for partition %s failed due" " to %s, retrying.", error_type.__name__, partition) - future.failure(error_type(partition)) - return + partitions_to_retry.add(partition) elif error_type is Errors.UnknownTopicOrPartitionError: log.warning("Received unknown topic or partition error in ListOffsets " "request for partition %s. The topic/partition " + "may not exist or the user may not have Describe access " "to it.", partition) - future.failure(error_type(partition)) - return + partitions_to_retry.add(partition) + elif error_type is Errors.TopicAuthorizationFailedError: + unauthorized_topics.add(topic) else: log.warning("Attempt to fetch offsets for partition %s failed due to:" " %s", partition, error_type.__name__) - future.failure(error_type(partition)) - return - if not future.is_done: - future.success(timestamp_offset_map) + partitions_to_retry.add(partition) + if unauthorized_topics: + future.failure(Errors.TopicAuthorizationFailedError(unauthorized_topics)) + else: + future.success((fetched_offsets, partitions_to_retry)) def _fetchable_partitions(self): fetchable = self._subscriptions.fetchable_partitions() @@ -636,17 +634,17 @@ def _create_fetch_requests(self): elif not self._client.connected(node_id) and self._client.connection_delay(node_id) > 0: # If we try to send during the reconnect backoff window, then the request is just # going to be failed anyway before being sent, so skip the send for now - log.log(0, "Skipping fetch for partition %s because node %s is awaiting reconnect backoff", + log.debug("Skipping fetch for partition %s because node %s is awaiting reconnect backoff", partition, node_id) elif self._client.throttle_delay(node_id) > 0: # If we try to send while throttled, then the request is just # going to be failed anyway before being sent, so skip the send for now - log.log(0, "Skipping fetch for partition %s because node %s is throttled", + log.debug("Skipping fetch for partition %s because node %s is throttled", partition, node_id) elif node_id in self._nodes_with_pending_fetch_requests: - log.log(0, "Skipping fetch for partition %s because there is a pending fetch request to node %s", + log.debug("Skipping fetch for partition %s because there is a pending fetch request to node %s", partition, node_id) continue @@ -851,7 +849,7 @@ def _parse_fetched_data(self, completed_fetch): " current offset %d", tp, fetch_offset, position.offset) elif self._subscriptions.has_default_offset_reset_policy(): log.info("Fetch offset %s is out of range for topic-partition %s", fetch_offset, tp) - self._subscriptions.need_offset_reset(tp) + self._subscriptions.request_offset_reset(tp) else: raise Errors.OffsetOutOfRangeError({tp: fetch_offset}) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 078f49c39..a86ececf4 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -572,9 +572,8 @@ def committed(self, partition, metadata=False, timeout_ms=None): This offset will be used as the position for the consumer in the event of a failure. - This call may block to do a remote call if the partition in question - isn't assigned to this consumer or if the consumer hasn't yet - initialized its cache of committed offsets. + This call will block to do a remote call to get the latest committed + offsets from the server. Arguments: partition (TopicPartition): The partition to check. @@ -586,28 +585,16 @@ def committed(self, partition, metadata=False, timeout_ms=None): Raises: KafkaTimeoutError if timeout_ms provided + BrokerResponseErrors if OffsetFetchRequest raises an error. """ assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1' assert self.config['group_id'] is not None, 'Requires group_id' if not isinstance(partition, TopicPartition): raise TypeError('partition must be a TopicPartition namedtuple') - if self._subscription.is_assigned(partition): - committed = self._subscription.assignment[partition].committed - if committed is None: - self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=timeout_ms) - committed = self._subscription.assignment[partition].committed - else: - commit_map = self._coordinator.fetch_committed_offsets([partition], timeout_ms=timeout_ms) - if partition in commit_map: - committed = commit_map[partition] - else: - committed = None - - if committed is not None: - if metadata: - return committed - else: - return committed.offset + committed = self._coordinator.fetch_committed_offsets([partition], timeout_ms=timeout_ms) + if partition not in committed: + return None + return committed[partition] if metadata else committed[partition].offset def _fetch_all_topic_metadata(self): """A blocking call that fetches topic metadata for all topics in the @@ -717,10 +704,7 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): if not self._coordinator.poll(timeout_ms=inner_timeout_ms()): return {} - # Fetch positions if we have partitions we're subscribed to that we - # don't know the offset for - if not self._subscription.has_all_fetch_positions(): - self._update_fetch_positions(self._subscription.missing_fetch_positions(), timeout_ms=inner_timeout_ms()) + has_all_fetch_positions = self._update_fetch_positions(timeout_ms=inner_timeout_ms()) # If data is available already, e.g. from a previous network client # poll() call to commit, then just return it immediately @@ -737,7 +721,13 @@ def _poll_once(self, timeout_ms, max_records, update_offsets=True): if records: return records - self._client.poll(timeout_ms=inner_timeout_ms(self._coordinator.time_to_next_poll() * 1000)) + # 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 = inner_timeout_ms(self._coordinator.time_to_next_poll() * 1000) + if not has_all_fetch_positions: + poll_timeout_ms = min(poll_timeout_ms, self.config['retry_backoff_ms']) + + self._client.poll(timeout_ms=poll_timeout_ms) # after the long poll, we should check whether the group needs to rebalance # prior to returning data so that the group can stabilize faster if self._coordinator.need_rejoin(): @@ -758,12 +748,18 @@ def position(self, partition, timeout_ms=None): if not isinstance(partition, TopicPartition): raise TypeError('partition must be a TopicPartition namedtuple') assert self._subscription.is_assigned(partition), 'Partition is not assigned' + + inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout retrieving partition position') position = self._subscription.assignment[partition].position - if position is None: - # batch update fetch positions for any partitions without a valid position - self._update_fetch_positions(self._subscription.assigned_partitions(), timeout_ms=timeout_ms) - position = self._subscription.assignment[partition].position - return position.offset if position else None + try: + while position is None: + # batch update fetch positions for any partitions without a valid position + self._update_fetch_positions(timeout_ms=inner_timeout_ms()) + position = self._subscription.assignment[partition].position + except KafkaTimeoutError: + return None + else: + return position.offset def highwater(self, partition): """Last known highwater offset for a partition. @@ -1056,7 +1052,7 @@ def offsets_for_times(self, timestamps): raise ValueError( "The target time for partition {} is {}. The target time " "cannot be negative.".format(tp, ts)) - return self._fetcher.get_offsets_by_times( + return self._fetcher.offsets_by_times( timestamps, self.config['request_timeout_ms']) def beginning_offsets(self, partitions): @@ -1122,7 +1118,7 @@ def _use_consumer_group(self): return False return True - def _update_fetch_positions(self, partitions, timeout_ms=None): + def _update_fetch_positions(self, timeout_ms=None): """Set the fetch position to the committed position (if there is one) or reset it using the offset reset policy the user has configured. @@ -1136,29 +1132,30 @@ def _update_fetch_positions(self, partitions, timeout_ms=None): NoOffsetForPartitionError: If no offset is stored for a given partition and no offset reset policy is defined. """ - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout updating fetch positions') - try: - # Lookup any positions for partitions which are awaiting reset (which may be the - # case if the user called :meth:`seek_to_beginning` or :meth:`seek_to_end`. We do - # this check first to avoid an unnecessary lookup of committed offsets (which - # typically occurs when the user is manually assigning partitions and managing - # their own offsets). - self._fetcher.reset_offsets_if_needed(partitions, timeout_ms=inner_timeout_ms()) - - if not self._subscription.has_all_fetch_positions(partitions): - # if we still don't have offsets for the given partitions, then we should either - # seek to the last committed position or reset using the auto reset policy - if (self.config['api_version'] >= (0, 8, 1) and - self.config['group_id'] is not None): - # first refresh commits for all assigned partitions - self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=inner_timeout_ms()) - - # Then, do any offset lookups in case some positions are not known - self._fetcher.update_fetch_positions(partitions, timeout_ms=inner_timeout_ms()) + if self._subscription.has_all_fetch_positions(): return True - except KafkaTimeoutError: - return False + if (self.config['api_version'] >= (0, 8, 1) and + self.config['group_id'] is not None): + try: + # If there are any partitions which do not have a valid position and are not + # awaiting reset, then we need to fetch committed offsets. We will only do a + # coordinator lookup if there are partitions which have missing positions, so + # a consumer with manually assigned partitions can avoid a coordinator dependence + # by always ensuring that assigned partitions have an initial position. + self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=timeout_ms) + except KafkaTimeoutError: + pass + + # If there are partitions still needing a position and a reset policy is defined, + # request reset using the default policy. If no reset strategy is defined and there + # are partitions with a missing position, then we will raise an exception. + self._subscription.reset_missing_positions() + + # Finally send an asynchronous request to lookup and update the positions of any + # partitions which are awaiting reset. + self._fetcher.reset_offsets_if_needed() + return False def _message_generator_v2(self): timeout_ms = 1000 * max(0, self._consumer_timeout - time.time()) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index 0f479a55b..cc3675b1d 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -15,10 +15,11 @@ import logging import random import re +import time from kafka.vendor import six -from kafka.errors import IllegalStateError +import kafka.errors as Errors from kafka.protocol.list_offsets import OffsetResetStrategy from kafka.structs import OffsetAndMetadata from kafka.util import ensure_valid_topic_name @@ -52,10 +53,6 @@ class SubscriptionState(object): Note that pause state as well as fetch/consumed positions are not preserved when partition assignment is changed whether directly by the user or through a group rebalance. - - This class also maintains a cache of the latest commit position for each of - the assigned partitions. This is updated through committed() and can be used - to set the initial fetch position (e.g. Fetcher._reset_offset() ). """ _SUBSCRIPTION_EXCEPTION_MESSAGE = ( "You must choose only one way to configure your consumer:" @@ -85,10 +82,8 @@ def __init__(self, offset_reset_strategy='earliest'): self._group_subscription = set() self._user_assignment = set() self.assignment = OrderedDict() - self.listener = None - - # initialize to true for the consumers to fetch offset upon starting up - self.needs_fetch_committed_offsets = True + self.rebalance_listener = None + self.listeners = [] def _set_subscription_type(self, subscription_type): if not isinstance(subscription_type, SubscriptionType): @@ -96,7 +91,7 @@ def _set_subscription_type(self, subscription_type): if self.subscription_type == SubscriptionType.NONE: self.subscription_type = subscription_type elif self.subscription_type != subscription_type: - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) def subscribe(self, topics=(), pattern=None, listener=None): """Subscribe to a list of topics, or a topic regex pattern. @@ -135,7 +130,7 @@ def subscribe(self, topics=(), pattern=None, listener=None): """ assert topics or pattern, 'Must provide topics or pattern' if (topics and pattern): - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) elif pattern: self._set_subscription_type(SubscriptionType.AUTO_PATTERN) @@ -150,7 +145,7 @@ def subscribe(self, topics=(), pattern=None, listener=None): if listener and not isinstance(listener, ConsumerRebalanceListener): raise TypeError('listener must be a ConsumerRebalanceListener') - self.listener = listener + self.rebalance_listener = listener def change_subscription(self, topics): """Change the topic subscription. @@ -166,7 +161,7 @@ def change_subscription(self, topics): - a topic name does not consist of ASCII-characters/'-'/'_'/'.' """ if not self.partitions_auto_assigned(): - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) if isinstance(topics, six.string_types): topics = [topics] @@ -193,13 +188,13 @@ def group_subscribe(self, topics): topics (list of str): topics to add to the group subscription """ if not self.partitions_auto_assigned(): - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) self._group_subscription.update(topics) def reset_group_subscription(self): """Reset the group's subscription to only contain topics subscribed by this consumer.""" if not self.partitions_auto_assigned(): - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) assert self.subscription is not None, 'Subscription required' self._group_subscription.intersection_update(self.subscription) @@ -226,7 +221,6 @@ def assign_from_user(self, partitions): self._user_assignment = set(partitions) self._set_assignment({partition: self.assignment.get(partition, TopicPartitionState()) for partition in partitions}) - self.needs_fetch_committed_offsets = True def assign_from_subscribed(self, assignments): """Update the assignment to the specified partitions @@ -241,16 +235,14 @@ def assign_from_subscribed(self, assignments): consumer instance. """ if not self.partitions_auto_assigned(): - raise IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) for tp in assignments: if tp.topic not in self.subscription: raise ValueError("Assigned partition %s for non-subscribed topic." % (tp,)) - # after rebalancing, we always reinitialize the assignment value # randomized ordering should improve balance for short-lived consumers self._set_assignment({partition: TopicPartitionState() for partition in assignments}, randomize=True) - self.needs_fetch_committed_offsets = True log.info("Updated partition assignment: %s", assignments) def _set_assignment(self, partition_states, randomize=False): @@ -300,8 +292,10 @@ def seek(self, partition, offset): Arguments: partition (TopicPartition): partition for seek operation - offset (int): message offset in partition + offset (int or OffsetAndMetadata): message offset in partition """ + if not isinstance(offset, (int, OffsetAndMetadata)): + raise TypeError("offset must be type in or OffsetAndMetadata") self.assignment[partition].seek(offset) def assigned_partitions(self): @@ -333,7 +327,7 @@ def all_consumed_offsets(self): all_consumed[partition] = state.position return all_consumed - def need_offset_reset(self, partition, offset_reset_strategy=None): + def request_offset_reset(self, partition, offset_reset_strategy=None): """Mark partition for offset reset using specified or default strategy. Arguments: @@ -342,7 +336,11 @@ def need_offset_reset(self, partition, offset_reset_strategy=None): """ if offset_reset_strategy is None: offset_reset_strategy = self._default_offset_reset_strategy - self.assignment[partition].await_reset(offset_reset_strategy) + self.assignment[partition].reset(offset_reset_strategy) + + def set_reset_pending(self, partitions, next_allowed_reset_time): + for partition in partitions: + self.assignment[partition].set_reset_pending(next_allowed_reset_time) def has_default_offset_reset_policy(self): """Return True if default offset reset policy is Earliest or Latest""" @@ -351,24 +349,41 @@ def has_default_offset_reset_policy(self): def is_offset_reset_needed(self, partition): return self.assignment[partition].awaiting_reset - def has_all_fetch_positions(self, partitions=None): - if partitions is None: - partitions = self.assigned_partitions() - for tp in partitions: - if not self.has_valid_position(tp): + def has_all_fetch_positions(self): + for state in six.itervalues(self.assignment): + if not state.has_valid_position: return False return True def missing_fetch_positions(self): missing = set() for partition, state in six.iteritems(self.assignment): - if not state.has_valid_position: + if state.is_missing_position(): missing.add(partition) return missing def has_valid_position(self, partition): return partition in self.assignment and self.assignment[partition].has_valid_position + def reset_missing_positions(self): + partitions_with_no_offsets = set() + for tp, state in six.iteritems(self.assignment): + if state.is_missing_position(): + if self._default_offset_reset_strategy == OffsetResetStrategy.NONE: + partitions_with_no_offsets.add(tp) + else: + state.reset(self._default_offset_reset_strategy) + + if partitions_with_no_offsets: + raise Errors.NoOffsetForPartitionError(partitions_with_no_offsets) + + def partitions_needing_reset(self): + partitions = set() + for tp, state in six.iteritems(self.assignment): + if state.awaiting_reset and state.is_reset_allowed(): + partitions.add(tp) + return partitions + def is_assigned(self, partition): return partition in self.assignment @@ -384,6 +399,10 @@ def pause(self, partition): def resume(self, partition): self.assignment[partition].resume() + def reset_failed(self, partitions, next_retry_time): + for partition in partitions: + self.assignment[partition].reset_failed(next_retry_time) + def move_partition_to_end(self, partition): if partition in self.assignment: try: @@ -398,14 +417,12 @@ def position(self, partition): class TopicPartitionState(object): def __init__(self): - self.committed = None # last committed OffsetAndMetadata - self.has_valid_position = False # whether we have valid position self.paused = False # whether this partition has been paused by the user - self.awaiting_reset = False # whether we are awaiting reset self.reset_strategy = None # the reset strategy if awaiting_reset is set self._position = None # OffsetAndMetadata exposed to the user self.highwater = None self.drop_pending_record_batch = False + self.next_allowed_retry_time = None def _set_position(self, offset): assert self.has_valid_position, 'Valid position required' @@ -417,18 +434,37 @@ def _get_position(self): position = property(_get_position, _set_position, None, "last position") - def await_reset(self, strategy): - self.awaiting_reset = True + def reset(self, strategy): + assert strategy is not None self.reset_strategy = strategy self._position = None - self.has_valid_position = False + self.next_allowed_retry_time = None + + def is_reset_allowed(self): + return self.next_allowed_retry_time is None or self.next_allowed_retry_time < time.time() + + @property + def awaiting_reset(self): + return self.reset_strategy is not None + + def set_reset_pending(self, next_allowed_retry_time): + self.next_allowed_retry_time = next_allowed_retry_time + + def reset_failed(self, next_allowed_retry_time): + self.next_allowed_retry_time = next_allowed_retry_time + + @property + def has_valid_position(self): + return self._position is not None + + def is_missing_position(self): + return not self.has_valid_position and not self.awaiting_reset def seek(self, offset): - self._position = OffsetAndMetadata(offset, '', -1) - self.awaiting_reset = False + self._position = offset if isinstance(offset, OffsetAndMetadata) else OffsetAndMetadata(offset, '', -1) self.reset_strategy = None - self.has_valid_position = True self.drop_pending_record_batch = True + self.next_allowed_retry_time = None def pause(self): self.paused = True diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 2944c7ec7..d4943da31 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -229,10 +229,6 @@ def _on_join_complete(self, generation, member_id, protocol, assignment = ConsumerProtocol.ASSIGNMENT.decode(member_assignment_bytes) - # set the flag to refresh last committed offsets - self._subscription.needs_fetch_committed_offsets = True - - # update partition assignment try: self._subscription.assign_from_subscribed(assignment.partitions()) except ValueError as e: @@ -253,13 +249,13 @@ def _on_join_complete(self, generation, member_id, protocol, assigned, self.group_id) # execute the user's callback after rebalance - if self._subscription.listener: + if self._subscription.rebalance_listener: try: - self._subscription.listener.on_partitions_assigned(assigned) + self._subscription.rebalance_listener.on_partitions_assigned(assigned) except Exception: - log.exception("User provided listener %s for group %s" + log.exception("User provided rebalance listener %s for group %s" " failed on partition assignment: %s", - self._subscription.listener, self.group_id, + self._subscription.rebalance_listener, self.group_id, assigned) def poll(self, timeout_ms=None): @@ -360,14 +356,14 @@ def _on_join_prepare(self, generation, member_id, timeout_ms=None): # execute the user's callback before rebalance log.info("Revoking previously assigned partitions %s for group %s", self._subscription.assigned_partitions(), self.group_id) - if self._subscription.listener: + if self._subscription.rebalance_listener: try: revoked = set(self._subscription.assigned_partitions()) - self._subscription.listener.on_partitions_revoked(revoked) + self._subscription.rebalance_listener.on_partitions_revoked(revoked) except Exception: - log.exception("User provided subscription listener %s" + log.exception("User provided subscription rebalance listener %s" " for group %s failed on_partitions_revoked", - self._subscription.listener, self.group_id) + self._subscription.rebalance_listener, self.group_id) self._is_leader = False self._subscription.reset_group_subscription() @@ -398,13 +394,11 @@ def need_rejoin(self): def refresh_committed_offsets_if_needed(self, timeout_ms=None): """Fetch committed offsets for assigned partitions.""" - if self._subscription.needs_fetch_committed_offsets: - offsets = self.fetch_committed_offsets(self._subscription.assigned_partitions(), timeout_ms=timeout_ms) - for partition, offset in six.iteritems(offsets): - # verify assignment is still active - if self._subscription.is_assigned(partition): - self._subscription.assignment[partition].committed = offset - self._subscription.needs_fetch_committed_offsets = False + missing_fetch_positions = set(self._subscription.missing_fetch_positions()) + offsets = self.fetch_committed_offsets(missing_fetch_positions, timeout_ms=timeout_ms) + for partition, offset in six.iteritems(offsets): + log.debug("Setting offset for partition %s to the committed offset %s", partition, offset.offset); + self._subscription.seek(partition, offset.offset) def fetch_committed_offsets(self, partitions, timeout_ms=None): """Fetch the current committed offsets for specified partitions @@ -505,7 +499,6 @@ def _do_commit_offsets_async(self, offsets, callback=None): offsets.values())) if callback is None: callback = self.config['default_offset_commit_callback'] - self._subscription.needs_fetch_committed_offsets = True future = self._send_offset_commit_request(offsets) future.add_both(lambda res: self.completed_offset_commits.appendleft((callback, offsets, res))) return future @@ -703,8 +696,6 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): if error_type is Errors.NoError: log.debug("Group %s committed offset %s for partition %s", self.group_id, offset, tp) - if self._subscription.is_assigned(tp): - self._subscription.assignment[tp].committed = offset elif error_type is Errors.GroupAuthorizationFailedError: log.error("Not authorized to commit offsets for group %s", self.group_id) diff --git a/kafka/errors.py b/kafka/errors.py index 900dcd5e2..dfdc75015 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -77,6 +77,10 @@ class NoBrokersAvailable(KafkaError): invalid_metadata = True +class NoOffsetForPartitionError(KafkaError): + pass + + class NodeNotReadyError(KafkaError): retriable = True diff --git a/test/integration/test_consumer_integration.py b/test/integration/test_consumer_integration.py index b181845a4..71cf2642d 100644 --- a/test/integration/test_consumer_integration.py +++ b/test/integration/test_consumer_integration.py @@ -9,7 +9,7 @@ from kafka.vendor.six.moves import range import kafka.codec -from kafka.errors import UnsupportedCodecError, UnsupportedVersionError +from kafka.errors import KafkaTimeoutError, UnsupportedCodecError, UnsupportedVersionError from kafka.structs import TopicPartition, OffsetAndTimestamp from test.testutil import Timer, assert_message_count, env_kafka_version, random_string @@ -300,4 +300,5 @@ def test_kafka_consumer_offsets_for_times_errors(kafka_consumer_factory, topic): with pytest.raises(ValueError): consumer.offsets_for_times({tp: -1}) - assert consumer.offsets_for_times({bad_tp: 0}) == {bad_tp: None} + with pytest.raises(KafkaTimeoutError): + consumer.offsets_for_times({bad_tp: 0}) diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 8c114c90f..bfd3a2187 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -231,17 +231,23 @@ def test_need_rejoin(coordinator): def test_refresh_committed_offsets_if_needed(mocker, coordinator): + tp0 = TopicPartition('foobar', 0) + tp1 = TopicPartition('foobar', 1) mocker.patch.object(ConsumerCoordinator, 'fetch_committed_offsets', return_value = { - TopicPartition('foobar', 0): OffsetAndMetadata(123, '', -1), - TopicPartition('foobar', 1): OffsetAndMetadata(234, '', -1)}) - coordinator._subscription.assign_from_user([TopicPartition('foobar', 0)]) - assert coordinator._subscription.needs_fetch_committed_offsets is True + tp0: OffsetAndMetadata(123, '', -1), + tp1: OffsetAndMetadata(234, '', -1)}) + coordinator._subscription.assign_from_user([tp0, tp1]) + coordinator._subscription.request_offset_reset(tp0) + coordinator._subscription.request_offset_reset(tp1) + assert coordinator._subscription.is_offset_reset_needed(tp0) + assert coordinator._subscription.is_offset_reset_needed(tp1) coordinator.refresh_committed_offsets_if_needed() assignment = coordinator._subscription.assignment - assert assignment[TopicPartition('foobar', 0)].committed == OffsetAndMetadata(123, '', -1) - assert TopicPartition('foobar', 1) not in assignment - assert coordinator._subscription.needs_fetch_committed_offsets is False + assert assignment[tp0].position == OffsetAndMetadata(123, '', -1) + assert assignment[tp1].position == OffsetAndMetadata(234, '', -1) + assert not coordinator._subscription.is_offset_reset_needed(tp0) + assert not coordinator._subscription.is_offset_reset_needed(tp1) def test_fetch_committed_offsets(mocker, coordinator): diff --git a/test/test_fetcher.py b/test/test_fetcher.py index cc4789e6d..740fa1bab 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -18,9 +18,10 @@ from kafka.protocol.fetch import FetchRequest, FetchResponse from kafka.protocol.list_offsets import ListOffsetsResponse, OffsetResetStrategy from kafka.errors import ( - StaleMetadata, LeaderNotAvailableError, NotLeaderForPartitionError, + StaleMetadata, NotLeaderForPartitionError, UnknownTopicOrPartitionError, OffsetOutOfRangeError ) +from kafka.future import Future from kafka.record.memory_records import MemoryRecordsBuilder, MemoryRecords from kafka.structs import OffsetAndMetadata, OffsetAndTimestamp, TopicPartition @@ -107,47 +108,41 @@ def test_create_fetch_requests(fetcher, mocker, api_version, fetch_version): assert set([r.API_VERSION for (r, _offsets) in requests_and_offsets]) == set([fetch_version]) -def test_update_fetch_positions(fetcher, topic, mocker): - mocker.patch.object(fetcher, '_reset_offsets') +def test_reset_offsets_if_needed(fetcher, topic, mocker): + mocker.patch.object(fetcher, '_reset_offsets_async') partition = TopicPartition(topic, 0) - # unassigned partition - fetcher.update_fetch_positions([TopicPartition('fizzbuzz', 0)]) - assert fetcher._reset_offsets.call_count == 0 - # fetchable partition (has offset, not paused) - fetcher.update_fetch_positions([partition]) - assert fetcher._reset_offsets.call_count == 0 - - # partition needs reset, no committed offset - fetcher._subscriptions.need_offset_reset(partition) - fetcher._subscriptions.assignment[partition].awaiting_reset = False - fetcher.update_fetch_positions([partition]) - fetcher._reset_offsets.assert_called_with(set([partition]), timeout_ms=None) + fetcher.reset_offsets_if_needed() + assert fetcher._reset_offsets_async.call_count == 0 + + # partition needs reset, no valid position + fetcher._subscriptions.request_offset_reset(partition) + fetcher.reset_offsets_if_needed() + fetcher._reset_offsets_async.assert_called_with({partition: OffsetResetStrategy.EARLIEST}) assert fetcher._subscriptions.assignment[partition].awaiting_reset is True - fetcher.update_fetch_positions([partition]) - fetcher._reset_offsets.assert_called_with(set([partition]), timeout_ms=None) + fetcher.reset_offsets_if_needed() + fetcher._reset_offsets_async.assert_called_with({partition: OffsetResetStrategy.EARLIEST}) - # partition needs reset, has committed offset - fetcher._reset_offsets.reset_mock() - fetcher._subscriptions.need_offset_reset(partition) - fetcher._subscriptions.assignment[partition].awaiting_reset = False - fetcher._subscriptions.assignment[partition].committed = OffsetAndMetadata(123, '', -1) - mocker.patch.object(fetcher._subscriptions, 'seek') - fetcher.update_fetch_positions([partition]) - assert fetcher._reset_offsets.call_count == 0 - fetcher._subscriptions.seek.assert_called_with(partition, 123) + # partition needs reset, has valid position + fetcher._reset_offsets_async.reset_mock() + fetcher._subscriptions.request_offset_reset(partition) + fetcher._subscriptions.seek(partition, 123) + fetcher.reset_offsets_if_needed() + assert fetcher._reset_offsets_async.call_count == 0 -def test__reset_offsets(fetcher, mocker): +def test__reset_offsets_async(fetcher, mocker): tp = TopicPartition("topic", 0) fetcher._subscriptions.subscribe(topics=["topic"]) fetcher._subscriptions.assign_from_subscribed([tp]) - fetcher._subscriptions.need_offset_reset(tp) - mocked = mocker.patch.object(fetcher, '_retrieve_offsets') - - mocked.return_value = {tp: OffsetAndTimestamp(1001, None, -1)} - fetcher._reset_offsets([tp]) + fetcher._subscriptions.request_offset_reset(tp) + fetched_offsets = {tp: OffsetAndTimestamp(1001, None, -1)} + mocker.patch.object(fetcher._client, 'ready', return_value=True) + mocker.patch.object(fetcher, '_send_list_offsets_request', + return_value=Future().success((fetched_offsets, set()))) + mocker.patch.object(fetcher._client.cluster, "leader_for_partition", return_value=0) + fetcher._reset_offsets_async({tp: OffsetResetStrategy.EARLIEST}) assert not fetcher._subscriptions.assignment[tp].awaiting_reset assert fetcher._subscriptions.assignment[tp].position.offset == 1001 @@ -180,7 +175,7 @@ def send_side_effect(*args, **kw): # Leader == -1 fut = fetcher._send_list_offsets_requests({tp: 0}) assert fut.failed() - assert isinstance(fut.exception, LeaderNotAvailableError) + assert isinstance(fut.exception, StaleMetadata) assert not mocked_send.called # Leader == 0, send failed @@ -197,9 +192,9 @@ def send_side_effect(*args, **kw): assert not fut.is_done assert mocked_send.called # Check that we bound the futures correctly to chain success - send_futures.pop().success({tp: (10, 10000)}) + send_futures.pop().success(({tp: (10, 10000)}, set())) assert fut.succeeded() - assert fut.value == {tp: (10, 10000)} + assert fut.value == ({tp: (10, 10000)}, set()) def test__send_list_offsets_requests_multiple_nodes(fetcher, mocker): @@ -233,7 +228,7 @@ def send_side_effect(node_id, timestamps): req_by_node[node] = timestamps if node == 0: # Say tp3 does not have any messages so it's missing - f.success({tp1: (11, 1001)}) + f.success(({tp1: (11, 1001)}, set())) else: second_future = f assert req_by_node == { @@ -243,15 +238,15 @@ def send_side_effect(node_id, timestamps): # We only resolved 1 future so far, so result future is not yet ready assert not fut.is_done - second_future.success({tp2: (12, 1002), tp4: (14, 1004)}) + second_future.success(({tp2: (12, 1002), tp4: (14, 1004)}, set())) assert fut.succeeded() - assert fut.value == {tp1: (11, 1001), tp2: (12, 1002), tp4: (14, 1004)} + assert fut.value == ({tp1: (11, 1001), tp2: (12, 1002), tp4: (14, 1004)}, set()) # -- First succeeded second not del send_futures[:] fut = fetcher._send_list_offsets_requests(tss) assert len(send_futures) == 2 - send_futures[0][2].success({tp1: (11, 1001)}) + send_futures[0][2].success(({tp1: (11, 1001)}, set())) send_futures[1][2].failure(UnknownTopicOrPartitionError(tp1)) assert fut.failed() assert isinstance(fut.exception, UnknownTopicOrPartitionError) @@ -261,7 +256,7 @@ def send_side_effect(node_id, timestamps): fut = fetcher._send_list_offsets_requests(tss) assert len(send_futures) == 2 send_futures[0][2].failure(UnknownTopicOrPartitionError(tp1)) - send_futures[1][2].success({tp1: (11, 1001)}) + send_futures[1][2].success(({tp1: (11, 1001)}, set())) assert fut.failed() assert isinstance(fut.exception, UnknownTopicOrPartitionError) @@ -275,7 +270,7 @@ def test__handle_list_offsets_response_v1(fetcher, mocker): ]) fetcher._handle_list_offsets_response(fut, res) assert fut.succeeded() - assert fut.value == {TopicPartition("topic", 1): OffsetAndTimestamp(9999, 1000, -1)} + assert fut.value == ({TopicPartition("topic", 1): OffsetAndTimestamp(9999, 1000, -1)}, set()) # Broker returns NotLeaderForPartitionError fut = Future() @@ -283,8 +278,8 @@ def test__handle_list_offsets_response_v1(fetcher, mocker): ("topic", [(0, 6, -1, -1)]), ]) fetcher._handle_list_offsets_response(fut, res) - assert fut.failed() - assert isinstance(fut.exception, NotLeaderForPartitionError) + assert fut.succeeded() + assert fut.value == ({}, set([TopicPartition("topic", 0)])) # Broker returns UnknownTopicOrPartitionError fut = Future() @@ -292,21 +287,21 @@ def test__handle_list_offsets_response_v1(fetcher, mocker): ("topic", [(0, 3, -1, -1)]), ]) fetcher._handle_list_offsets_response(fut, res) - assert fut.failed() - assert isinstance(fut.exception, UnknownTopicOrPartitionError) + assert fut.succeeded() + assert fut.value == ({}, set([TopicPartition("topic", 0)])) # Broker returns many errors and 1 result - # Will fail on 1st error and return fut = Future() res = ListOffsetsResponse[1]([ - ("topic", [(0, 43, -1, -1)]), - ("topic", [(1, 6, -1, -1)]), - ("topic", [(2, 3, -1, -1)]), + ("topic", [(0, 43, -1, -1)]), # not retriable + ("topic", [(1, 6, -1, -1)]), # retriable + ("topic", [(2, 3, -1, -1)]), # retriable ("topic", [(3, 0, 1000, 9999)]) ]) fetcher._handle_list_offsets_response(fut, res) - assert fut.failed() - assert isinstance(fut.exception, NotLeaderForPartitionError) + assert fut.succeeded() + assert fut.value == ({TopicPartition("topic", 3): OffsetAndTimestamp(9999, 1000, -1)}, + set([TopicPartition("topic", 1), TopicPartition("topic", 2)])) def test__handle_list_offsets_response_v2_v3(fetcher, mocker): @@ -318,7 +313,7 @@ def test__handle_list_offsets_response_v2_v3(fetcher, mocker): ]) fetcher._handle_list_offsets_response(fut, res) assert fut.succeeded() - assert fut.value == {TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, -1)} + assert fut.value == ({TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, -1)}, set()) # v3 response is the same format fut = Future() @@ -328,7 +323,7 @@ def test__handle_list_offsets_response_v2_v3(fetcher, mocker): ]) fetcher._handle_list_offsets_response(fut, res) assert fut.succeeded() - assert fut.value == {TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, -1)} + assert fut.value == ({TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, -1)}, set()) def test__handle_list_offsets_response_v4_v5(fetcher, mocker): @@ -340,7 +335,7 @@ def test__handle_list_offsets_response_v4_v5(fetcher, mocker): ]) fetcher._handle_list_offsets_response(fut, res) assert fut.succeeded() - assert fut.value == {TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, 1234)} + assert fut.value == ({TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, 1234)}, set()) # v5 response is the same format fut = Future() @@ -350,7 +345,7 @@ def test__handle_list_offsets_response_v4_v5(fetcher, mocker): ]) fetcher._handle_list_offsets_response(fut, res) assert fut.succeeded() - assert fut.value == {TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, 1234)} + assert fut.value == ({TopicPartition("topic", 0): OffsetAndTimestamp(9999, 1000, 1234)}, set()) def test_fetched_records(fetcher, topic, mocker): @@ -628,15 +623,19 @@ def test_partition_records_compacted_offset(mocker): assert msgs[0].offset == fetch_offset + 1 -def test_update_fetch_positions_paused(subscription_state, client, mocker): +def test_reset_offsets_paused(subscription_state, client, mocker): fetcher = Fetcher(client, subscription_state) tp = TopicPartition('foo', 0) subscription_state.assign_from_user([tp]) subscription_state.pause(tp) # paused partition does not have a valid position - subscription_state.need_offset_reset(tp, OffsetResetStrategy.LATEST) + subscription_state.request_offset_reset(tp, OffsetResetStrategy.LATEST) - mocker.patch.object(fetcher, '_retrieve_offsets', return_value={tp: OffsetAndTimestamp(10, 1, -1)}) - fetcher.update_fetch_positions([tp]) + fetched_offsets = {tp: OffsetAndTimestamp(10, 1, -1)} + mocker.patch.object(fetcher._client, 'ready', return_value=True) + mocker.patch.object(fetcher, '_send_list_offsets_request', + return_value=Future().success((fetched_offsets, set()))) + mocker.patch.object(fetcher._client.cluster, "leader_for_partition", return_value=0) + fetcher.reset_offsets_if_needed() assert not subscription_state.is_offset_reset_needed(tp) assert not subscription_state.is_fetchable(tp) # because tp is paused @@ -644,14 +643,19 @@ def test_update_fetch_positions_paused(subscription_state, client, mocker): assert subscription_state.position(tp) == OffsetAndMetadata(10, '', -1) -def test_update_fetch_positions_paused_without_valid(subscription_state, client, mocker): +def test_reset_offsets_paused_without_valid(subscription_state, client, mocker): fetcher = Fetcher(client, subscription_state) tp = TopicPartition('foo', 0) subscription_state.assign_from_user([tp]) subscription_state.pause(tp) # paused partition does not have a valid position + subscription_state.reset_missing_positions() - mocker.patch.object(fetcher, '_retrieve_offsets', return_value={tp: OffsetAndTimestamp(0, 1, -1)}) - fetcher.update_fetch_positions([tp]) + fetched_offsets = {tp: OffsetAndTimestamp(0, 1, -1)} + mocker.patch.object(fetcher._client, 'ready', return_value=True) + mocker.patch.object(fetcher, '_send_list_offsets_request', + return_value=Future().success((fetched_offsets, set()))) + mocker.patch.object(fetcher._client.cluster, "leader_for_partition", return_value=0) + fetcher.reset_offsets_if_needed() assert not subscription_state.is_offset_reset_needed(tp) assert not subscription_state.is_fetchable(tp) # because tp is paused @@ -659,16 +663,16 @@ def test_update_fetch_positions_paused_without_valid(subscription_state, client, assert subscription_state.position(tp) == OffsetAndMetadata(0, '', -1) -def test_update_fetch_positions_paused_with_valid(subscription_state, client, mocker): +def test_reset_offsets_paused_with_valid(subscription_state, client, mocker): fetcher = Fetcher(client, subscription_state) tp = TopicPartition('foo', 0) subscription_state.assign_from_user([tp]) - subscription_state.assignment[tp].committed = OffsetAndMetadata(0, '', -1) - subscription_state.seek(tp, 10) + subscription_state.seek(tp, 0) + subscription_state.assignment[tp].position = OffsetAndMetadata(10, '', -1) subscription_state.pause(tp) # paused partition already has a valid position - mocker.patch.object(fetcher, '_retrieve_offsets', return_value={tp: OffsetAndTimestamp(0, 1, -1)}) - fetcher.update_fetch_positions([tp]) + mocker.patch.object(fetcher, '_fetch_offsets_by_times', return_value={tp: OffsetAndTimestamp(0, 1, -1)}) + fetcher.reset_offsets_if_needed() assert not subscription_state.is_offset_reset_needed(tp) assert not subscription_state.is_fetchable(tp) # because tp is paused From 9188385cefc1978d4c12b74459dcce78e43e42ff Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 23 Apr 2025 16:37:29 -0700 Subject: [PATCH 141/205] Fix ElectionNotNeededError handling in admin client --- kafka/admin/client.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 94de5a863..5bbc99f30 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -15,7 +15,7 @@ from kafka.coordinator.protocol import ConsumerProtocolMemberMetadata, ConsumerProtocolMemberAssignment, ConsumerProtocol import kafka.errors as Errors from kafka.errors import ( - IncompatibleBrokerVersion, KafkaConfigurationError, NotControllerError, UnknownTopicOrPartitionError, + IncompatibleBrokerVersion, KafkaConfigurationError, UnknownTopicOrPartitionError, UnrecognizedBrokerVersion, IllegalArgumentError) from kafka.metrics import MetricConfig, Metrics from kafka.protocol.admin import ( @@ -411,7 +411,7 @@ def _parse_topic_request_response(self, topic_error_tuples, request, response, t # extra values (usually the error_message) for topic, error_code in map(lambda e: e[:2], topic_error_tuples): error_type = Errors.for_code(error_code) - if tries and error_type is NotControllerError: + if tries and error_type is Errors.NotControllerError: # No need to inspect the rest of the errors for # non-retriable errors because NotControllerError should # either be thrown for all errors or no errors. @@ -431,13 +431,13 @@ def _parse_topic_partition_request_response(self, request, response, tries): for topic, partition_results in response.replication_election_results: for partition_id, error_code in map(lambda e: e[:2], partition_results): error_type = Errors.for_code(error_code) - if tries and error_type is NotControllerError: + if tries and error_type is Errors.NotControllerError: # No need to inspect the rest of the errors for # non-retriable errors because NotControllerError should # either be thrown for all errors or no errors. self._refresh_controller_id() return False - elif error_type not in [Errors.NoError, Errors.ElectionNotNeeded]: + elif error_type not in (Errors.NoError, Errors.ElectionNotNeededError): raise error_type( "Request '{}' failed with response '{}'." .format(request, response)) From 92037ce0b83793e274e3be6942cac61f4399ed17 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 24 Apr 2025 14:29:34 -0700 Subject: [PATCH 142/205] Do not include log_start_offset in producer RecordMetadata --- kafka/producer/future.py | 6 +++--- kafka/producer/record_accumulator.py | 9 +++------ kafka/producer/sender.py | 20 +++++++++----------- test/test_record_accumulator.py | 5 ++--- test/test_sender.py | 12 ++++++------ 5 files changed, 23 insertions(+), 29 deletions(-) diff --git a/kafka/producer/future.py b/kafka/producer/future.py index 07fa4adb4..f67db0979 100644 --- a/kafka/producer/future.py +++ b/kafka/producer/future.py @@ -38,7 +38,7 @@ def __init__(self, produce_future, relative_offset, timestamp_ms, checksum, seri produce_future.add_errback(self.failure) def _produce_success(self, offset_and_timestamp): - offset, produce_timestamp_ms, log_start_offset = offset_and_timestamp + offset, produce_timestamp_ms = offset_and_timestamp # Unpacking from args tuple is minor speed optimization (relative_offset, timestamp_ms, checksum, @@ -51,7 +51,7 @@ def _produce_success(self, offset_and_timestamp): 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, log_start_offset, + metadata = RecordMetadata(tp[0], tp[1], tp, offset, timestamp_ms, checksum, serialized_key_size, serialized_value_size, serialized_header_size) self.success(metadata) @@ -67,5 +67,5 @@ def get(self, timeout=None): RecordMetadata = collections.namedtuple( - 'RecordMetadata', ['topic', 'partition', 'topic_partition', 'offset', 'timestamp', 'log_start_offset', + 'RecordMetadata', ['topic', 'partition', 'topic_partition', 'offset', 'timestamp', 'checksum', 'serialized_key_size', 'serialized_value_size', 'serialized_header_size']) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 83802ef96..a9695f367 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -84,14 +84,11 @@ def done(self, base_offset=None, timestamp_ms=None, exception=None, log_start_of log.warning('Batch is already closed -- ignoring batch.done()') return elif exception is None: - log.debug("Produced messages to topic-partition %s with base offset" - " %s log start offset %s.", self.topic_partition, base_offset, - log_start_offset) # trace - self.produce_future.success((base_offset, timestamp_ms, log_start_offset)) + log.debug("Produced messages to topic-partition %s with base offset %s", self.topic_partition, base_offset) + self.produce_future.success((base_offset, timestamp_ms)) else: log.warning("Failed to produce messages to topic-partition %s with base offset" - " %s log start offset %s and error %s.", self.topic_partition, base_offset, - log_start_offset, exception) # trace + " %s: %s", self.topic_partition, base_offset, exception) self.produce_future.failure(exception) def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full, now=None): diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 0e3806175..1f2ad2d38 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -349,28 +349,27 @@ def _handle_produce_response(self, node_id, send_time, batches, response): for topic, partitions in response.topics: for partition_info in partitions: - log_start_offset = None if response.API_VERSION < 2: partition, error_code, offset = partition_info ts = None elif 2 <= response.API_VERSION <= 4: partition, error_code, offset, ts = partition_info elif 5 <= response.API_VERSION <= 7: - partition, error_code, offset, ts, log_start_offset = partition_info + partition, error_code, offset, ts, _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, offset, ts, _log_start_offset, _record_errors, _global_error = 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, log_start_offset=log_start_offset) + self._complete_batch(batch, error, offset, timestamp_ms=ts) else: # this is the acks = 0 case, just complete all requests for batch in batches: self._complete_batch(batch, None, -1) - def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None, log_start_offset=None): + 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 \ @@ -392,12 +391,12 @@ def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None, log elif self._transaction_manager.is_transactional(): self._transaction_manager.transition_to_abortable_error(exception) - batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, exception=exception, log_start_offset=log_start_offset) + batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, exception=exception) self._accumulator.deallocate(batch) if self._sensors: self._sensors.record_errors(batch.topic_partition.topic, batch.record_count) - def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_start_offset=None): + def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): """Complete or retry the given batch of records. Arguments: @@ -405,7 +404,6 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star 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 - log_start_offset (int, optional): The start offset of the log at the time this produce response was created """ # Standardize no-error to None if error is Errors.NoError: @@ -433,13 +431,13 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star 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, log_start_offset=log_start_offset) + self._fail_batch(batch, error, base_offset=base_offset, timestamp_ms=timestamp_ms) 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, log_start_offset=log_start_offset) + self._fail_batch(batch, error, base_offset=base_offset, timestamp_ms=timestamp_ms) if error is Errors.UnknownTopicOrPartitionError: log.warning("%s: Received unknown topic or partition error in produce request on partition %s." @@ -450,7 +448,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_star self._metadata.request_update() else: - batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, log_start_offset=log_start_offset) + batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms) self._accumulator.deallocate(batch) if self._transaction_manager and self._transaction_manager.producer_id_and_epoch.match(batch): diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py index 42f980712..205883cd9 100644 --- a/test/test_record_accumulator.py +++ b/test/test_record_accumulator.py @@ -32,7 +32,7 @@ def test_producer_batch_try_append(magic): 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, log_start_offset=0) + 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: @@ -44,8 +44,7 @@ def test_producer_batch_try_append(magic): expected_metadata = RecordMetadata( topic=tp[0], partition=tp[1], topic_partition=tp, - offset=123, timestamp=456, log_start_offset=0, - checksum=checksum, + offset=123, timestamp=456, checksum=checksum, serialized_key_size=3, serialized_value_size=5, serialized_header_size=-1) assert future.value == expected_metadata diff --git a/test/test_sender.py b/test/test_sender.py index ee057ff3a..229b0fe5a 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -92,11 +92,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, log_start_offset=456) + sender._complete_batch(batch, None, 0, timestamp_ms=123) assert batch.is_done assert batch.produce_future.is_done assert batch.produce_future.succeeded() - assert batch.produce_future.value == (0, 123, 456) + assert batch.produce_future.value == (0, 123) def test_complete_batch_transaction(sender, transaction_manager): @@ -201,8 +201,8 @@ def test_fail_batch(sender, accumulator, transaction_manager, mocker): 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, log_start_offset=None) - batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error, log_start_offset=None) + 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) def test_out_of_order_sequence_number_reset_producer_id(sender, accumulator, transaction_manager, mocker): @@ -213,9 +213,9 @@ def test_out_of_order_sequence_number_reset_producer_id(sender, accumulator, tra 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, log_start_offset=None) + sender._fail_batch(batch, base_offset=0, timestamp_ms=None, exception=error) sender._transaction_manager.reset_producer_id.assert_called_once() - batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error, log_start_offset=None) + batch.done.assert_called_with(base_offset=0, timestamp_ms=None, exception=error) def test_handle_produce_response(): From 186d480f80bcfe8daa7b1544afbaf5e6cb4eea4b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 24 Apr 2025 14:32:19 -0700 Subject: [PATCH 143/205] KAFKA-5429 - Ignore produce response if batch was previously aborted --- kafka/producer/record_accumulator.py | 43 +++++++++++++++++++--- test/test_record_accumulator.py | 53 +++++++++++++++++++++++++++- 2 files changed, 90 insertions(+), 6 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index a9695f367..b2af8dcc6 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -6,6 +6,13 @@ 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.record.memory_records import MemoryRecordsBuilder @@ -34,6 +41,12 @@ 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): self.max_record_size = 0 @@ -47,6 +60,7 @@ def __init__(self, tp, records, now=None): self.topic_partition = tp self.produce_future = FutureProduceResult(tp) self._retry = False + self._final_state = None @property def record_count(self): @@ -79,10 +93,29 @@ def try_append(self, timestamp_ms, key, value, headers, now=None): sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1) return future - def done(self, base_offset=None, timestamp_ms=None, exception=None, log_start_offset=None): - if self.produce_future.is_done: - log.warning('Batch is already closed -- ignoring batch.done()') + 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): + if self._final_state is None: + self._final_state = FinalState.SUCCEEDED if exception is None else FinalState.FAILED + elif self._final_state is FinalState.ABORTED: + log.debug("ProduceResponse returned for %s after batch had already been aborted.", self.topic_partition) return + else: + raise Errors.IllegalStateError("Batch has already been completed in final state %s" % self._final_state) + + self._complete_future(base_offset, timestamp_ms, exception) + + 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: log.debug("Produced messages to topic-partition %s with base offset %s", self.topic_partition, base_offset) self.produce_future.success((base_offset, timestamp_ms)) @@ -588,7 +621,7 @@ def _abort_batches(self, error): with self._tp_locks[tp]: batch.records.close() self._batches[tp].remove(batch) - batch.done(exception=error) + batch.abort(error) self.deallocate(batch) def abort_undrained_batches(self, error): @@ -601,7 +634,7 @@ def abort_undrained_batches(self, error): batch.records.close() self._batches[tp].remove(batch) if aborted: - batch.done(exception=error) + batch.abort(error) self.deallocate(batch) def close(self): diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py index 205883cd9..738d12c83 100644 --- a/test/test_record_accumulator.py +++ b/test/test_record_accumulator.py @@ -4,7 +4,7 @@ import pytest import io -from kafka.errors import KafkaTimeoutError +from kafka.errors import IllegalStateError, KafkaError, KafkaTimeoutError from kafka.producer.future import FutureRecordMetadata, RecordMetadata from kafka.producer.record_accumulator import RecordAccumulator, ProducerBatch from kafka.record.memory_records import MemoryRecordsBuilder @@ -72,3 +72,54 @@ def test_producer_batch_maybe_expire(): assert future.is_done assert future.failed() assert isinstance(future.exception, KafkaTimeoutError) + +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 From 17ba216b7655bacac28bac965c76d3c4ed17a4bd Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 24 Apr 2025 14:33:07 -0700 Subject: [PATCH 144/205] from __future__ import division for produce batch time calcs --- kafka/producer/record_accumulator.py | 24 ++++++++++-------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index b2af8dcc6..03e37a5f6 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -1,4 +1,4 @@ -from __future__ import absolute_import +from __future__ import absolute_import, division import collections import copy @@ -138,9 +138,9 @@ def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full, """ now = time.time() if now is None else now since_append = now - self.last_append - since_ready = now - (self.created + linger_ms / 1000.0) - since_backoff = now - (self.last_attempt + retry_backoff_ms / 1000.0) - timeout = request_timeout_ms / 1000.0 + since_ready = now - (self.created + linger_ms / 1000) + since_backoff = now - (self.last_attempt + retry_backoff_ms / 1000) + timeout = request_timeout_ms / 1000 error = None if not self.in_retry() and is_full and timeout < since_append: @@ -431,10 +431,10 @@ def ready(self, cluster, now=None): if not dq: continue batch = dq[0] - retry_backoff = self.config['retry_backoff_ms'] / 1000.0 - linger = self.config['linger_ms'] / 1000.0 - backing_off = bool(batch.attempts > 0 and - batch.last_attempt + retry_backoff > now) + retry_backoff = self.config['retry_backoff_ms'] / 1000 + linger = self.config['linger_ms'] / 1000 + backing_off = bool(batch.attempts > 0 + and (batch.last_attempt + retry_backoff) > now) waited_time = now - batch.last_attempt time_to_wait = retry_backoff if backing_off else linger time_left = max(time_to_wait - waited_time, 0) @@ -499,12 +499,8 @@ def drain(self, cluster, nodes, max_size, now=None): dq = self._batches[tp] if dq: first = dq[0] - backoff = ( - bool(first.attempts > 0) and - bool(first.last_attempt + - self.config['retry_backoff_ms'] / 1000.0 - > now) - ) + backoff = bool(first.attempts > 0 and + first.last_attempt + self.config['retry_backoff_ms'] / 1000 > now) # Only drain the batch if it is not during backoff if not backoff: if (size + first.records.size_in_bytes() > max_size From 3ee0419eb1c318bf1d9f42306ffee130231f4760 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 10:41:39 -0700 Subject: [PATCH 145/205] DefaultRecordsBuilder.size_in_bytes classmethod --- kafka/record/default_records.py | 13 +++++++------ test/record/test_default_records.py | 4 ++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index 91d4a9d62..b495c76fe 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -679,14 +679,15 @@ def size(self): """ return len(self._buffer) - def size_in_bytes(self, offset, timestamp, key, value, headers): - if self._first_timestamp is not None: - timestamp_delta = timestamp - self._first_timestamp - else: - timestamp_delta = 0 + @classmethod + def header_size_in_bytes(self): + return self.HEADER_STRUCT.size + + @classmethod + def size_in_bytes(self, offset_delta, timestamp_delta, key, value, headers): size_of_body = ( 1 + # Attrs - size_of_varint(offset) + + size_of_varint(offset_delta) + size_of_varint(timestamp_delta) + self.size_of(key, value, headers) ) diff --git a/test/record/test_default_records.py b/test/record/test_default_records.py index 79d3975a5..540705d50 100644 --- a/test/record/test_default_records.py +++ b/test/record/test_default_records.py @@ -57,8 +57,8 @@ def test_written_bytes_equals_size_in_bytes_v2(): producer_id=-1, producer_epoch=-1, base_sequence=-1, batch_size=999999) - size_in_bytes = builder.size_in_bytes( - 0, timestamp=9999999, key=key, value=value, headers=headers) + size_in_bytes = DefaultRecordBatchBuilder.size_in_bytes( + offset_delta=0, timestamp_delta=0, key=key, value=value, headers=headers) pos = builder.size() meta = builder.append( From 1f635141c526e833f9487169f53f52e52c27e19a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 10:42:11 -0700 Subject: [PATCH 146/205] MemoryRecords iterator; MemoryRecordsBuilder records() helper --- kafka/record/memory_records.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/kafka/record/memory_records.py b/kafka/record/memory_records.py index 4bf3115c8..9df733059 100644 --- a/kafka/record/memory_records.py +++ b/kafka/record/memory_records.py @@ -109,6 +109,16 @@ def next_batch(self, _min_slice=MIN_SLICE, else: return DefaultRecordBatch(next_slice) + def __iter__(self): + return self + + def __next__(self): + if not self.has_next(): + raise StopIteration + return self.next_batch() + + next = __next__ + class MemoryRecordsBuilder(object): @@ -186,6 +196,10 @@ def producer_id(self): def producer_epoch(self): return self._producer_epoch + def records(self): + assert self._closed + return MemoryRecords(self._buffer) + def close(self): # This method may be called multiple times on the same batch # i.e., on retries From 3b766e2c409d103789f327e208567df6b35104b5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 11:05:44 -0700 Subject: [PATCH 147/205] KIP-91: KafkaProducer delivery_timeout_ms --- kafka/producer/kafka.py | 22 +- kafka/producer/record_accumulator.py | 345 ++++++++++++++------------- kafka/producer/sender.py | 85 ++++++- test/test_record_accumulator.py | 181 ++++++++++++-- 4 files changed, 426 insertions(+), 207 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 9bb958138..42baae59c 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -106,7 +106,16 @@ class KafkaProducer(object): defaults to be suitable. If the values are set to something incompatible with the idempotent producer, a KafkaConfigurationError will be raised. - + delivery_timeout_ms (float): An upper bound on the time to report success + or failure after producer.send() returns. This limits the total time + that a record will be delayed prior to sending, the time to await + acknowledgement from the broker (if expected), and the time allowed + for retriable send failures. The producer may report failure to send + a record earlier than this config if either an unrecoverable error is + encountered, the retries have been exhausted, or the record is added + to a batch which reached an earlier delivery expiration deadline. + The value of this config should be greater than or equal to the + sum of (request_timeout_ms + linger_ms). Default: 120000. acks (0, 1, 'all'): The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the durability of records that are sent. The @@ -142,8 +151,12 @@ class KafkaProducer(object): potentially change the ordering of records because if two batches are sent to a single partition, and the first fails and is retried but the second succeeds, then the records in the second batch may - appear first. - Default: 0. + appear first. Note additionally that produce requests will be + failed before the number of retries has been exhausted if the timeout + configured by delivery_timeout_ms expires first before successful + acknowledgement. Users should generally prefer to leave this config + unset and instead use delivery_timeout_ms to control retry behavior. + Default: 2147483647 (java max int). batch_size (int): Requests sent to brokers will contain multiple batches, one for each partition with data available to be sent. A small batch size will make batching less common and may reduce @@ -320,10 +333,11 @@ class KafkaProducer(object): 'enable_idempotence': False, 'transactional_id': None, 'transaction_timeout_ms': 60000, + 'delivery_timeout_ms': 120000, 'acks': 1, 'bootstrap_topics_filter': set(), 'compression_type': None, - 'retries': 0, + 'retries': 2147483647, 'batch_size': 16384, 'linger_ms': 0, 'partitioner': DefaultPartitioner(), diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 03e37a5f6..446d87272 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -49,8 +49,8 @@ class FinalState(IntEnum): class ProducerBatch(object): def __init__(self, tp, records, now=None): - self.max_record_size = 0 now = time.time() if now is None else now + self.max_record_size = 0 self.created = now self.drained = None self.attempts = 0 @@ -62,6 +62,10 @@ def __init__(self, tp, records, now=None): 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() @@ -86,11 +90,14 @@ def try_append(self, timestamp_ms, key, value, headers, now=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) + 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): @@ -103,66 +110,66 @@ def abort(self, 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 = FinalState.SUCCEEDED if exception is None else FinalState.FAILED - elif self._final_state is FinalState.ABORTED: - log.debug("ProduceResponse returned for %s after batch had already been aborted.", self.topic_partition) - return - else: - raise Errors.IllegalStateError("Batch has already been completed in final state %s" % self._final_state) + 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 - self._complete_future(base_offset, timestamp_ms, exception) + 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: - log.debug("Produced messages to topic-partition %s with base offset %s", self.topic_partition, base_offset) self.produce_future.success((base_offset, timestamp_ms)) else: - log.warning("Failed to produce messages to topic-partition %s with base offset" - " %s: %s", self.topic_partition, base_offset, exception) self.produce_future.failure(exception) - def maybe_expire(self, request_timeout_ms, retry_backoff_ms, linger_ms, is_full, now=None): - """Expire batches if metadata is not available - - A batch whose metadata is not available should be expired if one - of the following is true: - - * the batch is not in retry AND request timeout has elapsed after - it is ready (full or linger.ms has reached). - - * the batch is in retry AND request timeout has elapsed after the - backoff period ended. - """ + def has_reached_delivery_timeout(self, delivery_timeout_ms, now=None): now = time.time() if now is None else now - since_append = now - self.last_append - since_ready = now - (self.created + linger_ms / 1000) - since_backoff = now - (self.last_attempt + retry_backoff_ms / 1000) - timeout = request_timeout_ms / 1000 - - error = None - if not self.in_retry() and is_full and timeout < since_append: - error = "%d seconds have passed since last append" % (since_append,) - elif not self.in_retry() and timeout < since_ready: - error = "%d seconds have passed since batch creation plus linger time" % (since_ready,) - elif self.in_retry() and timeout < since_backoff: - error = "%d seconds have passed since last attempt plus backoff time" % (since_backoff,) - - if error: - self.records.close() - self.done(base_offset=-1, exception=Errors.KafkaTimeoutError( - "Batch for %s containing %s record(s) expired: %s" % ( - self.topic_partition, self.records.next_offset(), error))) - return True - return False + return delivery_timeout_ms / 1000 <= now - self.created def in_retry(self): return self._retry - def set_retry(self): + 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): @@ -207,9 +214,11 @@ class RecordAccumulator(object): 'batch_size': 16384, 'compression_attrs': 0, 'linger_ms': 0, + 'request_timeout_ms': 30000, + 'delivery_timeout_ms': 120000, 'retry_backoff_ms': 100, 'transaction_manager': None, - 'message_version': 0, + 'message_version': 2, } def __init__(self, **configs): @@ -229,8 +238,20 @@ def __init__(self, **configs): # so we don't need to protect them w/ locking. self.muted = set() self._drain_index = 0 + self._next_batch_expiry_time_ms = float('inf') + + if self.config['delivery_timeout_ms'] < self.config['linger_ms'] + self.config['request_timeout_ms']: + raise Errors.KafkaConfigurationError("Must set delivery_timeout_ms higher than linger_ms + request_timeout_ms") + + @property + def delivery_timeout_ms(self): + return self.config['delivery_timeout_ms'] + + @property + def next_expiry_time_ms(self): + return self._next_batch_expiry_time_ms - def append(self, tp, timestamp_ms, key, value, headers): + def append(self, tp, timestamp_ms, key, value, headers, now=None): """Add a record to the accumulator, return the append result. The append result will contain the future metadata, and flag for @@ -249,6 +270,7 @@ def append(self, tp, timestamp_ms, key, value, headers): """ assert isinstance(tp, TopicPartition), 'not TopicPartition' assert not self._closed, 'RecordAccumulator is closed' + now = time.time() if now is None else now # We keep track of the number of appending thread to make sure we do # not miss batches in abortIncompleteBatches(). self._appends_in_progress.increment() @@ -263,7 +285,7 @@ def append(self, tp, timestamp_ms, key, value, headers): dq = self._batches[tp] if dq: last = dq[-1] - future = last.try_append(timestamp_ms, key, value, headers) + future = last.try_append(timestamp_ms, key, value, headers, now=now) if future is not None: batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False @@ -275,7 +297,7 @@ def append(self, tp, timestamp_ms, key, value, headers): if dq: last = dq[-1] - future = last.try_append(timestamp_ms, key, value, headers) + future = last.try_append(timestamp_ms, key, value, headers, now=now) if future is not None: # Somebody else found us a batch, return the one we # waited for! Hopefully this doesn't happen often... @@ -292,8 +314,8 @@ def append(self, tp, timestamp_ms, key, value, headers): self.config['batch_size'] ) - batch = ProducerBatch(tp, records) - future = batch.try_append(timestamp_ms, key, value, headers) + batch = ProducerBatch(tp, records, now=now) + future = batch.try_append(timestamp_ms, key, value, headers, now=now) if not future: raise Exception() @@ -304,72 +326,36 @@ def append(self, tp, timestamp_ms, key, value, headers): finally: self._appends_in_progress.decrement() - def abort_expired_batches(self, request_timeout_ms, cluster): - """Abort the batches that have been sitting in RecordAccumulator for - more than the configured request_timeout due to metadata being - unavailable. - - Arguments: - request_timeout_ms (int): milliseconds to timeout - cluster (ClusterMetadata): current metadata for kafka cluster + def maybe_update_next_batch_expiry_time(self, batch): + self._next_batch_expiry_time_ms = min(self._next_batch_expiry_time_ms, batch.created * 1000 + self.delivery_timeout_ms) - Returns: - list of ProducerBatch that were expired - """ + def expired_batches(self, now=None): + """Get a list of batches which have been sitting in the accumulator too long and need to be expired.""" expired_batches = [] - to_remove = [] - count = 0 for tp in list(self._batches.keys()): assert tp in self._tp_locks, 'TopicPartition not in locks dict' - - # We only check if the batch should be expired if the partition - # does not have a batch in flight. This is to avoid the later - # batches get expired when an earlier batch is still in progress. - # This protection only takes effect when user sets - # max.in.flight.request.per.connection=1. Otherwise the expiration - # order is not guranteed. - if tp in self.muted: - continue - with self._tp_locks[tp]: # iterate over the batches and expire them if they have stayed # in accumulator for more than request_timeout_ms dq = self._batches[tp] - for batch in dq: - is_full = bool(bool(batch != dq[-1]) or batch.records.is_full()) - # check if the batch is expired - if batch.maybe_expire(request_timeout_ms, - self.config['retry_backoff_ms'], - self.config['linger_ms'], - is_full): + while dq: + batch = dq[0] + if batch.has_reached_delivery_timeout(self.delivery_timeout_ms, now=now): + dq.popleft() + batch.records.close() expired_batches.append(batch) - to_remove.append(batch) - count += 1 - self.deallocate(batch) else: # Stop at the first batch that has not expired. + self.maybe_update_next_batch_expiry_time(batch) break - - # Python does not allow us to mutate the dq during iteration - # Assuming expired batches are infrequent, this is better than - # creating a new copy of the deque for iteration on every loop - if to_remove: - for batch in to_remove: - dq.remove(batch) - to_remove = [] - - if expired_batches: - log.warning("Expired %d batches in accumulator", count) # trace - return expired_batches def reenqueue(self, batch, now=None): - """Re-enqueue the given record batch in the accumulator to retry.""" - now = time.time() if now is None else now - batch.attempts += 1 - batch.last_attempt = now - batch.last_append = now - batch.set_retry() + """ + Re-enqueue the given record batch in the accumulator. In Sender.completeBatch method, we check + whether the batch has reached deliveryTimeoutMs or not. Hence we do not do the delivery timeout check here. + """ + batch.retry(now=now) assert batch.topic_partition in self._tp_locks, 'TopicPartition not in locks dict' assert batch.topic_partition in self._batches, 'TopicPartition not in batches' dq = self._batches[batch.topic_partition] @@ -465,6 +451,88 @@ def has_undrained(self): return True return False + def _should_stop_drain_batches_for_partition(self, first, tp): + if self._transaction_manager: + if not self._transaction_manager.is_send_to_partition_allowed(tp): + return True + if not self._transaction_manager.producer_id_and_epoch.is_valid: + # we cannot send the batch until we have refreshed the PID + log.debug("Waiting to send ready batches because transaction producer id is not valid") + return True + return False + + def drain_batches_for_one_node(self, cluster, node_id, max_size, now=None): + now = time.time() if now is None else now + size = 0 + ready = [] + partitions = list(cluster.partitions_for_broker(node_id)) + if not partitions: + return ready + # to make starvation less likely this loop doesn't start at 0 + self._drain_index %= len(partitions) + start = None + while start != self._drain_index: + tp = partitions[self._drain_index] + if start is None: + start = self._drain_index + self._drain_index += 1 + self._drain_index %= len(partitions) + + # Only proceed if the partition has no in-flight batches. + if tp in self.muted: + continue + + if tp not in self._batches: + continue + + with self._tp_locks[tp]: + dq = self._batches[tp] + if len(dq) == 0: + continue + first = dq[0] + backoff = bool(first.attempts > 0 and + first.last_attempt + self.config['retry_backoff_ms'] / 1000 > now) + # Only drain the batch if it is not during backoff + if backoff: + continue + + if (size + first.records.size_in_bytes() > max_size + and len(ready) > 0): + # there is a rare case that a single batch + # size is larger than the request size due + # to compression; in this case we will + # still eventually send this batch in a + # single request + break + else: + if self._should_stop_drain_batches_for_partition(first, tp): + break + + batch = dq.popleft() + if self._transaction_manager and not batch.in_retry(): + # If the batch is in retry, then we should not change the pid and + # sequence number, since this may introduce duplicates. In particular, + # the previous attempt may actually have been accepted, and if we change + # the pid and sequence here, this attempt will also be accepted, causing + # a duplicate. + sequence_number = self._transaction_manager.sequence_number(batch.topic_partition) + log.debug("Dest: %s: %s producer_id=%s epoch=%s sequence=%s", + node_id, batch.topic_partition, + self._transaction_manager.producer_id_and_epoch.producer_id, + self._transaction_manager.producer_id_and_epoch.epoch, + sequence_number) + batch.records.set_producer_state( + self._transaction_manager.producer_id_and_epoch.producer_id, + self._transaction_manager.producer_id_and_epoch.epoch, + sequence_number, + self._transaction_manager.is_transactional() + ) + batch.records.close() + size += batch.records.size_in_bytes() + ready.append(batch) + batch.drained = now + return ready + def drain(self, cluster, nodes, max_size, now=None): """ Drain all the data for the given nodes and collate them into a list of @@ -486,70 +554,7 @@ def drain(self, cluster, nodes, max_size, now=None): now = time.time() if now is None else now batches = {} for node_id in nodes: - size = 0 - partitions = list(cluster.partitions_for_broker(node_id)) - ready = [] - # to make starvation less likely this loop doesn't start at 0 - self._drain_index %= len(partitions) - start = self._drain_index - while True: - tp = partitions[self._drain_index] - if tp in self._batches and tp not in self.muted: - with self._tp_locks[tp]: - dq = self._batches[tp] - if dq: - first = dq[0] - backoff = bool(first.attempts > 0 and - first.last_attempt + self.config['retry_backoff_ms'] / 1000 > now) - # Only drain the batch if it is not during backoff - if not backoff: - if (size + first.records.size_in_bytes() > max_size - and len(ready) > 0): - # there is a rare case that a single batch - # size is larger than the request size due - # to compression; in this case we will - # still eventually send this batch in a - # single request - break - else: - producer_id_and_epoch = None - if self._transaction_manager: - if not self._transaction_manager.is_send_to_partition_allowed(tp): - break - producer_id_and_epoch = self._transaction_manager.producer_id_and_epoch - if not producer_id_and_epoch.is_valid: - # we cannot send the batch until we have refreshed the PID - log.debug("Waiting to send ready batches because transaction producer id is not valid") - break - - batch = dq.popleft() - if producer_id_and_epoch and not batch.in_retry(): - # If the batch is in retry, then we should not change the pid and - # sequence number, since this may introduce duplicates. In particular, - # the previous attempt may actually have been accepted, and if we change - # the pid and sequence here, this attempt will also be accepted, causing - # a duplicate. - sequence_number = self._transaction_manager.sequence_number(batch.topic_partition) - log.debug("Dest: %s: %s producer_id=%s epoch=%s sequence=%s", - node_id, batch.topic_partition, producer_id_and_epoch.producer_id, producer_id_and_epoch.epoch, - sequence_number) - batch.records.set_producer_state( - producer_id_and_epoch.producer_id, - producer_id_and_epoch.epoch, - sequence_number, - self._transaction_manager.is_transactional() - ) - batch.records.close() - size += batch.records.size_in_bytes() - ready.append(batch) - batch.drained = now - - self._drain_index += 1 - self._drain_index %= len(partitions) - if start == self._drain_index: - break - - batches[node_id] = ready + batches[node_id] = self.drain_batches_for_one_node(cluster, node_id, max_size, now=now) return batches def deallocate(self, batch): diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 1f2ad2d38..1e5f9a0db 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -2,6 +2,7 @@ import collections import copy +import heapq import logging import threading import time @@ -59,6 +60,45 @@ def __init__(self, client, metadata, accumulator, **configs): else: self._sensors = None self._transaction_manager = self.config['transaction_manager'] + # A per-partition queue of batches ordered by creation time for tracking the in-flight batches + self._in_flight_batches = collections.defaultdict(list) + + def _maybe_remove_from_inflight_batches(self, batch): + try: + queue = self._in_flight_batches[batch.topic_partition] + except KeyError: + return + try: + idx = queue.index((batch.created, batch)) + except ValueError: + return + # https://stackoverflow.com/questions/10162679/python-delete-element-from-heap + queue[idx] = queue[-1] + queue.pop() + heapq.heapify(queue) + + def _get_expired_inflight_batches(self): + """Get the in-flight batches that has reached delivery timeout.""" + expired_batches = [] + to_remove = [] + for tp, queue in six.iteritems(self._in_flight_batches): + while queue: + _created_at, batch = queue[0] + if batch.has_reached_delivery_timeout(self._accumulator.delivery_timeout_ms): + heapq.heappop(queue) + if batch.final_state is None: + expired_batches.append(batch) + else: + raise Errors.IllegalStateError("%s batch created at %s gets unexpected final state %s" % (batch.topic_partition, batch.created, batch.final_state)) + else: + self._accumulator.maybe_update_next_batch_expiry_time(batch) + break + else: + # Avoid mutating in_flight_batches during iteration + to_remove.append(tp) + for tp in to_remove: + del self._in_flight_batches[tp] + return expired_batches def run(self): """The main run loop for the sender thread.""" @@ -131,7 +171,8 @@ def run_once(self): poll_timeout_ms = self._send_producer_data() self._client.poll(timeout_ms=poll_timeout_ms) - def _send_producer_data(self): + def _send_producer_data(self, now=None): + now = time.time() if now is None else now # get the list of partitions with data ready to send result = self._accumulator.ready(self._metadata) ready_nodes, next_ready_check_delay, unknown_leaders_exist = result @@ -156,14 +197,20 @@ def _send_producer_data(self): batches_by_node = self._accumulator.drain( self._metadata, ready_nodes, self.config['max_request_size']) + for batch_list in six.itervalues(batches_by_node): + for batch in batch_list: + item = (batch.created, batch) + queue = self._in_flight_batches[batch.topic_partition] + heapq.heappush(queue, item) + if self.config['guarantee_message_order']: # Mute all the partitions drained for batch_list in six.itervalues(batches_by_node): for batch in batch_list: self._accumulator.muted.add(batch.topic_partition) - expired_batches = self._accumulator.abort_expired_batches( - self.config['request_timeout_ms'], self._metadata) + expired_batches = self._accumulator.expired_batches() + expired_batches.extend(self._get_expired_inflight_batches()) if expired_batches: log.debug("%s: Expired %s batches in accumulator", str(self), len(expired_batches)) @@ -193,12 +240,18 @@ def _send_producer_data(self): requests = self._create_produce_requests(batches_by_node) # If we have any nodes that are ready to send + have sendable data, # poll with 0 timeout so this can immediately loop and try sending more - # data. Otherwise, the timeout is determined by nodes that have - # partitions with data that isn't yet sendable (e.g. lingering, backing - # off). Note that this specifically does not include nodes with + # data. Otherwise, the timeout will be the smaller value between next + # batch expiry time, and the delay time for checking data availability. + # Note that the nodes may have data that isn't yet sendable due to + # lingering, backing off, etc. This specifically does not include nodes with # sendable data that aren't ready to send since they would cause busy # looping. - poll_timeout_ms = min(next_ready_check_delay * 1000, not_ready_timeout_ms) + poll_timeout_ms = min(next_ready_check_delay * 1000, + not_ready_timeout_ms, + self._accumulator.next_expiry_time_ms - now * 1000) + if poll_timeout_ms < 0: + poll_timeout_ms = 0 + if ready_nodes: log.debug("%s: Nodes with data ready to send: %s", str(self), ready_nodes) # trace log.debug("%s: Created %d produce requests: %s", str(self), len(requests), requests) # trace @@ -391,11 +444,13 @@ def _fail_batch(self, batch, exception, base_offset=None, timestamp_ms=None): elif self._transaction_manager.is_transactional(): self._transaction_manager.transition_to_abortable_error(exception) - batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms, exception=exception) - self._accumulator.deallocate(batch) 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): + self._maybe_remove_from_inflight_batches(batch) + self._accumulator.deallocate(batch) + def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): """Complete or retry the given batch of records. @@ -424,6 +479,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): str(self), batch.topic_partition, self._transaction_manager.sequence_number(batch.topic_partition) if self._transaction_manager else None) self._accumulator.reenqueue(batch) + self._maybe_remove_from_inflight_batches(batch) if self._sensors: self._sensors.record_retries(batch.topic_partition.topic, batch.record_count) else: @@ -448,8 +504,9 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): self._metadata.request_update() else: - batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms) - self._accumulator.deallocate(batch) + if batch.done(base_offset=base_offset, timestamp_ms=timestamp_ms): + self._maybe_remove_from_inflight_batches(batch) + self._accumulator.deallocate(batch) if self._transaction_manager and self._transaction_manager.producer_id_and_epoch.match(batch): self._transaction_manager.increment_sequence_number(batch.topic_partition, batch.record_count) @@ -465,8 +522,10 @@ def _can_retry(self, batch, error): We can retry a send if the error is transient and the number of attempts taken is fewer than the maximum allowed """ - return (batch.attempts < self.config['retries'] - and getattr(error, 'retriable', False)) + return (not batch.has_reached_delivery_timeout(self._accumulator.delivery_timeout_ms) and + batch.attempts < self.config['retries'] and + batch.final_state is None and + getattr(error, 'retriable', False)) def _create_produce_requests(self, collated): """ diff --git a/test/test_record_accumulator.py b/test/test_record_accumulator.py index 738d12c83..5c7134e5c 100644 --- a/test/test_record_accumulator.py +++ b/test/test_record_accumulator.py @@ -1,16 +1,28 @@ # pylint: skip-file -from __future__ import absolute_import +from __future__ import absolute_import, division import pytest -import io -from kafka.errors import IllegalStateError, KafkaError, KafkaTimeoutError +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.record.default_records import DefaultRecordBatchBuilder from kafka.record.memory_records import MemoryRecordsBuilder from kafka.structs import TopicPartition +@pytest.fixture +def tp(): + return TopicPartition('foo', 0) + +@pytest.fixture +def cluster(tp, mocker): + metadata = ClusterMetadata() + mocker.patch.object(metadata, 'leader_for_partition', return_value=0) + mocker.patch.object(metadata, 'partitions_for_broker', return_value=[tp]) + return metadata + def test_producer_batch_producer_id(): tp = TopicPartition('foo', 0) records = MemoryRecordsBuilder( @@ -54,25 +66,9 @@ def test_producer_batch_retry(): magic=2, compression_type=0, batch_size=100000) batch = ProducerBatch(tp, records) assert not batch.in_retry() - batch.set_retry() + batch.retry() assert batch.in_retry() -def test_producer_batch_maybe_expire(): - tp = TopicPartition('foo', 0) - records = MemoryRecordsBuilder( - magic=2, compression_type=0, batch_size=100000) - batch = ProducerBatch(tp, records, now=1) - future = batch.try_append(0, b'key', b'value', [], now=2) - request_timeout_ms = 5000 - retry_backoff_ms = 200 - linger_ms = 1000 - is_full = True - batch.maybe_expire(request_timeout_ms, retry_backoff_ms, linger_ms, is_full, now=20) - assert batch.is_done - assert future.is_done - assert future.failed() - assert isinstance(future.exception, KafkaTimeoutError) - def test_batch_abort(): tp = TopicPartition('foo', 0) records = MemoryRecordsBuilder( @@ -123,3 +119,148 @@ def test_batch_cannot_complete_twice(): assert record_metadata.offset == 500 assert record_metadata.timestamp == 10 + +def test_linger(tp, cluster): + now = 0 + accum = RecordAccumulator(linger_ms=10) + accum.append(tp, 0, b'key', b'value', [], now=now) + ready, next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert len(ready) == 0, 'No partitions should be ready' + assert next_ready_check == .01 # linger_ms in secs + now += .01 + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert ready == set([0]), "Our partitions leader should be ready" + batches = accum.drain(cluster, ready, 0, 2147483647)[0] + assert len(batches) == 1 + batch = batches[0] + assert batch.records.is_full() + + parsed = list(batch.records.records()) + assert len(parsed) == 1 + records = list(parsed[0]) + assert len(records) == 1 + assert records[0].key == b'key', 'Keys should match' + assert records[0].value == b'value', 'Values should match' + +def _advance_now_ms(now, ms): + return now + ms / 1000 + 1/10000 # add extra .1 ms to each advance to avoid rounding issues when converting back to seconds + +def _do_expire_batch_single(cluster, tp, delivery_timeout_ms): + now = 0 + linger_ms = 300 + accum = RecordAccumulator(linger_ms=linger_ms, delivery_timeout_ms=delivery_timeout_ms, request_timeout_ms=(delivery_timeout_ms-linger_ms-100)) + + # Make the batches ready due to linger. These batches are not in retry + for mute in [False, True]: + accum.append(tp, 0, b'key', b'value', [], now=now) + ready, next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert len(ready) == 0, 'No partitions should be ready' + assert next_ready_check == linger_ms / 1000 + + now = _advance_now_ms(now, linger_ms) + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert ready == set([0]), "Our partitions leader should be ready" + + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 0, "The batch should not expire when just linger has passed" + + if mute: + accum.muted.add(tp) + else: + try: + accum.muted.remove(tp) + except KeyError: + pass + + # Advance the clock to expire the batch. + now = _advance_now_ms(now, delivery_timeout_ms - linger_ms) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 1, "The batch may expire when the partition is muted" + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert len(ready) == 0, "No partitions should be ready." + +def test_expired_batch_single(cluster, tp): + _do_expire_batch_single(cluster, tp, 3200) + +def test_expired_batch_single_max_value(cluster, tp): + _do_expire_batch_single(cluster, tp, 2147483647) + +def _expected_num_appends(batch_size): + size = DefaultRecordBatchBuilder.header_size_in_bytes() + offset_delta = 0 + while True: + record_size = DefaultRecordBatchBuilder.size_in_bytes(offset_delta, 0, b'key', b'value', []) + if size + record_size > batch_size: + return offset_delta + offset_delta += 1 + size += record_size + +def test_expired_batches(cluster, tp): + now = 0 + retry_backoff_ms = 100 + linger_ms = 30 + request_timeout_ms = 60 + delivery_timeout_ms = 3200 + batch_size = 1024 + accum = RecordAccumulator(linger_ms=linger_ms, delivery_timeout_ms=delivery_timeout_ms, request_timeout_ms=request_timeout_ms, retry_backoff_ms=retry_backoff_ms, batch_size=batch_size) + appends = _expected_num_appends(batch_size) + + # Test batches not in retry + for i in range(appends): + accum.append(tp, 0, b'key', b'value', [], now=now) + ready, next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert len(ready) == 0, 'No partitions should be ready' + assert next_ready_check == linger_ms / 1000 + + # Make the batches ready due to batch full + accum.append(tp, 0, b'key', b'value', [], now=now) + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert ready == set([0]), "Our partitions leader should be ready" + + # Advance the clock to expire the batch. + now = _advance_now_ms(now, delivery_timeout_ms + 1) + accum.muted.add(tp) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 2, "The batches will be expired no matter if the partition is muted or not" + + accum.muted.remove(tp) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 0, "All batches should have been expired earlier" + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert len(ready) == 0, "No partitions should be ready." + + # Test batches in retry. + # Create a retried batch + accum.append(tp, 0, b'key', b'value', [], now=now) + now = _advance_now_ms(now, linger_ms) + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert ready == set([0]), "Our partitions leader should be ready" + + drained = accum.drain(cluster, ready, 2147483647, now=now) + assert len(drained[0]) == 1, "There should be only one batch." + now = _advance_now_ms(now, 1000) + accum.reenqueue(drained[0][0], now=now) + + # test expiration. + now = _advance_now_ms(now, request_timeout_ms + retry_backoff_ms) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 0, "The batch should not be expired." + now = _advance_now_ms(now, 1) + + accum.muted.add(tp) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 0, "The batch should not be expired when the partition is muted" + + accum.muted.remove(tp) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 0, "The batch should not be expired when the partition is unmuted" + + now = _advance_now_ms(now, linger_ms) + ready, _next_ready_check, _unknown_leaders_exist = accum.ready(cluster, now=now) + assert ready == set([0]), "Our partitions leader should be ready" + + # Advance the clock to expire the batch. + now = _advance_now_ms(now, delivery_timeout_ms + 1) + accum.muted.add(tp) + expired_batches = accum.expired_batches(now=now) + assert len(expired_batches) == 1, "The batch should not be expired when the partition is muted" From 4f87a877fbf2a72fb8e312c1c13176632a347c0e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 11:42:59 -0700 Subject: [PATCH 148/205] Default retries -> infinite --- kafka/producer/kafka.py | 11 ++++------- kafka/producer/sender.py | 4 ++-- test/test_sender.py | 1 + 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 42baae59c..3253668a3 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -143,7 +143,7 @@ class KafkaProducer(object): Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression). Default: None. - retries (int): Setting a value greater than zero will cause the client + retries (numeric): Setting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error. Note that this retry is no different than if the client resent the record upon receiving the error. Allowing retries @@ -156,7 +156,7 @@ class KafkaProducer(object): configured by delivery_timeout_ms expires first before successful acknowledgement. Users should generally prefer to leave this config unset and instead use delivery_timeout_ms to control retry behavior. - Default: 2147483647 (java max int). + Default: float('inf') (infinite) batch_size (int): Requests sent to brokers will contain multiple batches, one for each partition with data available to be sent. A small batch size will make batching less common and may reduce @@ -337,7 +337,7 @@ class KafkaProducer(object): 'acks': 1, 'bootstrap_topics_filter': set(), 'compression_type': None, - 'retries': 2147483647, + 'retries': float('inf'), 'batch_size': 16384, 'linger_ms': 0, 'partitioner': DefaultPartitioner(), @@ -485,10 +485,7 @@ def __init__(self, **configs): else: log.info("%s: Instantiated an idempotent producer.", str(self)) - if 'retries' not in user_provided_configs: - log.info("%s: Overriding the default 'retries' config to 3 since the idempotent producer is enabled.", str(self)) - self.config['retries'] = 3 - elif self.config['retries'] == 0: + if self.config['retries'] == 0: raise Errors.KafkaConfigurationError("Must set 'retries' to non-zero when using the idempotent producer.") if 'max_in_flight_requests_per_connection' not in user_provided_configs: diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 1e5f9a0db..dcb3ecbdc 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -30,7 +30,7 @@ class Sender(threading.Thread): DEFAULT_CONFIG = { 'max_request_size': 1048576, 'acks': 1, - 'retries': 0, + 'retries': float('inf'), 'request_timeout_ms': 30000, 'retry_backoff_ms': 100, 'metrics': None, @@ -468,7 +468,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None): if self._can_retry(batch, error): # retry log.warning("%s: Got error produce response on topic-partition %s," - " retrying (%d attempts left). Error: %s", + " retrying (%s attempts left). Error: %s", str(self), batch.topic_partition, self.config['retries'] - batch.attempts - 1, error) diff --git a/test/test_sender.py b/test/test_sender.py index 229b0fe5a..0731454df 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -131,6 +131,7 @@ def test_complete_batch_transaction(sender, transaction_manager): def test_complete_batch_error(sender, error, refresh_metadata): sender._client.cluster._last_successful_refresh_ms = (time.time() - 10) * 1000 sender._client.cluster._need_update = False + sender.config['retries'] = 0 assert sender._client.cluster.ttl() > 0 batch = producer_batch() sender._complete_batch(batch, error, -1) From 707913fb5e2fc9e09c5fd6ce21d103ee9d952aa0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 11:43:15 -0700 Subject: [PATCH 149/205] Expand KafkaProducer docstring w/ idempotent and transactional notes --- kafka/producer/kafka.py | 45 +++++++++++++++++++++++++++++++++++++++-- 1 file changed, 43 insertions(+), 2 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 3253668a3..6861ec93a 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -37,8 +37,8 @@ class KafkaProducer(object): The producer is thread safe and sharing a single producer instance across threads will generally be faster than having multiple instances. - The producer consists of a pool of buffer space that holds records that - haven't yet been transmitted to the server as well as a background I/O + The producer consists of a RecordAccumulator which holds records that + haven't yet been transmitted to the server, and a Sender background I/O thread that is responsible for turning these records into requests and transmitting them to the cluster. @@ -77,6 +77,47 @@ class KafkaProducer(object): The key_serializer and value_serializer instruct how to turn the key and value objects the user provides into bytes. + From Kafka 0.11, the KafkaProducer supports two additional modes: + the idempotent producer and the transactional producer. + The idempotent producer strengthens Kafka's delivery semantics from + at least once to exactly once delivery. In particular, producer retries + will no longer introduce duplicates. The transactional producer allows an + application to send messages to multiple partitions (and topics!) + atomically. + + To enable idempotence, the `enable_idempotence` configuration must be set + to True. If set, the `retries` config will default to `float('inf')` and + the `acks` config will default to 'all'. There are no API changes for the + idempotent producer, so existing applications will not need to be modified + to take advantage of this feature. + + To take advantage of the idempotent producer, it is imperative to avoid + application level re-sends since these cannot be de-duplicated. As such, if + an application enables idempotence, it is recommended to leave the + `retries` config unset, as it will be defaulted to `float('inf')`. + Additionally, if a :meth:`~kafka.KafkaProducer.send` returns an error even + with infinite retries (for instance if the message expires in the buffer + before being sent), then it is recommended to shut down the producer and + check the contents of the last produced message to ensure that it is not + duplicated. Finally, the producer can only guarantee idempotence for + messages sent within a single session. + + To use the transactional producer and the attendant APIs, you must set the + `transactional_id` configuration property. If the `transactional_id` is + set, idempotence is automatically enabled along with the producer configs + which idempotence depends on. Further, topics which are included in + transactions should be configured for durability. In particular, the + `replication.factor` should be at least `3`, and the `min.insync.replicas` + for these topics should be set to 2. Finally, in order for transactional + guarantees to be realized from end-to-end, the consumers must be + configured to read only committed messages as well. + + The purpose of the `transactional_id` is to enable transaction recovery + across multiple sessions of a single producer instance. It would typically + be derived from the shard identifier in a partitioned, stateful, + application. As such, it should be unique to each producer instance running + within a partitioned application. + Keyword Arguments: bootstrap_servers: 'host[:port]' string (or list of 'host[:port]' strings) that the producer should contact to bootstrap initial From b1dae2efd94da7335fbd1fd57948a156b91212d5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 12:09:10 -0700 Subject: [PATCH 150/205] RecordAccumulator: Use helper method to get/set _tp_locks; get dq with lock in reenqueue() --- kafka/producer/record_accumulator.py | 39 ++++++++++++++-------------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 446d87272..1c250ee40 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -251,6 +251,13 @@ def delivery_timeout_ms(self): def next_expiry_time_ms(self): return self._next_batch_expiry_time_ms + def _tp_lock(self, tp): + if tp not in self._tp_locks: + with self._tp_locks[None]: + if tp not in self._tp_locks: + self._tp_locks[tp] = threading.Lock() + return self._tp_locks[tp] + def append(self, tp, timestamp_ms, key, value, headers, now=None): """Add a record to the accumulator, return the append result. @@ -275,12 +282,7 @@ def append(self, tp, timestamp_ms, key, value, headers, now=None): # not miss batches in abortIncompleteBatches(). self._appends_in_progress.increment() try: - if tp not in self._tp_locks: - with self._tp_locks[None]: - if tp not in self._tp_locks: - self._tp_locks[tp] = threading.Lock() - - with self._tp_locks[tp]: + with self._tp_lock(tp): # check if we have an in-progress batch dq = self._batches[tp] if dq: @@ -290,7 +292,7 @@ def append(self, tp, timestamp_ms, key, value, headers, now=None): batch_is_full = len(dq) > 1 or last.records.is_full() return future, batch_is_full, False - with self._tp_locks[tp]: + with self._tp_lock(tp): # Need to check if producer is closed again after grabbing the # dequeue lock. assert not self._closed, 'RecordAccumulator is closed' @@ -333,8 +335,7 @@ def expired_batches(self, now=None): """Get a list of batches which have been sitting in the accumulator too long and need to be expired.""" expired_batches = [] for tp in list(self._batches.keys()): - assert tp in self._tp_locks, 'TopicPartition not in locks dict' - with self._tp_locks[tp]: + with self._tp_lock(tp): # iterate over the batches and expire them if they have stayed # in accumulator for more than request_timeout_ms dq = self._batches[tp] @@ -352,14 +353,12 @@ def expired_batches(self, now=None): def reenqueue(self, batch, now=None): """ - Re-enqueue the given record batch in the accumulator. In Sender.completeBatch method, we check - whether the batch has reached deliveryTimeoutMs or not. Hence we do not do the delivery timeout check here. + Re-enqueue the given record batch in the accumulator. In Sender._complete_batch method, we check + whether the batch has reached delivery_timeout_ms or not. Hence we do not do the delivery timeout check here. """ batch.retry(now=now) - assert batch.topic_partition in self._tp_locks, 'TopicPartition not in locks dict' - assert batch.topic_partition in self._batches, 'TopicPartition not in batches' - dq = self._batches[batch.topic_partition] - with self._tp_locks[batch.topic_partition]: + with self._tp_lock(batch.topic_partition): + dq = self._batches[batch.topic_partition] dq.appendleft(batch) def ready(self, cluster, now=None): @@ -412,7 +411,7 @@ def ready(self, cluster, now=None): elif tp in self.muted: continue - with self._tp_locks[tp]: + with self._tp_lock(tp): dq = self._batches[tp] if not dq: continue @@ -445,7 +444,7 @@ def ready(self, cluster, now=None): def has_undrained(self): """Check whether there are any batches which haven't been drained""" for tp in list(self._batches.keys()): - with self._tp_locks[tp]: + with self._tp_lock(tp): dq = self._batches[tp] if len(dq): return True @@ -485,7 +484,7 @@ def drain_batches_for_one_node(self, cluster, node_id, max_size, now=None): if tp not in self._batches: continue - with self._tp_locks[tp]: + with self._tp_lock(tp): dq = self._batches[tp] if len(dq) == 0: continue @@ -619,7 +618,7 @@ def _abort_batches(self, error): for batch in self._incomplete.all(): tp = batch.topic_partition # Close the batch before aborting - with self._tp_locks[tp]: + with self._tp_lock(tp): batch.records.close() self._batches[tp].remove(batch) batch.abort(error) @@ -628,7 +627,7 @@ def _abort_batches(self, error): def abort_undrained_batches(self, error): for batch in self._incomplete.all(): tp = batch.topic_partition - with self._tp_locks[tp]: + with self._tp_lock(tp): aborted = False if not batch.is_done: aborted = True From fb046266f26ea8b6b177fa44b7ac11a608c706e4 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 27 Apr 2025 13:34:46 -0700 Subject: [PATCH 151/205] KIP-394: handle MEMBER_ID_REQUIRED error w/ second join group request (#2598) --- kafka/coordinator/base.py | 32 +++++++++++++++++++++++++------- kafka/protocol/group.py | 20 +++++++++++++++++--- test/test_coordinator.py | 2 +- 3 files changed, 43 insertions(+), 11 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index ad644aa52..4aa5c89bc 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -33,6 +33,16 @@ 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 __eq__(self, other): + return (self.generation_id == other.generation_id and + self.member_id == other.member_id and + self.protocol == other.protocol) + + Generation.NO_GENERATION = Generation(DEFAULT_GENERATION_ID, UNKNOWN_MEMBER_ID, None) @@ -461,7 +471,8 @@ def join_group(self, timeout_ms=None): exception = future.exception if isinstance(exception, (Errors.UnknownMemberIdError, Errors.RebalanceInProgressError, - Errors.IllegalGenerationError)): + Errors.IllegalGenerationError, + Errors.MemberIdRequiredError)): continue elif not future.retriable(): raise exception # pylint: disable-msg=raising-bad-type @@ -491,7 +502,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=3) + version = self._client.api_version(JoinGroupRequest, max_version=4) if version == 0: request = JoinGroupRequest[version]( self.group_id, @@ -585,6 +596,11 @@ def _handle_join_group_response(self, future, send_time, response): future.failure(error) elif error_type is Errors.GroupAuthorizationFailedError: future.failure(error_type(self.group_id)) + 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. + self.reset_generation(response.member_id) + future.failure(error_type()) else: # unexpected error, throw the exception error = error_type() @@ -762,10 +778,10 @@ def generation(self): return None return self._generation - def reset_generation(self): - """Reset the generation and memberId because we have fallen out of the group.""" + def reset_generation(self, member_id=UNKNOWN_MEMBER_ID): + """Reset the generation and member_id because we have fallen out of the group.""" with self._lock: - self._generation = Generation.NO_GENERATION + self._generation = Generation(DEFAULT_GENERATION_ID, member_id, None) self.rejoin_needed = True self.state = MemberState.UNJOINED @@ -799,8 +815,10 @@ def _close_heartbeat_thread(self, timeout_ms=None): self._heartbeat_thread = None def __del__(self): - if hasattr(self, '_heartbeat_thread'): + try: self._close_heartbeat_thread() + except (TypeError, AttributeError): + pass def close(self, timeout_ms=None): """Close the coordinator, leave the current group, @@ -816,7 +834,7 @@ def maybe_leave_group(self, timeout_ms=None): with self._client._lock, self._lock: if (not self.coordinator_unknown() and self.state is not MemberState.UNJOINED - and self._generation is not Generation.NO_GENERATION): + and self._generation.is_valid): # this is a minimal effort attempt to leave the group. we do not # attempt any resending if the request fails or times out. diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py index 3b32590ec..74e19c94b 100644 --- a/kafka/protocol/group.py +++ b/kafka/protocol/group.py @@ -52,6 +52,12 @@ class JoinGroupResponse_v3(Response): SCHEMA = JoinGroupResponse_v2.SCHEMA +class JoinGroupResponse_v4(Response): + API_KEY = 11 + API_VERSION = 4 + SCHEMA = JoinGroupResponse_v3.SCHEMA + + class JoinGroupRequest_v0(Request): API_KEY = 11 API_VERSION = 0 @@ -95,14 +101,22 @@ class JoinGroupRequest_v3(Request): API_VERSION = 3 RESPONSE_TYPE = JoinGroupResponse_v3 SCHEMA = JoinGroupRequest_v2.SCHEMA - UNKNOWN_MEMBER_ID = '' + + +class JoinGroupRequest_v4(Request): + API_KEY = 11 + API_VERSION = 4 + RESPONSE_TYPE = JoinGroupResponse_v4 + SCHEMA = JoinGroupRequest_v3.SCHEMA JoinGroupRequest = [ - JoinGroupRequest_v0, JoinGroupRequest_v1, JoinGroupRequest_v2, JoinGroupRequest_v3 + JoinGroupRequest_v0, JoinGroupRequest_v1, JoinGroupRequest_v2, + JoinGroupRequest_v3, JoinGroupRequest_v4, ] JoinGroupResponse = [ - JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v2, JoinGroupResponse_v3 + JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v2, + JoinGroupResponse_v3, JoinGroupResponse_v4, ] diff --git a/test/test_coordinator.py b/test/test_coordinator.py index bfd3a2187..251de566a 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -304,7 +304,7 @@ def test_close(mocker, coordinator): coordinator._handle_leave_group_response.assert_called_with('foobar') assert coordinator.generation() is None - assert coordinator._generation is Generation.NO_GENERATION + assert coordinator._generation == Generation.NO_GENERATION assert coordinator.state is MemberState.UNJOINED assert coordinator.rejoin_needed is True From 603e10eb74fc3c8ba0f6c3e5e73479d9c51d7725 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Apr 2025 12:04:32 -0700 Subject: [PATCH 152/205] Release 2.2.0 --- CHANGES.md | 46 +++++++++++++++++++++++++++++++++++++++ docs/changelog.rst | 54 ++++++++++++++++++++++++++++++++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 101 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 749b83afb..2b3330c1d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,49 @@ +# 2.2.0 (Apr 28, 2025) + +KafkaProducer +* KIP-98: Add idempotent producer support (#2569) +* KIP-98: Transactional Producer (#2587) +* KIP-98: Add offsets support to transactional KafkaProducer (#2590) +* Prefix producer logs w/ client id and transactional id (#2591) +* KAFKA-5429: Ignore produce response if batch was previously aborted +* KIP-91: KafkaProducer `delivery_timeout_ms` +* Default retries -> infinite +* Expand KafkaProducer docstring w/ idempotent and transactional notes +* RecordAccumulator: Use helper method to get/set `_tp_locks`; get dq with lock in reenqueue() + +KafkaConsumer +* KIP-98: Add Consumer support for `READ_COMMITTED` (#2582) +* KIP-394: handle `MEMBER_ID_REQUIRED` error w/ second join group request (#2598) +* KAFKA-5078: Defer fetch record exception if iterator has already moved across a valid record +* KAFKA-5075: Defer consumer fetcher exception if fetch position has already increased +* KAFKA-4937: Batch offset fetches in the Consumer +* KAFKA-4547: Avoid resetting paused partitions to committed offsets +* KAFKA-6397: Consumer should not block setting positions of unavailable partitions (#2593) + +Potentially Breaking Changes (internal) +* Rename CorruptRecordException -> CorruptRecordError +* Rename Coordinator errors to generic not group (#2585) +* Rename `ClusterMetadata.add_group_coordinator` -> `add_coordinator` + support txn type +* Use SaslAuthenticationFailedError in kafka.conn connection failure; Drop unused AuthenticationFailedError +* Remove old/unused errors; reorder; KafkaTimeout -> retriable +* Drop `log_start_offset` from producer RecordMetadata + +Internal +* MemoryRecords iterator; MemoryRecordsBuilder records() helper +* Convert `DefaultRecordsBuilder.size_in_bytes` to classmethod + +Fixes +* Resolve datetime deprecation warnings (#2589) +* Avoid self refcount in log messages; test thread close on all pythons +* Fix client.wakeup() race from producer/sender close +* Fix ElectionNotNeededError handling in admin client + +Tests +* Move integration tests and fixtures to test/integration/; simplify unit fixtures (#2588) +* Expand Sender test coverage (#2586) +* py2 test fixups +* Drop unused KafkaClient import from `test_fetcher` + # 2.1.5 (Apr 4, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index e77885af7..d435ec95d 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,60 @@ Changelog ========= +2.2.0 (Apr 28, 2025) +#################### + +KafkaProducer +------------- +* KIP-98: Add idempotent producer support (#2569) +* KIP-98: Transactional Producer (#2587) +* KIP-98: Add offsets support to transactional KafkaProducer (#2590) +* Prefix producer logs w/ client id and transactional id (#2591) +* KAFKA-5429: Ignore produce response if batch was previously aborted +* KIP-91: KafkaProducer `delivery_timeout_ms` +* Default retries -> infinite +* Expand KafkaProducer docstring w/ idempotent and transactional notes +* RecordAccumulator: Use helper method to get/set `_tp_locks`; get dq with lock in reenqueue() + +KafkaConsumer +------------- +* KIP-98: Add Consumer support for `READ_COMMITTED` (#2582) +* KIP-394: handle `MEMBER_ID_REQUIRED` error w/ second join group request (#2598) +* KAFKA-5078: Defer fetch record exception if iterator has already moved across a valid record +* KAFKA-5075: Defer consumer fetcher exception if fetch position has already increased +* KAFKA-4937: Batch offset fetches in the Consumer +* KAFKA-4547: Avoid resetting paused partitions to committed offsets +* KAFKA-6397: Consumer should not block setting positions of unavailable partitions (#2593) + +Potentially Breaking Changes (internal) +--------------------------------------- +* Rename CorruptRecordException -> CorruptRecordError +* Rename Coordinator errors to generic not group (#2585) +* Rename `ClusterMetadata.add_group_coordinator` -> `add_coordinator` + support txn type +* Use SaslAuthenticationFailedError in kafka.conn connection failure; Drop unused AuthenticationFailedError +* Remove old/unused errors; reorder; KafkaTimeout -> retriable +* Drop `log_start_offset` from producer RecordMetadata + +Internal +-------- +* MemoryRecords iterator; MemoryRecordsBuilder records() helper +* Convert `DefaultRecordsBuilder.size_in_bytes` to classmethod + +Fixes +----- +* Resolve datetime deprecation warnings (#2589) +* Avoid self refcount in log messages; test thread close on all pythons +* Fix client.wakeup() race from producer/sender close +* Fix ElectionNotNeededError handling in admin client + +Tests +----- +* Move integration tests and fixtures to test/integration/; simplify unit fixtures (#2588) +* Expand Sender test coverage (#2586) +* py2 test fixups +* Drop unused KafkaClient import from `test_fetcher` + + 2.1.5 (Apr 4, 2025) ################### diff --git a/kafka/version.py b/kafka/version.py index d92194f41..04188a16d 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.0.dev' +__version__ = '2.2.0' From 0f481e07181462c6bf73b42c9eafc0c95252dcb7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Apr 2025 12:27:59 -0700 Subject: [PATCH 153/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 04188a16d..ac9611770 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.0' +__version__ = '2.2.1.dev' From 284c8a7c3096b9015cb45c2059f43f5a1dd15535 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Mon, 28 Apr 2025 12:30:38 -0700 Subject: [PATCH 154/205] Add transactional examples to docs --- README.rst | 23 +++++++++++++++++++++++ docs/index.rst | 46 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 69 insertions(+) diff --git a/README.rst b/README.rst index 2de04c673..b820c34eb 100644 --- a/README.rst +++ b/README.rst @@ -94,6 +94,14 @@ that expose basic message attributes: topic, partition, offset, key, and value: for msg in consumer: print (msg.headers) +.. code-block:: python + + # Read only committed messages from transactional topic + consumer = KafkaConsumer(isolation_level='read_committed') + consumer.subscribe(['txn_topic']) + for msg in consumer: + print(msg) + .. code-block:: python # Get consumer metrics @@ -153,6 +161,21 @@ for more details. for i in range(1000): producer.send('foobar', b'msg %d' % i) +.. code-block:: python + + # Use transactions + producer = KafkaProducer(transactional_id='fizzbuzz') + producer.init_transactions() + producer.begin_transaction() + future = producer.send('txn_topic', value=b'yes') + future.get() # wait for successful produce + producer.commit_transaction() # commit the transaction + + producer.begin_transaction() + future = producer.send('txn_topic', value=b'no') + future.get() # wait for successful produce + producer.abort_transaction() # abort the transaction + .. code-block:: python # Include record headers. The format is list of tuples with string key diff --git a/docs/index.rst b/docs/index.rst index 471a234f0..823780929 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -80,6 +80,26 @@ that expose basic message attributes: topic, partition, offset, key, and value: for msg in consumer: assert isinstance(msg.value, dict) +.. code-block:: python + + # Access record headers. The returned value is a list of tuples + # with str, bytes for key and value + for msg in consumer: + print (msg.headers) + +.. code-block:: python + + # Read only committed messages from transactional topic + consumer = KafkaConsumer(isolation_level='read_committed') + consumer.subscribe(['txn_topic']) + for msg in consumer: + print(msg) + +.. code-block:: python + + # Get consumer metrics + metrics = consumer.metrics() + KafkaProducer ************* @@ -133,6 +153,32 @@ client. See `KafkaProducer `_ for more details. for i in range(1000): producer.send('foobar', b'msg %d' % i) +.. code-block:: python + + # Use transactions + producer = KafkaProducer(transactional_id='fizzbuzz') + producer.init_transactions() + producer.begin_transaction() + future = producer.send('txn_topic', value=b'yes') + future.get() # wait for successful produce + producer.commit_transaction() # commit the transaction + + producer.begin_transaction() + future = producer.send('txn_topic', value=b'no') + future.get() # wait for successful produce + producer.abort_transaction() # abort the transaction + +.. code-block:: python + + # Include record headers. The format is list of tuples with string key + # and bytes value. + producer.send('foobar', value=b'c29tZSB2YWx1ZQ==', headers=[('content-encoding', b'base64')]) + +.. code-block:: python + + # Get producer performance metrics + metrics = producer.metrics() + Thread safety ************* From c35c1614d81eb596d6944fb9cfabfc8ddfa95003 Mon Sep 17 00:00:00 2001 From: cheterchao <42969446+zcc0077@users.noreply.github.com> Date: Wed, 30 Apr 2025 00:01:26 +0800 Subject: [PATCH 155/205] Fix SubscriptionState AttributeError in KafkaConsumer (#2599) --- kafka/consumer/group.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index a86ececf4..471ae5cda 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -877,7 +877,7 @@ def seek_to_beginning(self, *partitions): for tp in partitions: log.debug("Seeking to beginning of partition %s", tp) - self._subscription.need_offset_reset(tp, OffsetResetStrategy.EARLIEST) + self._subscription.request_offset_reset(tp, OffsetResetStrategy.EARLIEST) self._iterator = None def seek_to_end(self, *partitions): @@ -902,7 +902,7 @@ def seek_to_end(self, *partitions): for tp in partitions: log.debug("Seeking to end of partition %s", tp) - self._subscription.need_offset_reset(tp, OffsetResetStrategy.LATEST) + self._subscription.request_offset_reset(tp, OffsetResetStrategy.LATEST) self._iterator = None def subscribe(self, topics=(), pattern=None, listener=None): From 842f398ff3e733971c2880215cd52aba0aa6c90b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Apr 2025 09:35:10 -0700 Subject: [PATCH 156/205] Always try ApiVersionsRequest v0, even on broker disconnect (#2603) --- kafka/conn.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index 31e1f8be9..8dd65c1c0 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -301,6 +301,7 @@ def __init__(self, host, port, afi, **configs): if self.config['ssl_context'] is not None: self._ssl_context = self.config['ssl_context'] self._api_versions_future = None + self._api_versions_check_timeout = self.config['api_version_auto_timeout_ms'] self._sasl_auth_future = None self.last_attempt = 0 self._gai = [] @@ -557,7 +558,8 @@ def _try_api_versions_check(self): else: request = ApiVersionsRequest[version]() future = Future() - response = self._send(request, blocking=True, request_timeout_ms=(self.config['api_version_auto_timeout_ms'] * 0.8)) + self._api_versions_check_timeout /= 2 + response = self._send(request, blocking=True, request_timeout_ms=self._api_versions_check_timeout) response.add_callback(self._handle_api_versions_response, future) response.add_errback(self._handle_api_versions_failure, future) self._api_versions_future = future @@ -566,7 +568,8 @@ def _try_api_versions_check(self): elif self._check_version_idx < len(self.VERSION_CHECKS): version, request = self.VERSION_CHECKS[self._check_version_idx] future = Future() - response = self._send(request, blocking=True, request_timeout_ms=(self.config['api_version_auto_timeout_ms'] * 0.8)) + self._api_versions_check_timeout /= 2 + response = self._send(request, blocking=True, request_timeout_ms=self._api_versions_check_timeout) response.add_callback(self._handle_check_version_response, future, version) response.add_errback(self._handle_check_version_failure, future) self._api_versions_future = future @@ -618,7 +621,13 @@ def _handle_api_versions_response(self, future, response): def _handle_api_versions_failure(self, future, ex): future.failure(ex) - self._check_version_idx = 0 + # Modern brokers should not disconnect on unrecognized api-versions request, + # but in case they do we always want to try v0 as a fallback + # otherwise switch to check_version probe. + if self._api_versions_idx > 0: + self._api_versions_idx = 0 + else: + self._check_version_idx = 0 # after failure connection is closed, so state should already be DISCONNECTED def _handle_check_version_response(self, future, version, _response): From 8c397330060cb25988441b6639ae19ff87d44582 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Apr 2025 09:36:54 -0700 Subject: [PATCH 157/205] Patch Release 2.2.1 --- CHANGES.md | 9 +++++++++ docs/changelog.rst | 13 +++++++++++++ kafka/version.py | 2 +- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 2b3330c1d..ab037e3b4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,12 @@ +# 2.2.1 (Apr 29, 2025) + +Fixes +* Always try ApiVersionsRequest v0, even on broker disconnect (#2603) +* Fix SubscriptionState AttributeError in KafkaConsumer (#2599) + +Documentation +* Add transactional examples to docs + # 2.2.0 (Apr 28, 2025) KafkaProducer diff --git a/docs/changelog.rst b/docs/changelog.rst index d435ec95d..c701f5e9c 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,19 @@ Changelog ========= +2.2.1 (Apr 29, 2025) +#################### + +Fixes +----- +* Always try ApiVersionsRequest v0, even on broker disconnect (#2603) +* Fix SubscriptionState AttributeError in KafkaConsumer (#2599) + +Documentation +------------- +* Add transactional examples to docs + + 2.2.0 (Apr 28, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index ac9611770..36a511eca 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.1.dev' +__version__ = '2.2.1' From cb2868faabcf445afec62caca8eeac54d1c5d904 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Apr 2025 09:57:21 -0700 Subject: [PATCH 158/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 36a511eca..8c2fbc4b4 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.1' +__version__ = '2.2.2.dev' From 998efc250d3d228e29ce1a488a4d2c0d60d31a2b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 29 Apr 2025 10:10:28 -0700 Subject: [PATCH 159/205] Fix lint errors --- kafka/admin/client.py | 2 +- kafka/producer/sender.py | 2 +- kafka/producer/transaction_manager.py | 12 ++++++------ kafka/record/default_records.py | 2 ++ 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 5bbc99f30..82aaa68e9 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -274,7 +274,7 @@ def _refresh_controller_id(self, timeout_ms=30000): self._controller_id = controller_id return else: - raise Errors.NodeNotAvailableError('controller') + raise Errors.NodeNotReadyError('controller') else: raise UnrecognizedBrokerVersion( "Kafka Admin interface cannot determine the controller using MetadataRequest_v{}." diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index dcb3ecbdc..4a88b2f7a 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -315,7 +315,7 @@ def _maybe_send_transactional_request(self): return True except Exception as e: - log.warn("%s: Got an exception when trying to find a node to send a transactional request to. Going to back off and retry", str(self), e) + log.warn("%s: Got an exception when trying to find a node to send a transactional request to. Going to back off and retry: %s", str(self), e) if next_request_handler.needs_coordinator(): self._transaction_manager.lookup_coordinator_for_request(next_request_handler) break diff --git a/kafka/producer/transaction_manager.py b/kafka/producer/transaction_manager.py index f8f9d2f52..7302eb00e 100644 --- a/kafka/producer/transaction_manager.py +++ b/kafka/producer/transaction_manager.py @@ -260,7 +260,7 @@ def transition_to_abortable_error(self, exc): with self._lock: if self._current_state == TransactionState.ABORTING_TRANSACTION: log.debug("Skipping transition to abortable error state since the transaction is already being " - " aborted. Underlying exception: ", exc) + " aborted. Underlying exception: %s", exc) return self._transition_to(TransactionState.ABORTABLE_ERROR, error=exc) @@ -687,7 +687,7 @@ def handle_response(self, response): if error is Errors.NoError: continue elif error in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): - self.transaction_manager._lookup_coordinator('transaction', self.transactiona_id) + self.transaction_manager._lookup_coordinator('transaction', self.transactional_id) self.reenqueue() return elif error is Errors.ConcurrentTransactionsError: @@ -726,7 +726,7 @@ def handle_response(self, response): self.transaction_manager._pending_partitions_in_transaction -= partitions if unauthorized_topics: - self.abortable_error(Errors.TopicAuthorizationError(unauthorized_topics)) + self.abortable_error(Errors.TopicAuthorizationFailedError(unauthorized_topics)) elif has_partition_errors: self.abortable_error(Errors.KafkaError("Could not add partitions to transaction due to errors: %s" % (results))) else: @@ -795,7 +795,7 @@ def handle_response(self, response): elif error is Errors.TransactionalIdAuthorizationFailedError: self.fatal_error(error()) elif error is Errors.GroupAuthorizationFailedError: - self.abortable_error(Errors.GroupAuthorizationError(self._coord_key)) + self.abortable_error(error(self._coord_key)) else: self.fatal_error(Errors.KafkaError( "Could not find a coordinator with type %s with key %s due to" @@ -888,7 +888,7 @@ def handle_response(self, response): elif error is Errors.TransactionalIdAuthorizationFailedError: self.fatal_error(error()) elif error is Errors.GroupAuthorizationFailedError: - self.abortable_error(Errors.GroupAuthorizationError(self.consumer_group_id)) + self.abortable_error(error(self.consumer_group_id)) else: self.fatal_error(Errors.KafkaError("Unexpected error in AddOffsetsToTxnResponse: %s" % (error()))) @@ -955,7 +955,7 @@ def handle_response(self, response): elif error is Errors.UnknownTopicOrPartitionError: retriable_failure = True elif error is Errors.GroupAuthorizationFailedError: - self.abortable_error(Errors.GroupAuthorizationError(self.consumer_group_id)) + self.abortable_error(error(self.consumer_group_id)) return elif error in (Errors.TransactionalIdAuthorizationFailedError, Errors.InvalidProducerEpochError, diff --git a/kafka/record/default_records.py b/kafka/record/default_records.py index b495c76fe..a3b9cd5d8 100644 --- a/kafka/record/default_records.py +++ b/kafka/record/default_records.py @@ -117,6 +117,8 @@ def _assert_has_codec(self, compression_type): checker, name = codecs.has_lz4, "lz4" elif compression_type == self.CODEC_ZSTD: checker, name = codecs.has_zstd, "zstd" + else: + raise UnsupportedCodecError("Unrecognized compression type: %s" % (compression_type,)) if not checker(): raise UnsupportedCodecError( "Libraries for {} compression codec not found".format(name)) From 3d31c14bedd15e26b18cfc3c62811005b85596c1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Apr 2025 10:34:48 -0700 Subject: [PATCH 160/205] Patch Release 2.2.2 --- CHANGES.md | 5 +++++ docs/changelog.rst | 8 ++++++++ kafka/version.py | 2 +- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index ab037e3b4..62ac81460 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,8 @@ +# 2.2.2 (Apr 30, 2025) + +Fixes +* Fix lint errors + # 2.2.1 (Apr 29, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index c701f5e9c..f2d7d4702 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,14 @@ Changelog ========= +2.2.2 (Apr 30, 2025) +#################### + +Fixes +----- +* Fix lint errors + + 2.2.1 (Apr 29, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 8c2fbc4b4..f1edb192f 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.2.dev' +__version__ = '2.2.2' From 26fbd2eed37224a6815cf61f4f302bf060945576 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 30 Apr 2025 10:46:59 -0700 Subject: [PATCH 161/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index f1edb192f..9c8fb8355 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.2' +__version__ = '2.2.3.dev' From b6036f239b372aa9bf7eda29ea59b7515b55958c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 1 May 2025 12:55:30 -0700 Subject: [PATCH 162/205] Only create fetch requests for ready nodes (#2607) --- kafka/consumer/fetcher.py | 65 +++++++++++++++++++++++---------------- test/test_fetcher.py | 1 + 2 files changed, 39 insertions(+), 27 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index ceca1d9b6..16b3fbb68 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -153,6 +153,7 @@ def send_fetches(self): future = self._client.send(node_id, request, wakeup=False) future.add_callback(self._handle_fetch_response, node_id, fetch_offsets, time.time()) future.add_errback(self._handle_fetch_error, node_id) + future.add_both(self._clear_pending_fetch_request, node_id) futures.append(future) self._fetch_futures.extend(futures) self._clean_done_fetch_futures() @@ -643,36 +644,42 @@ def _create_fetch_requests(self): log.debug("Skipping fetch for partition %s because node %s is throttled", partition, node_id) + 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") + 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", partition, node_id) - continue - if version < 5: - partition_info = ( - partition.partition, - position.offset, - self.config['max_partition_fetch_bytes'] - ) - elif version <= 8: - partition_info = ( - partition.partition, - position.offset, - -1, # log_start_offset is used internally by brokers / replicas only - self.config['max_partition_fetch_bytes'], - ) else: - partition_info = ( - partition.partition, - position.leader_epoch, - position.offset, - -1, # log_start_offset is used internally by brokers / replicas only - self.config['max_partition_fetch_bytes'], - ) - - fetchable[node_id][partition] = partition_info - log.debug("Adding fetch request for partition %s at offset %d", - partition, position.offset) + # Leader is connected and does not have a pending fetch request + if version < 5: + partition_info = ( + partition.partition, + position.offset, + self.config['max_partition_fetch_bytes'] + ) + elif version <= 8: + partition_info = ( + partition.partition, + position.offset, + -1, # log_start_offset is used internally by brokers / replicas only + self.config['max_partition_fetch_bytes'], + ) + else: + partition_info = ( + partition.partition, + position.leader_epoch, + position.offset, + -1, # log_start_offset is used internally by brokers / replicas only + self.config['max_partition_fetch_bytes'], + ) + + fetchable[node_id][partition] = partition_info + log.debug("Adding fetch request for partition %s at offset %d", + partition, position.offset) requests = {} for node_id, next_partitions in six.iteritems(fetchable): @@ -761,14 +768,18 @@ def _handle_fetch_response(self, node_id, fetch_offsets, send_time, response): if self._sensors: self._sensors.fetch_latency.record((time.time() - send_time) * 1000) - self._nodes_with_pending_fetch_requests.remove(node_id) def _handle_fetch_error(self, node_id, exception): level = logging.INFO if isinstance(exception, Errors.Cancelled) else logging.ERROR log.log(level, 'Fetch to node %s failed: %s', node_id, exception) if node_id in self._session_handlers: self._session_handlers[node_id].handle_error(exception) - self._nodes_with_pending_fetch_requests.remove(node_id) + + def _clear_pending_fetch_request(self, node_id, _): + try: + self._nodes_with_pending_fetch_requests.remove(node_id) + except KeyError: + pass def _parse_fetched_data(self, completed_fetch): tp = completed_fetch.topic_partition diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 740fa1bab..f4e1f3f73 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -103,6 +103,7 @@ def test_create_fetch_requests(fetcher, mocker, api_version, fetch_version): fetcher._client._api_versions = BROKER_API_VERSIONS[api_version] mocker.patch.object(fetcher._client.cluster, "leader_for_partition", return_value=0) mocker.patch.object(fetcher._client.cluster, "leader_epoch_for_partition", return_value=0) + mocker.patch.object(fetcher._client, "ready", return_value=True) by_node = fetcher._create_fetch_requests() requests_and_offsets = by_node.values() assert set([r.API_VERSION for (r, _offsets) in requests_and_offsets]) == set([fetch_version]) From cef0ea69c1d589317aa58b8fcd92244ae1206dba Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 1 May 2025 12:59:03 -0700 Subject: [PATCH 163/205] Ignore leading SECURITY_PROTOCOL:// in bootstrap_servers (#2608) --- kafka/cluster.py | 25 +++++++++++++++++- kafka/conn.py | 28 +------------------- test/test_cluster.py | 61 +++++++++++++++++++++++++++++++++++++++++++- test/test_conn.py | 50 +----------------------------------- 4 files changed, 86 insertions(+), 78 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index ae822a401..d6ec82dba 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -3,13 +3,15 @@ import collections import copy import logging +import random +import re import threading import time from kafka.vendor import six from kafka import errors as Errors -from kafka.conn import collect_hosts +from kafka.conn import get_ip_port_afi from kafka.future import Future from kafka.structs import BrokerMetadata, PartitionMetadata, TopicPartition @@ -422,3 +424,24 @@ def with_partitions(self, partitions_to_add): def __str__(self): return 'ClusterMetadata(brokers: %d, topics: %d, coordinators: %d)' % \ (len(self._brokers), len(self._partitions), len(self._coordinators)) + + +def collect_hosts(hosts, randomize=True): + """ + Collects a comma-separated set of hosts (host:port) and optionally + randomize the returned list. + """ + + if isinstance(hosts, six.string_types): + hosts = hosts.strip().split(',') + + result = [] + for host_port in hosts: + # ignore leading SECURITY_PROTOCOL:// to mimic java client + host_port = re.sub('^.*://', '', host_port) + host, port, afi = get_ip_port_afi(host_port) + result.append((host, port, afi)) + + if randomize: + random.shuffle(result) + return result diff --git a/kafka/conn.py b/kafka/conn.py index 8dd65c1c0..c9cdd595f 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -4,7 +4,7 @@ import errno import io import logging -from random import shuffle, uniform +from random import uniform # selectors in stdlib as of py3.4 try: @@ -1496,32 +1496,6 @@ def get_ip_port_afi(host_and_port_str): return host, port, af -def collect_hosts(hosts, randomize=True): - """ - Collects a comma-separated set of hosts (host:port) and optionally - randomize the returned list. - """ - - if isinstance(hosts, six.string_types): - hosts = hosts.strip().split(',') - - result = [] - afi = socket.AF_INET - for host_port in hosts: - - host, port, afi = get_ip_port_afi(host_port) - - if port < 0: - port = DEFAULT_KAFKA_PORT - - result.append((host, port, afi)) - - if randomize: - shuffle(result) - - return result - - def is_inet_4_or_6(gai): """Given a getaddrinfo struct, return True iff ipv4 or ipv6""" return gai[0] in (socket.AF_INET, socket.AF_INET6) diff --git a/test/test_cluster.py b/test/test_cluster.py index f0a2f83d6..c57bd8f9f 100644 --- a/test/test_cluster.py +++ b/test/test_cluster.py @@ -1,7 +1,9 @@ # pylint: skip-file from __future__ import absolute_import -from kafka.cluster import ClusterMetadata +import socket + +from kafka.cluster import ClusterMetadata, collect_hosts from kafka.protocol.metadata import MetadataResponse @@ -132,3 +134,60 @@ def test_metadata_v7(): assert cluster.cluster_id == 'cluster-foo' assert cluster._partitions['topic-1'][0].offline_replicas == [12] assert cluster._partitions['topic-1'][0].leader_epoch == 0 + + +def test_collect_hosts__happy_path(): + hosts = "127.0.0.1:1234,127.0.0.1" + results = collect_hosts(hosts) + assert set(results) == set([ + ('127.0.0.1', 1234, socket.AF_INET), + ('127.0.0.1', 9092, socket.AF_INET), + ]) + + +def test_collect_hosts__ipv6(): + hosts = "[localhost]:1234,[2001:1000:2000::1],[2001:1000:2000::1]:1234" + results = collect_hosts(hosts) + assert set(results) == set([ + ('localhost', 1234, socket.AF_INET6), + ('2001:1000:2000::1', 9092, socket.AF_INET6), + ('2001:1000:2000::1', 1234, socket.AF_INET6), + ]) + + +def test_collect_hosts__string_list(): + hosts = [ + 'localhost:1234', + 'localhost', + '[localhost]', + '2001::1', + '[2001::1]', + '[2001::1]:1234', + ] + results = collect_hosts(hosts) + assert set(results) == set([ + ('localhost', 1234, socket.AF_UNSPEC), + ('localhost', 9092, socket.AF_UNSPEC), + ('localhost', 9092, socket.AF_INET6), + ('2001::1', 9092, socket.AF_INET6), + ('2001::1', 9092, socket.AF_INET6), + ('2001::1', 1234, socket.AF_INET6), + ]) + + +def test_collect_hosts__with_spaces(): + hosts = "localhost:1234, localhost" + results = collect_hosts(hosts) + assert set(results) == set([ + ('localhost', 1234, socket.AF_UNSPEC), + ('localhost', 9092, socket.AF_UNSPEC), + ]) + + +def test_collect_hosts__protocol(): + hosts = "SASL_SSL://foo.bar:1234,SASL_SSL://fizz.buzz:5678" + results = collect_hosts(hosts) + assert set(results) == set([ + ('foo.bar', 1234, socket.AF_UNSPEC), + ('fizz.buzz', 5678, socket.AF_UNSPEC), + ]) diff --git a/test/test_conn.py b/test/test_conn.py index b5deb748c..037cd015e 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -10,7 +10,7 @@ import mock import pytest -from kafka.conn import BrokerConnection, ConnectionStates, collect_hosts +from kafka.conn import BrokerConnection, ConnectionStates from kafka.future import Future from kafka.protocol.api import RequestHeader from kafka.protocol.group import HeartbeatResponse @@ -280,54 +280,6 @@ def test_close(conn): pass # TODO -def test_collect_hosts__happy_path(): - hosts = "127.0.0.1:1234,127.0.0.1" - results = collect_hosts(hosts) - assert set(results) == set([ - ('127.0.0.1', 1234, socket.AF_INET), - ('127.0.0.1', 9092, socket.AF_INET), - ]) - - -def test_collect_hosts__ipv6(): - hosts = "[localhost]:1234,[2001:1000:2000::1],[2001:1000:2000::1]:1234" - results = collect_hosts(hosts) - assert set(results) == set([ - ('localhost', 1234, socket.AF_INET6), - ('2001:1000:2000::1', 9092, socket.AF_INET6), - ('2001:1000:2000::1', 1234, socket.AF_INET6), - ]) - - -def test_collect_hosts__string_list(): - hosts = [ - 'localhost:1234', - 'localhost', - '[localhost]', - '2001::1', - '[2001::1]', - '[2001::1]:1234', - ] - results = collect_hosts(hosts) - assert set(results) == set([ - ('localhost', 1234, socket.AF_UNSPEC), - ('localhost', 9092, socket.AF_UNSPEC), - ('localhost', 9092, socket.AF_INET6), - ('2001::1', 9092, socket.AF_INET6), - ('2001::1', 9092, socket.AF_INET6), - ('2001::1', 1234, socket.AF_INET6), - ]) - - -def test_collect_hosts__with_spaces(): - hosts = "localhost:1234, localhost" - results = collect_hosts(hosts) - assert set(results) == set([ - ('localhost', 1234, socket.AF_UNSPEC), - ('localhost', 9092, socket.AF_UNSPEC), - ]) - - def test_lookup_on_connect(): hostname = 'example.org' port = 9092 From 2767d14b6cd33f2e55e7004e2e33467746a8fd9e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 1 May 2025 13:00:59 -0700 Subject: [PATCH 164/205] Patch Release 2.2.3 --- CHANGES.md | 6 ++++++ docs/changelog.rst | 9 +++++++++ kafka/version.py | 2 +- 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 62ac81460..c91c467d7 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,9 @@ +# 2.2.3 (May 1, 2025) + +Fixes +* Ignore leading SECURITY_PROTOCOL:// in bootstrap_servers (#2608) +* Only create fetch requests for ready nodes (#2607) + # 2.2.2 (Apr 30, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index f2d7d4702..4f9a90cf5 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,15 @@ Changelog ========= +2.2.3 (May 1, 2025) +################### + +Fixes +----- +* Ignore leading SECURITY_PROTOCOL:// in bootstrap_servers (#2608) +* Only create fetch requests for ready nodes (#2607) + + 2.2.2 (Apr 30, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 9c8fb8355..05633ca0c 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.3.dev' +__version__ = '2.2.3' From 827832ab7908581fcd82f13e30bc061baabb9bd0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 2 May 2025 10:05:02 -0700 Subject: [PATCH 165/205] Update changes w/ 2.1.6 backport release --- CHANGES.md | 5 +++++ docs/changelog.rst | 8 ++++++++ 2 files changed, 13 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index c91c467d7..aae374efb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -64,6 +64,11 @@ Tests * py2 test fixups * Drop unused KafkaClient import from `test_fetcher` +# 2.1.6 (May 2, 2025) + +Fixes +* Only create fetch requests for ready nodes (#2607) + # 2.1.5 (Apr 4, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 4f9a90cf5..8bbdff6b4 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -85,6 +85,14 @@ Tests * Drop unused KafkaClient import from `test_fetcher` +2.1.6 (May 2, 2025) +################### + +Fixes +----- +* Only create fetch requests for ready nodes (#2607) + + 2.1.5 (Apr 4, 2025) ################### From 41003190942ef525d4607e3bc05615235c42acbf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 3 May 2025 18:56:12 -0700 Subject: [PATCH 166/205] Fix Fetch._reset_offsets_async() KeyError when fetching from multiple nodes (#2612) --- kafka/consumer/fetcher.py | 8 ++++---- test/test_fetcher.py | 29 +++++++++++++++++++---------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 16b3fbb68..e7757e7b3 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -418,7 +418,7 @@ def _reset_offsets_async(self, timestamps): expire_at = time.time() + self.config['request_timeout_ms'] / 1000 self._subscriptions.set_reset_pending(partitions, expire_at) - def on_success(result): + def on_success(timestamps_and_epochs, result): fetched_offsets, partitions_to_retry = result if partitions_to_retry: self._subscriptions.reset_failed(partitions_to_retry, time.time() + self.config['retry_backoff_ms'] / 1000) @@ -428,7 +428,7 @@ def on_success(result): ts, _epoch = timestamps_and_epochs[partition] self._reset_offset_if_needed(partition, ts, offset.offset) - def on_failure(error): + def on_failure(partitions, error): self._subscriptions.reset_failed(partitions, time.time() + self.config['retry_backoff_ms'] / 1000) self._client.cluster.request_update() @@ -439,8 +439,8 @@ def on_failure(error): log.error("Discarding error in ListOffsetResponse because another error is pending: %s", error) future = self._send_list_offsets_request(node_id, timestamps_and_epochs) - future.add_callback(on_success) - future.add_errback(on_failure) + future.add_callback(on_success, timestamps_and_epochs) + future.add_errback(on_failure, partitions) def _send_list_offsets_requests(self, timestamps): """Fetch offsets for each partition in timestamps dict. This may send diff --git a/test/test_fetcher.py b/test/test_fetcher.py index f4e1f3f73..0ef349500 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -134,18 +134,27 @@ def test_reset_offsets_if_needed(fetcher, topic, mocker): def test__reset_offsets_async(fetcher, mocker): - tp = TopicPartition("topic", 0) + tp0 = TopicPartition("topic", 0) + tp1 = TopicPartition("topic", 1) fetcher._subscriptions.subscribe(topics=["topic"]) - fetcher._subscriptions.assign_from_subscribed([tp]) - fetcher._subscriptions.request_offset_reset(tp) - fetched_offsets = {tp: OffsetAndTimestamp(1001, None, -1)} + fetcher._subscriptions.assign_from_subscribed([tp0, tp1]) + fetcher._subscriptions.request_offset_reset(tp0) + fetcher._subscriptions.request_offset_reset(tp1) + mocker.patch.object(fetcher._client.cluster, "leader_for_partition", side_effect=[0, 1]) mocker.patch.object(fetcher._client, 'ready', return_value=True) - mocker.patch.object(fetcher, '_send_list_offsets_request', - return_value=Future().success((fetched_offsets, set()))) - mocker.patch.object(fetcher._client.cluster, "leader_for_partition", return_value=0) - fetcher._reset_offsets_async({tp: OffsetResetStrategy.EARLIEST}) - assert not fetcher._subscriptions.assignment[tp].awaiting_reset - assert fetcher._subscriptions.assignment[tp].position.offset == 1001 + future1 = Future() + future2 = Future() + mocker.patch.object(fetcher, '_send_list_offsets_request', side_effect=[future1, future2]) + fetcher._reset_offsets_async({ + tp0: OffsetResetStrategy.EARLIEST, + tp1: OffsetResetStrategy.EARLIEST, + }) + future1.success(({tp0: OffsetAndTimestamp(1001, None, -1)}, set())), + future2.success(({tp1: OffsetAndTimestamp(1002, None, -1)}, set())), + assert not fetcher._subscriptions.assignment[tp0].awaiting_reset + assert not fetcher._subscriptions.assignment[tp1].awaiting_reset + assert fetcher._subscriptions.assignment[tp0].position.offset == 1001 + assert fetcher._subscriptions.assignment[tp1].position.offset == 1002 def test__send_list_offsets_requests(fetcher, mocker): From 2f282ebc89d7e5a6b78646c55459309f9dd8f51a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 3 May 2025 19:39:22 -0700 Subject: [PATCH 167/205] Fix KafkaConsumer.poll() with zero timeout (#2613) --- kafka/client_async.py | 18 +++----- kafka/consumer/fetcher.py | 15 ++++--- kafka/consumer/group.py | 36 ++++++++-------- kafka/coordinator/base.py | 62 ++++++++++++++++++--------- kafka/coordinator/consumer.py | 81 ++++++++++++++++++++++------------- kafka/producer/kafka.py | 36 ++++++---------- kafka/util.py | 58 +++++++++++++++++-------- 7 files changed, 179 insertions(+), 127 deletions(-) diff --git a/kafka/client_async.py b/kafka/client_async.py index 448a995ba..7d466574f 100644 --- a/kafka/client_async.py +++ b/kafka/client_async.py @@ -27,7 +27,7 @@ from kafka.metrics.stats.rate import TimeUnit from kafka.protocol.broker_api_versions import BROKER_API_VERSIONS from kafka.protocol.metadata import MetadataRequest -from kafka.util import Dict, WeakMethod, ensure_valid_topic_name, timeout_ms_fn +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 @@ -645,12 +645,8 @@ def poll(self, timeout_ms=None, future=None): """ if not isinstance(timeout_ms, (int, float, type(None))): raise TypeError('Invalid type for timeout: %s' % type(timeout_ms)) + timer = Timer(timeout_ms) - begin = time.time() - if timeout_ms is not None: - timeout_at = begin + (timeout_ms / 1000) - else: - timeout_at = begin + (self.config['request_timeout_ms'] / 1000) # Loop for futures, break after first loop if None responses = [] while True: @@ -675,7 +671,7 @@ def poll(self, timeout_ms=None, future=None): if future is not None and future.is_done: timeout = 0 else: - user_timeout_ms = 1000 * max(0, timeout_at - time.time()) + user_timeout_ms = timer.timeout_ms if timeout_ms is not None else self.config['request_timeout_ms'] idle_connection_timeout_ms = self._idle_expiry_manager.next_check_ms() request_timeout_ms = self._next_ifr_request_timeout_ms() log.debug("Timeouts: user %f, metadata %f, idle connection %f, request %f", user_timeout_ms, metadata_timeout_ms, idle_connection_timeout_ms, request_timeout_ms) @@ -698,7 +694,7 @@ def poll(self, timeout_ms=None, future=None): break elif future.is_done: break - elif timeout_ms is not None and time.time() >= timeout_at: + elif timeout_ms is not None and timer.expired: break return responses @@ -1175,16 +1171,16 @@ def await_ready(self, node_id, timeout_ms=30000): This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with care. """ - inner_timeout_ms = timeout_ms_fn(timeout_ms, None) + timer = Timer(timeout_ms) self.poll(timeout_ms=0) if self.is_ready(node_id): return True - while not self.is_ready(node_id) and inner_timeout_ms() > 0: + while not self.is_ready(node_id) and not timer.expired: if self.connection_failed(node_id): raise Errors.KafkaConnectionError("Connection to %s failed." % (node_id,)) self.maybe_connect(node_id) - self.poll(timeout_ms=inner_timeout_ms()) + self.poll(timeout_ms=timer.timeout_ms) return self.is_ready(node_id) def send_and_receive(self, node_id, request): diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index e7757e7b3..42e2d660c 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -19,7 +19,7 @@ from kafka.record import MemoryRecords from kafka.serializer import Deserializer from kafka.structs import TopicPartition, OffsetAndMetadata, OffsetAndTimestamp -from kafka.util import timeout_ms_fn +from kafka.util import Timer log = logging.getLogger(__name__) @@ -230,7 +230,7 @@ def _fetch_offsets_by_times(self, timestamps, timeout_ms=None): if not timestamps: return {} - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout fetching offsets') + timer = Timer(timeout_ms, "Failed to get offsets by timestamps in %s ms" % (timeout_ms,)) timestamps = copy.copy(timestamps) fetched_offsets = dict() while True: @@ -238,7 +238,7 @@ def _fetch_offsets_by_times(self, timestamps, timeout_ms=None): return {} future = self._send_list_offsets_requests(timestamps) - self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + self._client.poll(future=future, timeout_ms=timer.timeout_ms) # Timeout w/o future completion if not future.is_done: @@ -256,12 +256,17 @@ def _fetch_offsets_by_times(self, timestamps, timeout_ms=None): if future.exception.invalid_metadata or self._client.cluster.need_update: refresh_future = self._client.cluster.request_update() - self._client.poll(future=refresh_future, timeout_ms=inner_timeout_ms()) + self._client.poll(future=refresh_future, timeout_ms=timer.timeout_ms) if not future.is_done: break else: - time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) + if timer.timeout_ms is None or timer.timeout_ms > self.config['retry_backoff_ms']: + time.sleep(self.config['retry_backoff_ms'] / 1000) + else: + time.sleep(timer.timeout_ms / 1000) + + timer.maybe_raise() raise Errors.KafkaTimeoutError( "Failed to get offsets by timestamps in %s ms" % (timeout_ms,)) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index 471ae5cda..ce3cf9203 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -18,7 +18,7 @@ from kafka.metrics import MetricConfig, Metrics from kafka.protocol.list_offsets import OffsetResetStrategy from kafka.structs import OffsetAndMetadata, TopicPartition -from kafka.util import timeout_ms_fn +from kafka.util import Timer from kafka.version import __version__ log = logging.getLogger(__name__) @@ -679,41 +679,40 @@ def poll(self, timeout_ms=0, max_records=None, update_offsets=True): assert not self._closed, 'KafkaConsumer is closed' # Poll for new data until the timeout expires - inner_timeout_ms = timeout_ms_fn(timeout_ms, None) + timer = Timer(timeout_ms) while not self._closed: - records = self._poll_once(inner_timeout_ms(), max_records, update_offsets=update_offsets) + records = self._poll_once(timer, max_records, update_offsets=update_offsets) if records: return records - - if inner_timeout_ms() <= 0: + elif timer.expired: break - return {} - def _poll_once(self, timeout_ms, max_records, update_offsets=True): + def _poll_once(self, timer, max_records, update_offsets=True): """Do one round of polling. In addition to checking for new data, this does any needed heart-beating, auto-commits, and offset updates. Arguments: - timeout_ms (int): The maximum time in milliseconds to block. + timer (Timer): The maximum time in milliseconds to block. Returns: dict: Map of topic to list of records (may be empty). """ - inner_timeout_ms = timeout_ms_fn(timeout_ms, None) - if not self._coordinator.poll(timeout_ms=inner_timeout_ms()): + if not self._coordinator.poll(timeout_ms=timer.timeout_ms): return {} - has_all_fetch_positions = self._update_fetch_positions(timeout_ms=inner_timeout_ms()) + has_all_fetch_positions = self._update_fetch_positions(timeout_ms=timer.timeout_ms) # If data is available already, e.g. from a previous network client # poll() call to commit, then just return it immediately records, partial = self._fetcher.fetched_records(max_records, update_offsets=update_offsets) + log.debug('Fetched records: %s, %s', records, partial) # Before returning the fetched records, we can send off the # next round of fetches and avoid block waiting for their # responses to enable pipelining while the user is handling the # fetched records. if not partial: + log.debug("Sending fetches") futures = self._fetcher.send_fetches() if len(futures): self._client.poll(timeout_ms=0) @@ -723,7 +722,7 @@ def _poll_once(self, timeout_ms, 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 = inner_timeout_ms(self._coordinator.time_to_next_poll() * 1000) + poll_timeout_ms = min(timer.timeout_ms, self._coordinator.time_to_next_poll() * 1000) if not has_all_fetch_positions: poll_timeout_ms = min(poll_timeout_ms, self.config['retry_backoff_ms']) @@ -749,15 +748,14 @@ def position(self, partition, timeout_ms=None): raise TypeError('partition must be a TopicPartition namedtuple') assert self._subscription.is_assigned(partition), 'Partition is not assigned' - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout retrieving partition position') + timer = Timer(timeout_ms) position = self._subscription.assignment[partition].position - try: - while position is None: - # batch update fetch positions for any partitions without a valid position - self._update_fetch_positions(timeout_ms=inner_timeout_ms()) + while position is 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 - except KafkaTimeoutError: - return None + elif timer.expired: + return None else: return position.offset diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 4aa5c89bc..1592f9154 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -16,7 +16,7 @@ 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.util import timeout_ms_fn +from kafka.util import Timer log = logging.getLogger('kafka.coordinator') @@ -256,9 +256,9 @@ def ensure_coordinator_ready(self, timeout_ms=None): timeout_ms (numeric, optional): Maximum number of milliseconds to block waiting to find coordinator. Default: None. - Raises: KafkaTimeoutError if timeout_ms is not None + Returns: True is coordinator found before timeout_ms, else False """ - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to find group coordinator') + timer = Timer(timeout_ms) with self._client._lock, self._lock: while self.coordinator_unknown(): @@ -272,27 +272,37 @@ def ensure_coordinator_ready(self, timeout_ms=None): else: self.coordinator_id = maybe_coordinator_id self._client.maybe_connect(self.coordinator_id) - continue + if timer.expired: + return False + else: + continue else: future = self.lookup_coordinator() - self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + self._client.poll(future=future, timeout_ms=timer.timeout_ms) if not future.is_done: - raise Errors.KafkaTimeoutError() + return False if future.failed(): if future.retriable(): if getattr(future.exception, 'invalid_metadata', False): log.debug('Requesting metadata for group coordinator request: %s', future.exception) metadata_update = self._client.cluster.request_update() - self._client.poll(future=metadata_update, timeout_ms=inner_timeout_ms()) + self._client.poll(future=metadata_update, timeout_ms=timer.timeout_ms) if not metadata_update.is_done: - raise Errors.KafkaTimeoutError() + return False else: - time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) + if timeout_ms is None or timer.timeout_ms > self.config['retry_backoff_ms']: + time.sleep(self.config['retry_backoff_ms'] / 1000) + else: + time.sleep(timer.timeout_ms / 1000) else: raise future.exception # pylint: disable-msg=raising-bad-type + if timer.expired: + return False + else: + return True def _reset_find_coordinator_future(self, result): self._find_coordinator_future = None @@ -407,21 +417,23 @@ def ensure_active_group(self, timeout_ms=None): timeout_ms (numeric, optional): Maximum number of milliseconds to block waiting to join group. Default: None. - Raises: KafkaTimeoutError if timeout_ms is not None + Returns: True if group initialized before timeout_ms, else False """ if self.config['api_version'] < (0, 9): raise Errors.UnsupportedVersionError('Group Coordinator APIs require 0.9+ broker') - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') - self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + timer = Timer(timeout_ms) + if not self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms): + return False self._start_heartbeat_thread() - self.join_group(timeout_ms=inner_timeout_ms()) + return self.join_group(timeout_ms=timer.timeout_ms) def join_group(self, timeout_ms=None): if self.config['api_version'] < (0, 9): raise Errors.UnsupportedVersionError('Group Coordinator APIs require 0.9+ broker') - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout attempting to join consumer group') + timer = Timer(timeout_ms) while self.need_rejoin(): - self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + if not self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms): + return False # call on_join_prepare if needed. We set a flag # to make sure that we do not call it a second @@ -434,7 +446,7 @@ def join_group(self, timeout_ms=None): if not self.rejoining: self._on_join_prepare(self._generation.generation_id, self._generation.member_id, - timeout_ms=inner_timeout_ms()) + timeout_ms=timer.timeout_ms) self.rejoining = True # fence off the heartbeat thread explicitly so that it cannot @@ -449,16 +461,19 @@ def join_group(self, timeout_ms=None): while not self.coordinator_unknown(): if not self._client.in_flight_request_count(self.coordinator_id): break - self._client.poll(timeout_ms=inner_timeout_ms(200)) + poll_timeout_ms = 200 if timer.timeout_ms is None or timer.timeout_ms > 200 else timer.timeout_ms + self._client.poll(timeout_ms=poll_timeout_ms) + if timer.expired: + return False else: continue future = self._initiate_join_group() - self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + self._client.poll(future=future, timeout_ms=timer.timeout_ms) if future.is_done: self._reset_join_group_future() else: - raise Errors.KafkaTimeoutError() + return False if future.succeeded(): self.rejoining = False @@ -467,6 +482,7 @@ def join_group(self, timeout_ms=None): self._generation.member_id, self._generation.protocol, future.value) + return True else: exception = future.exception if isinstance(exception, (Errors.UnknownMemberIdError, @@ -476,7 +492,13 @@ def join_group(self, timeout_ms=None): continue elif not future.retriable(): raise exception # pylint: disable-msg=raising-bad-type - time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) + elif timer.expired: + return False + else: + if timer.timeout_ms is None or timer.timeout_ms > self.config['retry_backoff_ms']: + time.sleep(self.config['retry_backoff_ms'] / 1000) + else: + time.sleep(timer.timeout_ms / 1000) def _send_join_group_request(self): """Join the group and return the assignment for the next generation. diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index d4943da31..4361b3dc3 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -19,7 +19,7 @@ from kafka.metrics.stats import Avg, Count, Max, Rate from kafka.protocol.commit import OffsetCommitRequest, OffsetFetchRequest from kafka.structs import OffsetAndMetadata, TopicPartition -from kafka.util import timeout_ms_fn, WeakMethod +from kafka.util import Timer, WeakMethod log = logging.getLogger(__name__) @@ -95,6 +95,7 @@ def __init__(self, client, subscription, **configs): self.auto_commit_interval = self.config['auto_commit_interval_ms'] / 1000 self.next_auto_commit_deadline = None self.completed_offset_commits = collections.deque() + self._offset_fetch_futures = dict() if self.config['default_offset_commit_callback'] is None: self.config['default_offset_commit_callback'] = self._default_offset_commit_callback @@ -269,10 +270,11 @@ def poll(self, timeout_ms=None): if self.group_id is None: return True - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.poll') + timer = Timer(timeout_ms) try: self._invoke_completed_offset_commit_callbacks() - self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + if not self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms): + return False if self.config['api_version'] >= (0, 9) and self._subscription.partitions_auto_assigned(): if self.need_rejoin(): @@ -289,9 +291,12 @@ def poll(self, timeout_ms=None): # description of the problem. if self._subscription.subscribed_pattern: metadata_update = self._client.cluster.request_update() - self._client.poll(future=metadata_update, timeout_ms=inner_timeout_ms()) + self._client.poll(future=metadata_update, timeout_ms=timer.timeout_ms) + if not metadata_update.is_done: + return False - self.ensure_active_group(timeout_ms=inner_timeout_ms()) + if not self.ensure_active_group(timeout_ms=timer.timeout_ms): + return False self.poll_heartbeat() @@ -395,10 +400,14 @@ def need_rejoin(self): def refresh_committed_offsets_if_needed(self, timeout_ms=None): """Fetch committed offsets for assigned partitions.""" missing_fetch_positions = set(self._subscription.missing_fetch_positions()) - offsets = self.fetch_committed_offsets(missing_fetch_positions, timeout_ms=timeout_ms) + try: + offsets = self.fetch_committed_offsets(missing_fetch_positions, timeout_ms=timeout_ms) + except Errors.KafkaTimeoutError: + return False for partition, offset in six.iteritems(offsets): - log.debug("Setting offset for partition %s to the committed offset %s", partition, offset.offset); + log.debug("Setting offset for partition %s to the committed offset %s", partition, offset.offset) self._subscription.seek(partition, offset.offset) + return True def fetch_committed_offsets(self, partitions, timeout_ms=None): """Fetch the current committed offsets for specified partitions @@ -415,24 +424,35 @@ def fetch_committed_offsets(self, partitions, timeout_ms=None): if not partitions: return {} - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.fetch_committed_offsets') + future_key = frozenset(partitions) + timer = Timer(timeout_ms) while True: - self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms) # contact coordinator to fetch committed offsets - future = self._send_offset_fetch_request(partitions) - self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + if future_key in self._offset_fetch_futures: + future = self._offset_fetch_futures[future_key] + else: + future = self._send_offset_fetch_request(partitions) + self._offset_fetch_futures[future_key] = future - if not future.is_done: - raise Errors.KafkaTimeoutError() + self._client.poll(future=future, timeout_ms=timer.timeout_ms) - if future.succeeded(): - return future.value + if future.is_done: + del self._offset_fetch_futures[future_key] - if not future.retriable(): - raise future.exception # pylint: disable-msg=raising-bad-type + if future.succeeded(): + return future.value - time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) + elif not future.retriable(): + raise future.exception # pylint: disable-msg=raising-bad-type + + # future failed but is retriable, or is not done yet + if timer.timeout_ms is None or timer.timeout_ms > self.config['retry_backoff_ms']: + time.sleep(self.config['retry_backoff_ms'] / 1000) + else: + time.sleep(timer.timeout_ms / 1000) + timer.maybe_raise() def close(self, autocommit=True, timeout_ms=None): """Close the coordinator, leave the current group, @@ -523,23 +543,26 @@ def commit_offsets_sync(self, offsets, timeout_ms=None): if not offsets: return - inner_timeout_ms = timeout_ms_fn(timeout_ms, 'Timeout in coordinator.poll') + timer = Timer(timeout_ms) while True: - self.ensure_coordinator_ready(timeout_ms=inner_timeout_ms()) + self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms) future = self._send_offset_commit_request(offsets) - self._client.poll(future=future, timeout_ms=inner_timeout_ms()) + self._client.poll(future=future, timeout_ms=timer.timeout_ms) - if not future.is_done: - raise Errors.KafkaTimeoutError() + if future.is_done: + if future.succeeded(): + return future.value - if future.succeeded(): - return future.value + elif not future.retriable(): + raise future.exception # pylint: disable-msg=raising-bad-type - if not future.retriable(): - raise future.exception # pylint: disable-msg=raising-bad-type - - time.sleep(inner_timeout_ms(self.config['retry_backoff_ms']) / 1000) + # future failed but is retriable, or it is still pending + if timer.timeout_ms is None or timer.timeout_ms > self.config['retry_backoff_ms']: + time.sleep(self.config['retry_backoff_ms'] / 1000) + else: + time.sleep(timer.timeout_ms / 1000) + timer.maybe_raise() def _maybe_auto_commit_offsets_sync(self, timeout_ms=None): if self.config['enable_auto_commit']: diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 6861ec93a..66208bbe1 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -5,7 +5,6 @@ import logging import socket import threading -import time import warnings import weakref @@ -24,7 +23,7 @@ from kafka.record.legacy_records import LegacyRecordBatchBuilder from kafka.serializer import Serializer from kafka.structs import TopicPartition -from kafka.util import ensure_valid_topic_name +from kafka.util import Timer, ensure_valid_topic_name log = logging.getLogger(__name__) @@ -664,8 +663,7 @@ def __getattr__(self, name): def partitions_for(self, topic): """Returns set of all known partitions for the topic.""" - max_wait = self.config['max_block_ms'] / 1000 - return self._wait_on_metadata(topic, max_wait) + return self._wait_on_metadata(topic, self.config['max_block_ms']) @classmethod def max_usable_produce_magic(cls, api_version): @@ -835,14 +833,11 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest assert not (value is None and key is None), 'Need at least one: key or value' ensure_valid_topic_name(topic) key_bytes = value_bytes = None + timer = Timer(self.config['max_block_ms'], "Failed to assign partition for message in max_block_ms.") try: assigned_partition = None - elapsed = 0.0 - begin = time.time() - timeout = self.config['max_block_ms'] / 1000 - while assigned_partition is None and elapsed < timeout: - elapsed = time.time() - begin - self._wait_on_metadata(topic, timeout - elapsed) + while assigned_partition is None and not timer.expired: + self._wait_on_metadata(topic, timer.timeout_ms) key_bytes = self._serialize( self.config['key_serializer'], @@ -856,7 +851,7 @@ def send(self, topic, value=None, key=None, headers=None, partition=None, timest assigned_partition = self._partition(topic, partition, key, value, key_bytes, value_bytes) if assigned_partition is None: - raise Errors.KafkaTimeoutError("Failed to assign partition for message after %s secs." % timeout) + raise Errors.KafkaTimeoutError("Failed to assign partition for message after %s secs." % timer.elapsed_ms / 1000) else: partition = assigned_partition @@ -931,7 +926,7 @@ def _ensure_valid_record_size(self, size): " the maximum request size you have configured with the" " max_request_size configuration" % (size,)) - def _wait_on_metadata(self, topic, max_wait): + def _wait_on_metadata(self, topic, max_wait_ms): """ Wait for cluster metadata including partitions for the given topic to be available. @@ -949,36 +944,29 @@ def _wait_on_metadata(self, topic, max_wait): """ # add topic to metadata topic list if it is not there already. self._sender.add_topic(topic) - begin = time.time() - elapsed = 0.0 + timer = Timer(max_wait_ms, "Failed to update metadata after %.1f secs." % (max_wait_ms * 1000,)) metadata_event = None while True: partitions = self._metadata.partitions_for_topic(topic) if partitions is not None: return partitions - - if elapsed >= max_wait: - raise Errors.KafkaTimeoutError( - "Failed to update metadata after %.1f secs." % (max_wait,)) - + timer.maybe_raise() if not metadata_event: metadata_event = threading.Event() log.debug("%s: Requesting metadata update for topic %s", str(self), topic) - metadata_event.clear() future = self._metadata.request_update() future.add_both(lambda e, *args: e.set(), metadata_event) self._sender.wakeup() - metadata_event.wait(max_wait - elapsed) + metadata_event.wait(timer.timeout_ms / 1000) if not metadata_event.is_set(): raise Errors.KafkaTimeoutError( - "Failed to update metadata after %.1f secs." % (max_wait,)) + "Failed to update metadata after %.1f secs." % (max_wait_ms * 1000,)) elif topic in self._metadata.unauthorized_topics: raise Errors.TopicAuthorizationFailedError(set([topic])) else: - elapsed = time.time() - begin - log.debug("%s: _wait_on_metadata woke after %s secs.", str(self), elapsed) + log.debug("%s: _wait_on_metadata woke after %s secs.", str(self), timer.elapsed_ms / 1000) def _serialize(self, f, topic, data): if not f: diff --git a/kafka/util.py b/kafka/util.py index 470200b1b..bfb9365ad 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -1,4 +1,4 @@ -from __future__ import absolute_import +from __future__ import absolute_import, division import binascii import re @@ -25,24 +25,44 @@ def crc32(data): from binascii import crc32 # noqa: F401 -def timeout_ms_fn(timeout_ms, error_message): - elapsed = 0.0 # noqa: F841 - begin = time.time() - def inner_timeout_ms(fallback=None): - if timeout_ms is None: - return fallback - elapsed = (time.time() - begin) * 1000 - if elapsed >= timeout_ms: - if error_message is not None: - raise KafkaTimeoutError(error_message) - else: - return 0 - ret = max(0, timeout_ms - elapsed) - if fallback is not None: - return min(ret, fallback) - return ret - return inner_timeout_ms - +class Timer: + __slots__ = ('_start_at', '_expire_at', '_timeout_ms', '_error_message') + + def __init__(self, timeout_ms, error_message=None, start_at=None): + self._timeout_ms = timeout_ms + self._start_at = start_at or time.time() + if timeout_ms is not None: + self._expire_at = self._start_at + timeout_ms / 1000 + else: + self._expire_at = float('inf') + self._error_message = error_message + + @property + def expired(self): + return time.time() >= self._expire_at + + @property + def timeout_ms(self): + if self._timeout_ms is None: + return None + elif self._expire_at == float('inf'): + return float('inf') + remaining = self._expire_at - time.time() + if remaining < 0: + return 0 + else: + return int(remaining * 1000) + + @property + def elapsed_ms(self): + return int(1000 * (time.time() - self._start_at)) + + def maybe_raise(self): + if self.expired: + raise KafkaTimeoutError(self._error_message) + + def __str__(self): + return "Timer(%s ms remaining)" % (self.timeout_ms) # Taken from: https://github.com/apache/kafka/blob/39eb31feaeebfb184d98cc5d94da9148c2319d81/clients/src/main/java/org/apache/kafka/common/internals/Topic.java#L29 TOPIC_MAX_LENGTH = 249 From d9c2009d755bdb84c83040d67b9d4896908dc8a5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 3 May 2025 19:57:38 -0700 Subject: [PATCH 168/205] Do not reset_generation after RebalanceInProgressError; improve CommitFailed error messages (#2614) --- kafka/coordinator/consumer.py | 30 ++++++++++++++++++++---------- kafka/errors.py | 24 ++++++++++++------------ 2 files changed, 32 insertions(+), 22 deletions(-) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 4361b3dc3..3db00d72c 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -614,18 +614,19 @@ def _send_offset_commit_request(self, offsets): for tp, offset in six.iteritems(offsets): offset_data[tp.topic][tp.partition] = offset - if self._subscription.partitions_auto_assigned(): - generation = self.generation() or Generation.NO_GENERATION + version = self._client.api_version(OffsetCommitRequest, max_version=6) + if version > 1 and self._subscription.partitions_auto_assigned(): + generation = self.generation() else: generation = Generation.NO_GENERATION # if the generation is None, we are not part of an active group # (and we expect to be). The only thing we can do is fail the commit # and let the user rejoin the group in poll() - if self.config['api_version'] >= (0, 9) and generation is None: - return Future().failure(Errors.CommitFailedError()) + if generation is None: + log.info("Failing OffsetCommit request since the consumer is not part of an active group") + return Future().failure(Errors.CommitFailedError('Group rebalance in progress')) - version = self._client.api_version(OffsetCommitRequest, max_version=6) if version == 0: request = OffsetCommitRequest[version]( self.group_id, @@ -747,13 +748,22 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): self.coordinator_dead(error_type()) future.failure(error_type(self.group_id)) return + elif error_type is Errors.RebalanceInProgressError: + # Consumer never tries to commit offset in between join-group and sync-group, + # and hence on broker-side it is not expected to see a commit offset request + # during CompletingRebalance phase; if it ever happens then broker would return + # this error. In this case we should just treat as a fatal CommitFailed exception. + # However, we do not need to reset generations and just request re-join, such that + # if the caller decides to proceed and poll, it would still try to proceed and re-join normally. + self.request_rejoin() + future.failure(Errors.CommitFailedError('Group rebalance in progress')) + return elif error_type in (Errors.UnknownMemberIdError, - Errors.IllegalGenerationError, - Errors.RebalanceInProgressError): - # need to re-join group + Errors.IllegalGenerationError): + # need reset generation and re-join group error = error_type(self.group_id) - log.debug("OffsetCommit for group %s failed: %s", - self.group_id, error) + log.warning("OffsetCommit for group %s failed: %s", + self.group_id, error) self.reset_generation() future.failure(Errors.CommitFailedError()) return diff --git a/kafka/errors.py b/kafka/errors.py index dfdc75015..898582615 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -21,18 +21,18 @@ class Cancelled(KafkaError): class CommitFailedError(KafkaError): - def __init__(self, *args, **kwargs): - super(CommitFailedError, self).__init__( - """Commit cannot be completed since the group has already - rebalanced and assigned the partitions to another member. - This means that the time between subsequent calls to poll() - was longer than the configured max_poll_interval_ms, which - typically implies that the poll loop is spending too much - time message processing. You can address this either by - increasing the rebalance timeout with max_poll_interval_ms, - or by reducing the maximum size of batches returned in poll() - with max_poll_records. - """, *args, **kwargs) + def __init__(self, *args): + if not args: + args = ("Commit cannot be completed since the group has already" + " rebalanced and assigned the partitions to another member." + " This means that the time between subsequent calls to poll()" + " was longer than the configured max_poll_interval_ms, which" + " typically implies that the poll loop is spending too much" + " time message processing. You can address this either by" + " increasing the rebalance timeout with max_poll_interval_ms," + " or by reducing the maximum size of batches returned in poll()" + " with max_poll_records.",) + super(CommitFailedError, self).__init__(*args) class IllegalArgumentError(KafkaError): From d8c9bb127baee020930ca335d9a816019adbb070 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 3 May 2025 19:59:30 -0700 Subject: [PATCH 169/205] Patch Release 2.2.4 --- CHANGES.md | 7 +++++++ docs/changelog.rst | 10 ++++++++++ kafka/version.py | 2 +- 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index aae374efb..743f3f246 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,10 @@ +# 2.2.4 (May 3, 2025) + +Fixes +* Do not `reset_generation` after RebalanceInProgressError; improve CommitFailed error messages (#2614) +* Fix KafkaConsumer.poll() with zero timeout (#2613) +* Fix Fetch._reset_offsets_async() KeyError when fetching from multiple nodes (#2612) + # 2.2.3 (May 1, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 8bbdff6b4..030114a3f 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,16 @@ Changelog ========= +2.2.4 (May 3, 2025) +################### + +Fixes +----- +* Do not `reset_generation` after RebalanceInProgressError; improve CommitFailed error messages (#2614) +* Fix KafkaConsumer.poll() with zero timeout (#2613) +* Fix Fetch._reset_offsets_async() KeyError when fetching from multiple nodes (#2612) + + 2.2.3 (May 1, 2025) ################### diff --git a/kafka/version.py b/kafka/version.py index 05633ca0c..83b6ab028 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.3' +__version__ = '2.2.4' From fef828f7d522ae7e1ba4cbc58fade5e0e9d0c959 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 3 May 2025 20:05:51 -0700 Subject: [PATCH 170/205] Bump version for development --- kafka/version.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/version.py b/kafka/version.py index 83b6ab028..e604ff743 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.4' +__version__ = '2.2.5.dev' From 590ef937f08a3c7cb8395c5b939e97367bd6290b Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 8 May 2025 10:55:41 -0700 Subject: [PATCH 171/205] Fix producer busy loop with no pending batches (#2616) --- kafka/producer/record_accumulator.py | 3 +++ kafka/producer/sender.py | 11 ++++++----- test/test_sender.py | 13 +++++++++++++ 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/kafka/producer/record_accumulator.py b/kafka/producer/record_accumulator.py index 1c250ee40..77d48d84f 100644 --- a/kafka/producer/record_accumulator.py +++ b/kafka/producer/record_accumulator.py @@ -328,6 +328,9 @@ def append(self, tp, timestamp_ms, key, value, headers, now=None): finally: self._appends_in_progress.decrement() + def reset_next_batch_expiry_time(self): + self._next_batch_expiry_time_ms = float('inf') + def maybe_update_next_batch_expiry_time(self, batch): self._next_batch_expiry_time_ms = min(self._next_batch_expiry_time_ms, batch.created * 1000 + self.delivery_timeout_ms) diff --git a/kafka/producer/sender.py b/kafka/producer/sender.py index 4a88b2f7a..7a4c557c8 100644 --- a/kafka/producer/sender.py +++ b/kafka/producer/sender.py @@ -77,7 +77,7 @@ def _maybe_remove_from_inflight_batches(self, batch): queue.pop() heapq.heapify(queue) - def _get_expired_inflight_batches(self): + def _get_expired_inflight_batches(self, now=None): """Get the in-flight batches that has reached delivery timeout.""" expired_batches = [] to_remove = [] @@ -174,7 +174,7 @@ def run_once(self): def _send_producer_data(self, now=None): now = time.time() if now is None else now # get the list of partitions with data ready to send - result = self._accumulator.ready(self._metadata) + result = self._accumulator.ready(self._metadata, now=now) ready_nodes, next_ready_check_delay, unknown_leaders_exist = result # if there are any partitions whose leaders are not known yet, force @@ -195,7 +195,7 @@ def _send_producer_data(self, now=None): # create produce requests batches_by_node = self._accumulator.drain( - self._metadata, ready_nodes, self.config['max_request_size']) + self._metadata, ready_nodes, self.config['max_request_size'], now=now) for batch_list in six.itervalues(batches_by_node): for batch in batch_list: @@ -209,8 +209,9 @@ def _send_producer_data(self, now=None): for batch in batch_list: self._accumulator.muted.add(batch.topic_partition) - expired_batches = self._accumulator.expired_batches() - expired_batches.extend(self._get_expired_inflight_batches()) + self._accumulator.reset_next_batch_expiry_time() + expired_batches = self._accumulator.expired_batches(now=now) + expired_batches.extend(self._get_expired_inflight_batches(now=now)) if expired_batches: log.debug("%s: Expired %s batches in accumulator", str(self), len(expired_batches)) diff --git a/test/test_sender.py b/test/test_sender.py index 0731454df..6d29c1e44 100644 --- a/test/test_sender.py +++ b/test/test_sender.py @@ -240,3 +240,16 @@ def test_maybe_wait_for_producer_id(): def test_run_once(): pass + + +def test__send_producer_data_expiry_time_reset(sender, accumulator, mocker): + now = time.time() + tp = TopicPartition('foo', 0) + mocker.patch.object(sender, '_failed_produce') + result = accumulator.append(tp, 0, b'key', b'value', [], now=now) + poll_timeout_ms = sender._send_producer_data(now=now) + assert poll_timeout_ms == accumulator.config['delivery_timeout_ms'] + sender._failed_produce.assert_not_called() + now += accumulator.config['delivery_timeout_ms'] + poll_timeout_ms = sender._send_producer_data(now=now) + assert poll_timeout_ms > 0 From 7536bcbfad97c4ec883dd649fba087982a3a8c3c Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 8 May 2025 10:54:23 -0700 Subject: [PATCH 172/205] Fixup py27 fetcher test failure --- test/test_fetcher.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/test_fetcher.py b/test/test_fetcher.py index 0ef349500..4794563ed 100644 --- a/test/test_fetcher.py +++ b/test/test_fetcher.py @@ -140,7 +140,8 @@ def test__reset_offsets_async(fetcher, mocker): fetcher._subscriptions.assign_from_subscribed([tp0, tp1]) fetcher._subscriptions.request_offset_reset(tp0) fetcher._subscriptions.request_offset_reset(tp1) - mocker.patch.object(fetcher._client.cluster, "leader_for_partition", side_effect=[0, 1]) + leaders = {tp0: 0, tp1: 1} + mocker.patch.object(fetcher._client.cluster, "leader_for_partition", side_effect=lambda tp: leaders[tp]) mocker.patch.object(fetcher._client, 'ready', return_value=True) future1 = Future() future2 = Future() From bf77e9a04bc78614b4d2e0d67feaedaa29b1acfc Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 8 May 2025 10:58:18 -0700 Subject: [PATCH 173/205] Patch Release 2.2.5 --- CHANGES.md | 6 ++++++ docs/changelog.rst | 9 +++++++++ kafka/version.py | 2 +- 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 743f3f246..07ba59d39 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,9 @@ +# 2.2.5 (May 8, 2025) + +Fixes +* Fix producer busy loop with no pending batches (#2616) +* Fixup py27 fetcher test failure + # 2.2.4 (May 3, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 030114a3f..cdc1e12e9 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,15 @@ Changelog ========= +2.2.5 (May 8, 2025) +################### + +Fixes +----- +* Fix producer busy loop with no pending batches (#2616) +* Fixup py27 fetcher test failure + + 2.2.4 (May 3, 2025) ################### diff --git a/kafka/version.py b/kafka/version.py index e604ff743..3db1b9fed 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.5.dev' +__version__ = '2.2.5' From 32a92852aa556d936bdd417de3eaba449056992a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 8 May 2025 12:28:11 -0700 Subject: [PATCH 174/205] Only disable heartbeat thread once at beginning of join-group (#2617) --- kafka/coordinator/base.py | 62 ++++++++++++++++++++------------------- 1 file changed, 32 insertions(+), 30 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 1592f9154..448659e62 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -19,6 +19,7 @@ from kafka.util import Timer log = logging.getLogger('kafka.coordinator') +heartbeat_log = logging.getLogger('kafka.coordinator.heartbeat') class MemberState(object): @@ -449,11 +450,12 @@ def join_group(self, timeout_ms=None): timeout_ms=timer.timeout_ms) self.rejoining = True - # fence off the heartbeat thread explicitly so that it cannot - # interfere with the join group. # Note that this must come after - # the call to onJoinPrepare since we must be able to continue - # sending heartbeats if that callback takes some time. - self._disable_heartbeat_thread() + # fence off the heartbeat thread explicitly so that it cannot + # interfere with the join group. # Note that this must come after + # the call to onJoinPrepare since we must be able to continue + # sending heartbeats if that callback takes some time. + log.debug("Disabling heartbeat thread during join-group") + self._disable_heartbeat_thread() # ensure that there are no pending requests to the coordinator. # This is important in particular to avoid resending a pending @@ -779,7 +781,7 @@ def _handle_group_coordinator_response(self, future, response): future.failure(error) else: error = error_type() - log.error("Group coordinator lookup for group %s failed: %s", + log.error("Group Coordinator lookup for group %s failed: %s", self.group_id, error) future.failure(error) @@ -815,11 +817,11 @@ def _start_heartbeat_thread(self): raise Errors.UnsupportedVersionError('Heartbeat APIs require 0.9+ broker') with self._lock: if self._heartbeat_thread is None: - log.info('Starting new heartbeat thread') + heartbeat_log.info('Starting new heartbeat thread') self._heartbeat_thread = HeartbeatThread(weakref.proxy(self)) self._heartbeat_thread.daemon = True self._heartbeat_thread.start() - log.debug("Started heartbeat thread %s", self._heartbeat_thread.ident) + heartbeat_log.debug("Started heartbeat thread %s", self._heartbeat_thread.ident) def _disable_heartbeat_thread(self): with self._lock: @@ -829,7 +831,7 @@ def _disable_heartbeat_thread(self): def _close_heartbeat_thread(self, timeout_ms=None): with self._lock: if self._heartbeat_thread is not None: - log.info('Stopping heartbeat thread') + heartbeat_log.info('Stopping heartbeat thread') try: self._heartbeat_thread.close(timeout_ms=timeout_ms) except ReferenceError: @@ -893,7 +895,7 @@ def _send_heartbeat_request(self): request = HeartbeatRequest[version](self.group_id, self._generation.generation_id, self._generation.member_id) - log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, request.member_id) # pylint: disable-msg=no-member + heartbeat_log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, request.member_id) # pylint: disable-msg=no-member future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_heartbeat_response, future, time.time()) @@ -906,38 +908,38 @@ def _handle_heartbeat_response(self, future, send_time, response): self._sensors.heartbeat_latency.record((time.time() - send_time) * 1000) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - log.debug("Received successful heartbeat response for group %s", + heartbeat_log.debug("Received successful heartbeat response for group %s", self.group_id) future.success(None) elif error_type in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): - log.warning("Heartbeat failed for group %s: coordinator (node %s)" + heartbeat_log.warning("Heartbeat failed for group %s: coordinator (node %s)" " is either not started or not valid", self.group_id, self.coordinator()) self.coordinator_dead(error_type()) future.failure(error_type()) elif error_type is Errors.RebalanceInProgressError: - log.warning("Heartbeat failed for group %s because it is" + heartbeat_log.warning("Heartbeat failed for group %s because it is" " rebalancing", self.group_id) self.request_rejoin() future.failure(error_type()) elif error_type is Errors.IllegalGenerationError: - log.warning("Heartbeat failed for group %s: generation id is not " + heartbeat_log.warning("Heartbeat failed for group %s: generation id is not " " current.", self.group_id) self.reset_generation() future.failure(error_type()) elif error_type is Errors.UnknownMemberIdError: - log.warning("Heartbeat: local member_id was not recognized;" + heartbeat_log.warning("Heartbeat: local member_id was not recognized;" " this consumer needs to re-join") self.reset_generation() future.failure(error_type) elif error_type is Errors.GroupAuthorizationFailedError: error = error_type(self.group_id) - log.error("Heartbeat failed: authorization error: %s", error) + heartbeat_log.error("Heartbeat failed: authorization error: %s", error) future.failure(error) else: error = error_type() - log.error("Heartbeat failed: Unhandled error: %s", error) + heartbeat_log.error("Heartbeat failed: Unhandled error: %s", error) future.failure(error) @@ -1003,14 +1005,14 @@ def __init__(self, coordinator): def enable(self): with self.coordinator._lock: - log.debug('Enabling heartbeat thread') + heartbeat_log.debug('Enabling heartbeat thread') self.enabled = True self.coordinator.heartbeat.reset_timeouts() self.coordinator._lock.notify() def disable(self): with self.coordinator._lock: - log.debug('Disabling heartbeat thread') + heartbeat_log.debug('Disabling heartbeat thread') self.enabled = False def close(self, timeout_ms=None): @@ -1032,24 +1034,24 @@ def close(self, timeout_ms=None): timeout_ms = self.coordinator.config['heartbeat_interval_ms'] self.join(timeout_ms / 1000) if self.is_alive(): - log.warning("Heartbeat thread did not fully terminate during close") + heartbeat_log.warning("Heartbeat thread did not fully terminate during close") def run(self): try: - log.debug('Heartbeat thread started') + heartbeat_log.debug('Heartbeat thread started') while not self.closed: self._run_once() except ReferenceError: - log.debug('Heartbeat thread closed due to coordinator gc') + heartbeat_log.debug('Heartbeat thread closed due to coordinator gc') except RuntimeError as e: - log.error("Heartbeat thread for group %s failed due to unexpected error: %s", + heartbeat_log.error("Heartbeat thread for group %s failed due to unexpected error: %s", self.coordinator.group_id, e) self.failed = e finally: - log.debug('Heartbeat thread closed') + heartbeat_log.debug('Heartbeat thread closed') def _run_once(self): with self.coordinator._client._lock, self.coordinator._lock: @@ -1063,16 +1065,16 @@ def _run_once(self): with self.coordinator._lock: if not self.enabled: - log.debug('Heartbeat disabled. Waiting') + heartbeat_log.debug('Heartbeat disabled. Waiting') self.coordinator._lock.wait() - log.debug('Heartbeat re-enabled.') + heartbeat_log.debug('Heartbeat re-enabled.') return if self.coordinator.state is not MemberState.STABLE: # the group is not stable (perhaps because we left the # group or because the coordinator kicked us out), so # disable heartbeats and wait for the main thread to rejoin. - log.debug('Group state is not stable, disabling heartbeats') + heartbeat_log.debug('Group state is not stable, disabling heartbeats') self.disable() return @@ -1088,14 +1090,14 @@ def _run_once(self): # the session timeout has expired without seeing a # successful heartbeat, so we should probably make sure # the coordinator is still healthy. - log.warning('Heartbeat session expired, marking coordinator dead') + heartbeat_log.warning('Heartbeat session expired, marking coordinator dead') self.coordinator.coordinator_dead('Heartbeat session expired') elif self.coordinator.heartbeat.poll_timeout_expired(): # the poll timeout has expired, which means that the # foreground thread has stalled in between calls to # poll(), so we explicitly leave the group. - log.warning('Heartbeat poll expired, leaving group') + heartbeat_log.warning('Heartbeat poll expired, leaving group') ### XXX # maybe_leave_group acquires client + coordinator lock; # if we hold coordinator lock before calling, we risk deadlock @@ -1106,7 +1108,7 @@ def _run_once(self): elif not self.coordinator.heartbeat.should_heartbeat(): # poll again after waiting for the retry backoff in case # the heartbeat failed or the coordinator disconnected - log.log(0, 'Not ready to heartbeat, waiting') + heartbeat_log.log(0, 'Not ready to heartbeat, waiting') self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) else: From be22ee5a9a3b2f4c477729e80cdf6bf89761f51e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 8 May 2025 12:29:47 -0700 Subject: [PATCH 175/205] Patch Release 2.2.6 --- CHANGES.md | 5 +++++ docs/changelog.rst | 8 ++++++++ kafka/version.py | 2 +- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 07ba59d39..1840038f9 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,8 @@ +# 2.2.6 (May 8, 2025) + +Fixes +* Only disable heartbeat thread once at beginning of join-group (#2617) + # 2.2.5 (May 8, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index cdc1e12e9..005fb2486 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,14 @@ Changelog ========= +2.2.6 (May 8, 2025) +################### + +Fixes +----- +* Only disable heartbeat thread once at beginning of join-group (#2617) + + 2.2.5 (May 8, 2025) ################### diff --git a/kafka/version.py b/kafka/version.py index 3db1b9fed..5d724e8d2 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.5' +__version__ = '2.2.6' From 71c10f419ff2f9719823926fa08f71db342c20c5 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 13 May 2025 10:08:53 -0700 Subject: [PATCH 176/205] Minor Heartbeat updates: catch more exceptions / log configuration / raise KafkaConfigurationError (#2618) --- kafka/coordinator/base.py | 18 +++++++++--------- kafka/coordinator/heartbeat.py | 22 +++++++++++++++++++--- 2 files changed, 28 insertions(+), 12 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 448659e62..6f1d1ee31 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -909,28 +909,28 @@ def _handle_heartbeat_response(self, future, send_time, response): error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: heartbeat_log.debug("Received successful heartbeat response for group %s", - self.group_id) + self.group_id) future.success(None) elif error_type in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): heartbeat_log.warning("Heartbeat failed for group %s: coordinator (node %s)" - " is either not started or not valid", self.group_id, + " is either not started or not valid", self.group_id, self.coordinator()) self.coordinator_dead(error_type()) future.failure(error_type()) elif error_type is Errors.RebalanceInProgressError: heartbeat_log.warning("Heartbeat failed for group %s because it is" - " rebalancing", self.group_id) + " rebalancing", self.group_id) self.request_rejoin() future.failure(error_type()) elif error_type is Errors.IllegalGenerationError: heartbeat_log.warning("Heartbeat failed for group %s: generation id is not " - " current.", self.group_id) + " current.", self.group_id) self.reset_generation() future.failure(error_type()) elif error_type is Errors.UnknownMemberIdError: heartbeat_log.warning("Heartbeat: local member_id was not recognized;" - " this consumer needs to re-join") + " this consumer needs to re-join") self.reset_generation() future.failure(error_type) elif error_type is Errors.GroupAuthorizationFailedError: @@ -1038,16 +1038,16 @@ def close(self, timeout_ms=None): def run(self): try: - heartbeat_log.debug('Heartbeat thread started') + heartbeat_log.debug('Heartbeat thread started: %s', self.coordinator.heartbeat) while not self.closed: self._run_once() except ReferenceError: heartbeat_log.debug('Heartbeat thread closed due to coordinator gc') - except RuntimeError as e: - heartbeat_log.error("Heartbeat thread for group %s failed due to unexpected error: %s", - self.coordinator.group_id, e) + except Exception as e: + heartbeat_log.exception("Heartbeat thread for group %s failed due to unexpected error: %s", + self.coordinator.group_id, e) self.failed = e finally: diff --git a/kafka/coordinator/heartbeat.py b/kafka/coordinator/heartbeat.py index 2f5930b63..edc9f4a36 100644 --- a/kafka/coordinator/heartbeat.py +++ b/kafka/coordinator/heartbeat.py @@ -1,8 +1,13 @@ from __future__ import absolute_import, division import copy +import logging import time +from kafka.errors import KafkaConfigurationError + +log = logging.getLogger(__name__) + class Heartbeat(object): DEFAULT_CONFIG = { @@ -20,9 +25,13 @@ def __init__(self, **configs): self.config[key] = configs[key] if self.config['group_id'] is not None: - assert (self.config['heartbeat_interval_ms'] - <= self.config['session_timeout_ms']), ( - 'Heartbeat interval must be lower than the session timeout') + if self.config['heartbeat_interval_ms'] >= self.config['session_timeout_ms']: + raise KafkaConfigurationError('Heartbeat interval must be lower than the session timeout (%s v %s)' % ( + self.config['heartbeat_interval_ms'], self.config['session_timeout_ms'])) + if self.config['heartbeat_interval_ms'] > (self.config['session_timeout_ms'] / 3): + log.warning('heartbeat_interval_ms is high relative to session_timeout_ms (%s v %s).' + ' Recommend heartbeat interval less than 1/3rd of session timeout', + self.config['heartbeat_interval_ms'], self.config['session_timeout_ms']) self.last_send = -1 * float('inf') self.last_receive = -1 * float('inf') @@ -66,3 +75,10 @@ def reset_timeouts(self): def poll_timeout_expired(self): return (time.time() - self.last_poll) > (self.config['max_poll_interval_ms'] / 1000) + + def __str__(self): + return ("").format(**self.config) From 00a5e6c6c304bdb95d059a6abd448db712bbf8ad Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 13 May 2025 11:00:13 -0700 Subject: [PATCH 177/205] Patch Release 2.2.7 --- CHANGES.md | 5 +++++ docs/changelog.rst | 8 ++++++++ kafka/version.py | 2 +- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 1840038f9..1d86bcc1e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,8 @@ +# 2.2.7 (May 13, 2025) + +Fixes +* Minor Heartbeat updates: catch more exceptions / log configuration / raise KafkaConfigurationError (#2618) + # 2.2.6 (May 8, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 005fb2486..ed0beabdb 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,14 @@ Changelog ========= +2.2.7 (May 13, 2025) +#################### + +Fixes +----- +* Minor Heartbeat updates: catch more exceptions / log configuration / raise KafkaConfigurationError (#2618) + + 2.2.6 (May 8, 2025) ################### diff --git a/kafka/version.py b/kafka/version.py index 5d724e8d2..04e669086 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.6' +__version__ = '2.2.7' From 8a424e91485fbffec546e99ce35bfcf99dad0be9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 15 May 2025 13:27:44 -0700 Subject: [PATCH 178/205] Acquire client lock in heartbeat thread before sending requests (#2620) --- kafka/coordinator/base.py | 29 +++++++++++++++---------- test/integration/test_consumer_group.py | 28 ++++++++++++++++++++++-- 2 files changed, 43 insertions(+), 14 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 6f1d1ee31..852157811 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -309,7 +309,7 @@ def _reset_find_coordinator_future(self, result): self._find_coordinator_future = None def lookup_coordinator(self): - with self._lock: + with self._client._lock, self._lock: if self._find_coordinator_future is not None: return self._find_coordinator_future @@ -883,6 +883,7 @@ def _handle_leave_group_response(self, response): def _send_heartbeat_request(self): """Send a heartbeat request""" + # Note: acquire both client + coordinator lock before calling if self.coordinator_unknown(): e = Errors.CoordinatorNotAvailableError(self.coordinator_id) return Future().failure(e) @@ -1054,7 +1055,9 @@ def run(self): heartbeat_log.debug('Heartbeat thread closed') def _run_once(self): - with self.coordinator._client._lock, self.coordinator._lock: + self.coordinator._client._lock.acquire() + self.coordinator._lock.acquire() + try: if self.enabled and self.coordinator.state is MemberState.STABLE: # TODO: When consumer.wakeup() is implemented, we need to # disable here to prevent propagating an exception to this @@ -1063,27 +1066,26 @@ def _run_once(self): # failure callback in consumer poll self.coordinator._client.poll(timeout_ms=0) - with self.coordinator._lock: if not self.enabled: heartbeat_log.debug('Heartbeat disabled. Waiting') + self.coordinator._client._lock.release() self.coordinator._lock.wait() heartbeat_log.debug('Heartbeat re-enabled.') - return - if self.coordinator.state is not MemberState.STABLE: + elif self.coordinator.state is not MemberState.STABLE: # the group is not stable (perhaps because we left the # group or because the coordinator kicked us out), so # disable heartbeats and wait for the main thread to rejoin. heartbeat_log.debug('Group state is not stable, disabling heartbeats') self.disable() - return - if self.coordinator.coordinator_unknown(): + elif self.coordinator.coordinator_unknown(): future = self.coordinator.lookup_coordinator() if not future.is_done or future.failed(): # the immediate future check ensures that we backoff # properly in the case that no brokers are available # to connect to (and the future is automatically failed). + self.coordinator._client._lock.release() self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) elif self.coordinator.heartbeat.session_timeout_expired(): @@ -1098,17 +1100,13 @@ def _run_once(self): # foreground thread has stalled in between calls to # poll(), so we explicitly leave the group. heartbeat_log.warning('Heartbeat poll expired, leaving group') - ### XXX - # maybe_leave_group acquires client + coordinator lock; - # if we hold coordinator lock before calling, we risk deadlock - # release() is safe here because this is the last code in the current context - self.coordinator._lock.release() self.coordinator.maybe_leave_group() elif not self.coordinator.heartbeat.should_heartbeat(): # poll again after waiting for the retry backoff in case # the heartbeat failed or the coordinator disconnected heartbeat_log.log(0, 'Not ready to heartbeat, waiting') + self.coordinator._client._lock.release() self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) else: @@ -1116,6 +1114,13 @@ def _run_once(self): future = self.coordinator._send_heartbeat_request() future.add_callback(self._handle_heartbeat_success) future.add_errback(self._handle_heartbeat_failure) + finally: + self.coordinator._lock.release() + try: + # Possibly released in block above to allow coordinator lock wait() + self.coordinator._client._lock.release() + except RuntimeError: + pass def _handle_heartbeat_success(self, result): with self.coordinator._lock: diff --git a/test/integration/test_consumer_group.py b/test/integration/test_consumer_group.py index b2908c757..eed570074 100644 --- a/test/integration/test_consumer_group.py +++ b/test/integration/test_consumer_group.py @@ -125,6 +125,20 @@ def consumer_thread(i): for partition in range(num_partitions)]) logging.info('Assignment looks good!') + logging.info('Verifying heartbeats') + while True: + for c in range(num_consumers): + heartbeat = consumers[c]._coordinator.heartbeat + last_hb = time.time() - 0.5 + if (heartbeat.heartbeat_failed or + heartbeat.last_receive < last_hb or + heartbeat.last_reset > last_hb): + time.sleep(0.1) + continue + else: + break + logging.info('Heartbeats look good') + finally: logging.info('Shutting down %s consumers', num_consumers) for c in range(num_consumers): @@ -163,18 +177,28 @@ def test_heartbeat_thread(kafka_broker, topic): heartbeat_interval_ms=500) # poll until we have joined group / have assignment + start = time.time() while not consumer.assignment(): consumer.poll(timeout_ms=100) assert consumer._coordinator.state is MemberState.STABLE last_poll = consumer._coordinator.heartbeat.last_poll - last_beat = consumer._coordinator.heartbeat.last_send + + # wait until we receive first heartbeat + while consumer._coordinator.heartbeat.last_receive < start: + time.sleep(0.1) + + last_send = consumer._coordinator.heartbeat.last_send + last_recv = consumer._coordinator.heartbeat.last_receive + assert last_poll > start + assert last_send > start + assert last_recv > start timeout = time.time() + 30 while True: if time.time() > timeout: raise RuntimeError('timeout waiting for heartbeat') - if consumer._coordinator.heartbeat.last_send > last_beat: + if consumer._coordinator.heartbeat.last_receive > last_recv: break time.sleep(0.5) From 2f95590fc441bb5939c2dd136d3b54f701013f5a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 15 May 2025 14:26:34 -0700 Subject: [PATCH 179/205] Wait for next heartbeat in thread loop; check for connected coordinator (#2622) --- kafka/coordinator/base.py | 38 +++++++++++++++++++++++--------------- test/test_coordinator.py | 1 + 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 852157811..b128e5548 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -250,6 +250,11 @@ def coordinator(self): else: return self.coordinator_id + def connected(self): + """Return True iff the coordinator node is connected""" + with self._lock: + return self.coordinator_id is not None and self._client.connected(self.coordinator_id) + def ensure_coordinator_ready(self, timeout_ms=None): """Block until the coordinator for this group is known. @@ -1058,28 +1063,28 @@ def _run_once(self): self.coordinator._client._lock.acquire() self.coordinator._lock.acquire() try: - if self.enabled and self.coordinator.state is MemberState.STABLE: - # TODO: When consumer.wakeup() is implemented, we need to - # disable here to prevent propagating an exception to this - # heartbeat thread - # must get client._lock, or maybe deadlock at heartbeat - # failure callback in consumer poll - self.coordinator._client.poll(timeout_ms=0) - if not self.enabled: heartbeat_log.debug('Heartbeat disabled. Waiting') self.coordinator._client._lock.release() self.coordinator._lock.wait() - heartbeat_log.debug('Heartbeat re-enabled.') + if self.enabled: + heartbeat_log.debug('Heartbeat re-enabled.') + return - elif self.coordinator.state is not MemberState.STABLE: + if self.coordinator.state is not MemberState.STABLE: # the group is not stable (perhaps because we left the # group or because the coordinator kicked us out), so # disable heartbeats and wait for the main thread to rejoin. heartbeat_log.debug('Group state is not stable, disabling heartbeats') self.disable() + return + + # TODO: When consumer.wakeup() is implemented, we need to + # disable here to prevent propagating an exception to this + # heartbeat thread + self.coordinator._client.poll(timeout_ms=0) - elif self.coordinator.coordinator_unknown(): + if self.coordinator.coordinator_unknown(): future = self.coordinator.lookup_coordinator() if not future.is_done or future.failed(): # the immediate future check ensures that we backoff @@ -1088,6 +1093,10 @@ def _run_once(self): self.coordinator._client._lock.release() self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) + elif not self.coordinator.connected(): + self.coordinator._client._lock.release() + self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) + elif self.coordinator.heartbeat.session_timeout_expired(): # the session timeout has expired without seeing a # successful heartbeat, so we should probably make sure @@ -1103,11 +1112,10 @@ def _run_once(self): self.coordinator.maybe_leave_group() elif not self.coordinator.heartbeat.should_heartbeat(): - # poll again after waiting for the retry backoff in case - # the heartbeat failed or the coordinator disconnected - heartbeat_log.log(0, 'Not ready to heartbeat, waiting') + next_hb = self.coordinator.heartbeat.time_to_next_heartbeat() + heartbeat_log.debug('Waiting %0.1f secs to send next heartbeat', next_hb) self.coordinator._client._lock.release() - self.coordinator._lock.wait(self.coordinator.config['retry_backoff_ms'] / 1000) + self.coordinator._lock.wait(next_hb) else: self.coordinator.heartbeat.sent_heartbeat() diff --git a/test/test_coordinator.py b/test/test_coordinator.py index 251de566a..4ffe1d28c 100644 --- a/test/test_coordinator.py +++ b/test/test_coordinator.py @@ -658,6 +658,7 @@ def test_heartbeat(mocker, patched_coord): heartbeat.enable() patched_coord.state = MemberState.STABLE mocker.spy(patched_coord, '_send_heartbeat_request') + mocker.patch.object(patched_coord, 'connected', return_value=True) mocker.patch.object(patched_coord.heartbeat, 'should_heartbeat', return_value=True) heartbeat._run_once() assert patched_coord._send_heartbeat_request.call_count == 1 From 8f77cc8b5e21be2163e7ff9bb23df1073aa66a94 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 15 May 2025 14:27:28 -0700 Subject: [PATCH 180/205] Update offset commit error handling; use RebalanceInProgressError if applicable (#2623) --- kafka/coordinator/base.py | 12 +++++++++++- kafka/coordinator/consumer.py | 26 +++++++++++++++++++++----- kafka/errors.py | 9 +-------- 3 files changed, 33 insertions(+), 14 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index b128e5548..c408c2607 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -5,6 +5,7 @@ import logging import threading import time +import warnings import weakref from kafka.vendor import six @@ -797,7 +798,7 @@ def coordinator_dead(self, error): self.coordinator_id, self.group_id, error) self.coordinator_id = None - def generation(self): + def generation_if_stable(self): """Get the current generation state if the group is stable. Returns: the current generation or None if the group is unjoined/rebalancing @@ -807,6 +808,15 @@ def generation(self): return None return self._generation + # deprecated + def generation(self): + warnings.warn("Function coordinator.generation() has been renamed to generation_if_stable()", + DeprecationWarning, stacklevel=2) + return self.generation_if_stable() + + def rebalance_in_progress(self): + return self.state is MemberState.REBALANCING + def reset_generation(self, member_id=UNKNOWN_MEMBER_ID): """Reset the generation and member_id because we have fallen out of the group.""" with self._lock: diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index 3db00d72c..ddd413b82 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -608,6 +608,11 @@ def _send_offset_commit_request(self, offsets): if node_id is None: return Future().failure(Errors.CoordinatorNotAvailableError) + # Verify node is ready + if not self._client.ready(node_id, metadata_priority=False): + log.debug("Node %s not ready -- failing offset commit request", + node_id) + return Future().failure(Errors.NodeNotReadyError) # create the offset commit request offset_data = collections.defaultdict(dict) @@ -616,7 +621,7 @@ def _send_offset_commit_request(self, offsets): version = self._client.api_version(OffsetCommitRequest, max_version=6) if version > 1 and self._subscription.partitions_auto_assigned(): - generation = self.generation() + generation = self.generation_if_stable() else: generation = Generation.NO_GENERATION @@ -625,7 +630,18 @@ def _send_offset_commit_request(self, offsets): # and let the user rejoin the group in poll() if generation is None: log.info("Failing OffsetCommit request since the consumer is not part of an active group") - return Future().failure(Errors.CommitFailedError('Group rebalance in progress')) + if self.rebalance_in_progress(): + # if the client knows it is already rebalancing, we can use RebalanceInProgressError instead of + # CommitFailedError to indicate this is not a fatal error + return Future().failure(Errors.RebalanceInProgressError( + "Offset commit cannot be completed since the" + " consumer is undergoing a rebalance for auto partition assignment. You can try completing the rebalance" + " by calling poll() and then retry the operation.")) + else: + return Future().failure(Errors.CommitFailedError( + "Offset commit cannot be completed since the" + " consumer is not part of an active group for auto partition assignment; it is likely that the consumer" + " was kicked out of the group.")) if version == 0: request = OffsetCommitRequest[version]( @@ -756,7 +772,7 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): # However, we do not need to reset generations and just request re-join, such that # if the caller decides to proceed and poll, it would still try to proceed and re-join normally. self.request_rejoin() - future.failure(Errors.CommitFailedError('Group rebalance in progress')) + future.failure(Errors.CommitFailedError(error_type())) return elif error_type in (Errors.UnknownMemberIdError, Errors.IllegalGenerationError): @@ -765,7 +781,7 @@ def _handle_offset_commit_response(self, offsets, future, send_time, response): log.warning("OffsetCommit for group %s failed: %s", self.group_id, error) self.reset_generation() - future.failure(Errors.CommitFailedError()) + future.failure(Errors.CommitFailedError(error_type())) return else: log.error("Group %s failed to commit partition %s at offset" @@ -804,7 +820,7 @@ def _send_offset_fetch_request(self, partitions): return Future().failure(Errors.CoordinatorNotAvailableError) # Verify node is ready - if not self._client.ready(node_id): + if not self._client.ready(node_id, metadata_priority=False): log.debug("Node %s not ready -- failing offset fetch request", node_id) return Future().failure(Errors.NodeNotReadyError) diff --git a/kafka/errors.py b/kafka/errors.py index 898582615..ac4eadfec 100644 --- a/kafka/errors.py +++ b/kafka/errors.py @@ -24,14 +24,7 @@ class CommitFailedError(KafkaError): def __init__(self, *args): if not args: args = ("Commit cannot be completed since the group has already" - " rebalanced and assigned the partitions to another member." - " This means that the time between subsequent calls to poll()" - " was longer than the configured max_poll_interval_ms, which" - " typically implies that the poll loop is spending too much" - " time message processing. You can address this either by" - " increasing the rebalance timeout with max_poll_interval_ms," - " or by reducing the maximum size of batches returned in poll()" - " with max_poll_records.",) + " rebalanced and assigned the partitions to another member.",) super(CommitFailedError, self).__init__(*args) From 5de666a17b9eb73a1b4c4a7201cd418e281d5901 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 15 May 2025 14:57:59 -0700 Subject: [PATCH 181/205] More coordinator / heartbeat logging (#2621) --- kafka/coordinator/base.py | 65 +++++++++++++++++++++++++++------------ 1 file changed, 45 insertions(+), 20 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index c408c2607..d2ba8873a 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -44,6 +44,9 @@ def __eq__(self, other): self.member_id == other.member_id and self.protocol == other.protocol) + def __str__(self): + return "" % (self.generation_id, self.member_id, self.protocol) + Generation.NO_GENERATION = Generation(DEFAULT_GENERATION_ID, UNKNOWN_MEMBER_ID, None) @@ -404,17 +407,16 @@ def _handle_join_success(self, member_assignment_bytes): # will be invoked even if the consumer is woken up before # finishing the rebalance with self._lock: - log.info("Successfully joined group %s with generation %s", - self.group_id, self._generation.generation_id) self.state = MemberState.STABLE if self._heartbeat_thread: self._heartbeat_thread.enable() - def _handle_join_failure(self, _): + def _handle_join_failure(self, exception): # we handle failures below after the request finishes. # if the join completes after having been woken up, # the exception is ignored and we will rejoin with self._lock: + log.info("Failed to join group %s: %s", self.group_id, exception) self.state = MemberState.UNJOINED def ensure_active_group(self, timeout_ms=None): @@ -572,10 +574,9 @@ def _failed_request(self, node_id, request, future, error): future.failure(error) def _handle_join_group_response(self, future, send_time, response): + log.debug("Received JoinGroup response: %s", response) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - log.debug("Received successful JoinGroup response for group %s: %s", - self.group_id, response) if self._sensors: self._sensors.join_latency.record((time.time() - send_time) * 1000) with self._lock: @@ -589,6 +590,7 @@ def _handle_join_group_response(self, future, send_time, response): response.member_id, response.group_protocol) + log.info("Successfully joined group %s %s", self.group_id, self._generation) if response.leader_id == response.member_id: log.info("Elected group leader -- performing partition" " assignments using %s", self._generation.protocol) @@ -597,24 +599,24 @@ def _handle_join_group_response(self, future, send_time, response): self._on_join_follower().chain(future) elif error_type is Errors.CoordinatorLoadInProgressError: - log.debug("Attempt to join group %s rejected since coordinator %s" - " is loading the group.", self.group_id, self.coordinator_id) + log.info("Attempt to join group %s rejected since coordinator %s" + " is loading the group.", self.group_id, self.coordinator_id) # backoff and retry future.failure(error_type(response)) elif error_type is Errors.UnknownMemberIdError: # reset the member id and retry immediately error = error_type(self._generation.member_id) self.reset_generation() - log.debug("Attempt to join group %s failed due to unknown member id", - self.group_id) + log.info("Attempt to join group %s failed due to unknown member id", + self.group_id) future.failure(error) elif error_type in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): # re-discover the coordinator and retry with backoff self.coordinator_dead(error_type()) - log.debug("Attempt to join group %s failed due to obsolete " - "coordinator information: %s", self.group_id, - error_type.__name__) + log.info("Attempt to join group %s failed due to obsolete " + "coordinator information: %s", self.group_id, + error_type.__name__) future.failure(error_type()) elif error_type in (Errors.InconsistentGroupProtocolError, Errors.InvalidSessionTimeoutError, @@ -625,12 +627,21 @@ def _handle_join_group_response(self, future, send_time, response): 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)) 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. + log.info("Received member id %s for group %s; will retry join-group", + response.member_id, self.group_id) self.reset_generation(response.member_id) future.failure(error_type()) + elif error_type is Errors.RebalanceInProgressError: + log.info("Attempt to join group %s failed due to RebalanceInProgressError," + " which could indicate a replication timeout on the broker. Will retry.", + self.group_id) + future.failure(error_type()) else: # unexpected error, throw the exception error = error_type() @@ -699,6 +710,7 @@ def _send_sync_group_request(self, request): return future def _handle_sync_group_response(self, future, send_time, response): + log.debug("Received SyncGroup response: %s", response) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: if self._sensors: @@ -745,13 +757,13 @@ def _send_group_coordinator_request(self): e = Errors.NodeNotReadyError(node_id) return Future().failure(e) - log.debug("Sending group coordinator request for group %s to broker %s", - self.group_id, node_id) version = self._client.api_version(FindCoordinatorRequest, max_version=2) if version == 0: request = FindCoordinatorRequest[version](self.group_id) else: request = FindCoordinatorRequest[version](self.group_id, 0) + log.debug("Sending group coordinator request for group %s to broker %s: %s", + self.group_id, node_id, request) future = Future() _f = self._client.send(node_id, request) _f.add_callback(self._handle_group_coordinator_response, future) @@ -880,6 +892,7 @@ def maybe_leave_group(self, timeout_ms=None): 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.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) future.add_errback(log.error, "LeaveGroup request failed: %s") @@ -888,10 +901,11 @@ def maybe_leave_group(self, timeout_ms=None): self.reset_generation() def _handle_leave_group_response(self, response): + log.debug("Received LeaveGroupResponse: %s", response) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - log.debug("LeaveGroup request for group %s returned successfully", - self.group_id) + log.info("LeaveGroup request for group %s returned successfully", + self.group_id) else: log.error("LeaveGroup request for group %s failed with error: %s", self.group_id, error_type()) @@ -911,7 +925,7 @@ def _send_heartbeat_request(self): request = HeartbeatRequest[version](self.group_id, self._generation.generation_id, self._generation.member_id) - heartbeat_log.debug("Heartbeat: %s[%s] %s", request.group, request.generation_id, request.member_id) # pylint: disable-msg=no-member + heartbeat_log.debug("Sending HeartbeatRequest to %s: %s", self.coordinator_id, request) future = Future() _f = self._client.send(self.coordinator_id, request) _f.add_callback(self._handle_heartbeat_response, future, time.time()) @@ -922,10 +936,10 @@ def _send_heartbeat_request(self): def _handle_heartbeat_response(self, future, send_time, response): if self._sensors: self._sensors.heartbeat_latency.record((time.time() - send_time) * 1000) + heartbeat_log.debug("Received heartbeat response for group %s: %s", + self.group_id, response) error_type = Errors.for_code(response.error_code) if error_type is Errors.NoError: - heartbeat_log.debug("Received successful heartbeat response for group %s", - self.group_id) future.success(None) elif error_type in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): @@ -1118,7 +1132,13 @@ def _run_once(self): # the poll timeout has expired, which means that the # foreground thread has stalled in between calls to # poll(), so we explicitly leave the group. - heartbeat_log.warning('Heartbeat poll expired, leaving group') + heartbeat_log.warning( + "Consumer poll timeout has expired. This means the time between subsequent calls to poll()" + " was longer than the configured max_poll_interval_ms, which typically implies that" + " the poll loop is spending too much time processing messages. You can address this" + " either by increasing max_poll_interval_ms or by reducing the maximum size of batches" + " returned in poll() with max_poll_records." + ) self.coordinator.maybe_leave_group() elif not self.coordinator.heartbeat.should_heartbeat(): @@ -1128,10 +1148,12 @@ def _run_once(self): self.coordinator._lock.wait(next_hb) else: + heartbeat_log.debug('Sending heartbeat for group %s %s', self.coordinator.group_id, self.coordinator._generation) self.coordinator.heartbeat.sent_heartbeat() future = self.coordinator._send_heartbeat_request() future.add_callback(self._handle_heartbeat_success) future.add_errback(self._handle_heartbeat_failure) + finally: self.coordinator._lock.release() try: @@ -1142,6 +1164,7 @@ def _run_once(self): def _handle_heartbeat_success(self, result): with self.coordinator._lock: + heartbeat_log.debug('Heartbeat success') self.coordinator.heartbeat.received_heartbeat() def _handle_heartbeat_failure(self, exception): @@ -1152,8 +1175,10 @@ def _handle_heartbeat_failure(self, exception): # member in the group for as long as the duration of the # rebalance timeout. If we stop sending heartbeats, however, # then the session timeout may expire before we can rejoin. + heartbeat_log.debug('Treating RebalanceInProgressError as successful heartbeat') self.coordinator.heartbeat.received_heartbeat() else: + heartbeat_log.debug('Heartbeat failure: %s', exception) self.coordinator.heartbeat.fail_heartbeat() # wake up the thread if it's sleeping to reschedule the heartbeat self.coordinator._lock.notify() From 452e354c5d405de54f5f06fe0857be87cf7dd040 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 15 May 2025 15:11:12 -0700 Subject: [PATCH 182/205] Log all SyncGroupResponse errors as info+ --- kafka/coordinator/base.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index d2ba8873a..9fffd19c6 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -723,19 +723,19 @@ def _handle_sync_group_response(self, future, send_time, response): if error_type is Errors.GroupAuthorizationFailedError: future.failure(error_type(self.group_id)) elif error_type is Errors.RebalanceInProgressError: - log.debug("SyncGroup for group %s failed due to coordinator" - " rebalance", self.group_id) + log.info("SyncGroup for group %s failed due to coordinator" + " rebalance", self.group_id) future.failure(error_type(self.group_id)) elif error_type in (Errors.UnknownMemberIdError, Errors.IllegalGenerationError): error = error_type() - log.debug("SyncGroup for group %s failed due to %s", self.group_id, error) + log.info("SyncGroup for group %s failed due to %s", self.group_id, error) self.reset_generation() future.failure(error) elif error_type in (Errors.CoordinatorNotAvailableError, Errors.NotCoordinatorError): error = error_type() - log.debug("SyncGroup for group %s failed due to %s", self.group_id, error) + log.info("SyncGroup for group %s failed due to %s", self.group_id, error) self.coordinator_dead(error) future.failure(error) else: From 62a7d80d791fdd5c489a0ae8ca333800280339b0 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 20 May 2025 19:49:56 -0700 Subject: [PATCH 183/205] Fix timeout seconds error message in KafkaProducer (#2627) --- kafka/producer/kafka.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 66208bbe1..2416cc3ef 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -944,7 +944,7 @@ def _wait_on_metadata(self, topic, max_wait_ms): """ # add topic to metadata topic list if it is not there already. self._sender.add_topic(topic) - timer = Timer(max_wait_ms, "Failed to update metadata after %.1f secs." % (max_wait_ms * 1000,)) + timer = Timer(max_wait_ms, "Failed to update metadata after %.1f secs." % (max_wait_ms / 1000,)) metadata_event = None while True: partitions = self._metadata.partitions_for_topic(topic) @@ -962,7 +962,7 @@ def _wait_on_metadata(self, topic, max_wait_ms): metadata_event.wait(timer.timeout_ms / 1000) if not metadata_event.is_set(): raise Errors.KafkaTimeoutError( - "Failed to update metadata after %.1f secs." % (max_wait_ms * 1000,)) + "Failed to update metadata after %.1f secs." % (max_wait_ms / 1000,)) elif topic in self._metadata.unauthorized_topics: raise Errors.TopicAuthorizationFailedError(set([topic])) else: From 95c2f3af7e074b01fd0e7178e0314739c74879a6 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 20 May 2025 19:58:28 -0700 Subject: [PATCH 184/205] Dont mark coordinator dead on ThrottlingQuotaExceededError --- kafka/coordinator/base.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka/coordinator/base.py b/kafka/coordinator/base.py index 9fffd19c6..0eb7f0eec 100644 --- a/kafka/coordinator/base.py +++ b/kafka/coordinator/base.py @@ -562,8 +562,9 @@ def _send_join_group_request(self): def _failed_request(self, node_id, request, future, error): # Marking coordinator dead - # unless the error is caused by internal client pipelining + # unless the error is caused by internal client pipelining or throttling if not isinstance(error, (Errors.NodeNotReadyError, + Errors.ThrottlingQuotaExceededError, Errors.TooManyInFlightRequests)): log.error('Error sending %s to node %s [%s]', request.__class__.__name__, node_id, error) From bcbd1b71a0718cd0182bba9525b1a11b7ea238ec Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 20 May 2025 19:58:53 -0700 Subject: [PATCH 185/205] Patch Release 2.2.8 --- CHANGES.md | 12 ++++++++++++ docs/changelog.rst | 16 ++++++++++++++++ kafka/version.py | 2 +- 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 1d86bcc1e..930fdb471 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,15 @@ +# 2.2.8 (May 20, 2025) + +Fixes +* Wait for next heartbeat in thread loop; check for connected coordinator (#2622) +* Acquire client lock in heartbeat thread before sending requests (#2620) + +Logging / Error Messages +* Log all SyncGroupResponse errors as info+ +* More coordinator / heartbeat logging (#2621) +* Fix timeout seconds error message in KafkaProducer (#2627) +* Update offset commit error handling; use RebalanceInProgressError if applicable (#2623) + # 2.2.7 (May 13, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index ed0beabdb..50cbdb2b2 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,22 @@ Changelog ========= +2.2.8 (May 20, 2025) +#################### + +Fixes +----- +* Wait for next heartbeat in thread loop; check for connected coordinator (#2622) +* Acquire client lock in heartbeat thread before sending requests (#2620) + +Logging / Error Messages +------------------------ +* Log all SyncGroupResponse errors as info+ +* More coordinator / heartbeat logging (#2621) +* Fix timeout seconds error message in KafkaProducer (#2627) +* Update offset commit error handling; use RebalanceInProgressError if applicable (#2623) + + 2.2.7 (May 13, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 04e669086..bc570f207 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.7' +__version__ = '2.2.8' From 3463f595763760314aab6172d276896dbb0329d8 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 21 May 2025 16:15:30 -0700 Subject: [PATCH 186/205] Do not reset fetch positions if offset commit fetch times out (#2629) --- kafka/consumer/fetcher.py | 7 ++++++- kafka/consumer/group.py | 21 +++++++++------------ kafka/coordinator/consumer.py | 3 ++- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/kafka/consumer/fetcher.py b/kafka/consumer/fetcher.py index 42e2d660c..b083deb1a 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -178,6 +178,9 @@ def reset_offsets_if_needed(self): Arguments: partitions ([TopicPartition]): the partitions that need offsets reset + Returns: + bool: True if any partitions need reset; otherwise False (no reset pending) + Raises: NoOffsetForPartitionError: if no offset reset strategy is defined KafkaTimeoutError if timeout_ms provided @@ -189,7 +192,8 @@ def reset_offsets_if_needed(self): partitions = self._subscriptions.partitions_needing_reset() if not partitions: - return + return False + log.debug('Resetting offsets for %s', partitions) offset_resets = dict() for tp in partitions: @@ -198,6 +202,7 @@ def reset_offsets_if_needed(self): offset_resets[tp] = ts self._reset_offsets_async(offset_resets) + return True def offsets_by_times(self, timestamps, timeout_ms=None): """Fetch offset for each partition passed in ``timestamps`` map. diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index ce3cf9203..d966ea009 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -1124,7 +1124,7 @@ def _update_fetch_positions(self, timeout_ms=None): partitions (List[TopicPartition]): The partitions that need updating fetch positions. - Returns True if fetch positions updated, False if timeout + Returns True if fetch positions updated, False if timeout or async reset is pending Raises: NoOffsetForPartitionError: If no offset is stored for a given @@ -1135,15 +1135,13 @@ def _update_fetch_positions(self, timeout_ms=None): if (self.config['api_version'] >= (0, 8, 1) and self.config['group_id'] is not None): - try: - # If there are any partitions which do not have a valid position and are not - # awaiting reset, then we need to fetch committed offsets. We will only do a - # coordinator lookup if there are partitions which have missing positions, so - # a consumer with manually assigned partitions can avoid a coordinator dependence - # by always ensuring that assigned partitions have an initial position. - self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=timeout_ms) - except KafkaTimeoutError: - pass + # If there are any partitions which do not have a valid position and are not + # awaiting reset, then we need to fetch committed offsets. We will only do a + # coordinator lookup if there are partitions which have missing positions, so + # a consumer with manually assigned partitions can avoid a coordinator dependence + # by always ensuring that assigned partitions have an initial position. + if not self._coordinator.refresh_committed_offsets_if_needed(timeout_ms=timeout_ms): + return False # If there are partitions still needing a position and a reset policy is defined, # request reset using the default policy. If no reset strategy is defined and there @@ -1152,8 +1150,7 @@ def _update_fetch_positions(self, timeout_ms=None): # Finally send an asynchronous request to lookup and update the positions of any # partitions which are awaiting reset. - self._fetcher.reset_offsets_if_needed() - return False + return not self._fetcher.reset_offsets_if_needed() def _message_generator_v2(self): timeout_ms = 1000 * max(0, self._consumer_timeout - time.time()) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index ddd413b82..d6fc802d9 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -427,7 +427,8 @@ def fetch_committed_offsets(self, partitions, timeout_ms=None): future_key = frozenset(partitions) timer = Timer(timeout_ms) while True: - self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms) + if not self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms): + timer.maybe_raise() # contact coordinator to fetch committed offsets if future_key in self._offset_fetch_futures: From bd2171322e5de98d68545987683aedb5c5c2eb66 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 21 May 2025 16:15:45 -0700 Subject: [PATCH 187/205] More / updated debug logging for coordinator / consumer (#2630) --- kafka/consumer/group.py | 7 +++++-- kafka/coordinator/consumer.py | 5 +++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/kafka/consumer/group.py b/kafka/consumer/group.py index d966ea009..4eb9e2ab4 100644 --- a/kafka/consumer/group.py +++ b/kafka/consumer/group.py @@ -699,6 +699,7 @@ def _poll_once(self, timer, max_records, update_offsets=True): dict: Map of topic to list of records (may be empty). """ if not self._coordinator.poll(timeout_ms=timer.timeout_ms): + log.debug('poll: timeout during coordinator.poll(); returning early') return {} has_all_fetch_positions = self._update_fetch_positions(timeout_ms=timer.timeout_ms) @@ -706,13 +707,13 @@ def _poll_once(self, timer, max_records, update_offsets=True): # If data is available already, e.g. from a previous network client # poll() call to commit, then just return it immediately records, partial = self._fetcher.fetched_records(max_records, update_offsets=update_offsets) - log.debug('Fetched records: %s, %s', records, partial) + log.debug('poll: fetched records: %s, %s', records, partial) # Before returning the fetched records, we can send off the # next round of fetches and avoid block waiting for their # responses to enable pipelining while the user is handling the # fetched records. if not partial: - log.debug("Sending fetches") + log.debug("poll: Sending fetches") futures = self._fetcher.send_fetches() if len(futures): self._client.poll(timeout_ms=0) @@ -724,12 +725,14 @@ def _poll_once(self, timer, max_records, update_offsets=True): # 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) 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']) self._client.poll(timeout_ms=poll_timeout_ms) # after the long poll, we should check whether the group needs to rebalance # prior to returning data so that the group can stabilize faster if self._coordinator.need_rejoin(): + log.debug('poll: coordinator needs rejoin; returning early') return {} records, _ = self._fetcher.fetched_records(max_records, update_offsets=update_offsets) diff --git a/kafka/coordinator/consumer.py b/kafka/coordinator/consumer.py index d6fc802d9..dca10ae1a 100644 --- a/kafka/coordinator/consumer.py +++ b/kafka/coordinator/consumer.py @@ -274,6 +274,7 @@ def poll(self, timeout_ms=None): try: self._invoke_completed_offset_commit_callbacks() if not self.ensure_coordinator_ready(timeout_ms=timer.timeout_ms): + log.debug('coordinator.poll: timeout in ensure_coordinator_ready; returning early') return False if self.config['api_version'] >= (0, 9) and self._subscription.partitions_auto_assigned(): @@ -293,9 +294,11 @@ def poll(self, timeout_ms=None): metadata_update = self._client.cluster.request_update() self._client.poll(future=metadata_update, timeout_ms=timer.timeout_ms) if not metadata_update.is_done: + log.debug('coordinator.poll: timeout updating metadata; returning early') return False if not self.ensure_active_group(timeout_ms=timer.timeout_ms): + log.debug('coordinator.poll: timeout in ensure_active_group; returning early') return False self.poll_heartbeat() @@ -723,6 +726,7 @@ def _send_offset_commit_request(self, offsets): return future def _handle_offset_commit_response(self, offsets, future, send_time, response): + log.debug("Received OffsetCommitResponse: %s", response) # TODO look at adding request_latency_ms to response (like java kafka) if self._consumer_sensors: self._consumer_sensors.commit_latency.record((time.time() - send_time) * 1000) @@ -849,6 +853,7 @@ def _send_offset_fetch_request(self, partitions): return future def _handle_offset_fetch_response(self, future, response): + log.debug("Received OffsetFetchResponse: %s", response) if response.API_VERSION >= 2 and response.error_code != Errors.NoError.errno: error_type = Errors.for_code(response.error_code) log.debug("Offset fetch failed: %s", error_type.__name__) From 48dd596462be5fc7cda004c85b26db823c184a39 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 21 May 2025 16:17:31 -0700 Subject: [PATCH 188/205] Patch Release 2.2.9 --- CHANGES.md | 8 ++++++++ docs/changelog.rst | 12 ++++++++++++ kafka/version.py | 2 +- 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 930fdb471..df08c71d7 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,11 @@ +# 2.2.9 (May 21, 2025) + +Fixes +* Do not reset fetch positions if offset commit fetch times out (#2629) + +Logging / Error Messages +* More / updated debug logging for coordinator / consumer (#2630) + # 2.2.8 (May 20, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 50cbdb2b2..6da6d849e 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,18 @@ Changelog ========= +2.2.9 (May 21, 2025) +#################### + +Fixes +----- +* Do not reset fetch positions if offset commit fetch times out (#2629) + +Logging / Error Messages +------------------------ +* More / updated debug logging for coordinator / consumer (#2630) + + 2.2.8 (May 20, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index bc570f207..00c3889f6 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.8' +__version__ = '2.2.9' From 7b7f3afceecccc80d619d02b0fa33bcb0013b2c9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 22 May 2025 13:55:16 -0700 Subject: [PATCH 189/205] Fix sasl gssapi plugin: do not rely on client_ctx.complete in auth_bytes() (#2631) --- kafka/sasl/gssapi.py | 17 +++++++++++----- test/sasl/test_gssapi.py | 42 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 54 insertions(+), 5 deletions(-) create mode 100644 test/sasl/test_gssapi.py diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index be84269da..6a4896585 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -26,14 +26,15 @@ def __init__(self, **config): raise ValueError('sasl_kerberos_service_name or sasl_kerberos_name required for GSSAPI sasl configuration') self._is_done = False self._is_authenticated = False + self.gssapi_name = None if config.get('sasl_kerberos_name', None) is not None: self.auth_id = str(config['sasl_kerberos_name']) + if isinstance(config['sasl_kerberos_name'], gssapi.Name): + self.gssapi_name = config['sasl_kerberos_name'] else: kerberos_domain_name = config.get('sasl_kerberos_domain_name', '') or config.get('host', '') self.auth_id = config['sasl_kerberos_service_name'] + '@' + kerberos_domain_name - if isinstance(config.get('sasl_kerberos_name', None), gssapi.Name): - self.gssapi_name = config['sasl_kerberos_name'] - else: + if self.gssapi_name is None: self.gssapi_name = gssapi.Name(self.auth_id, name_type=gssapi.NameType.hostbased_service).canonicalize(gssapi.MechType.kerberos) self._client_ctx = gssapi.SecurityContext(name=self.gssapi_name, usage='initiate') self._next_token = self._client_ctx.step(None) @@ -43,9 +44,8 @@ def auth_bytes(self): # so mark is_done after the final auth_bytes are provided # in practice we'll still receive a response when using SaslAuthenticate # but not when using the prior unframed approach. - if self._client_ctx.complete: + if self._is_authenticated: self._is_done = True - self._is_authenticated = True return self._next_token or b'' def receive(self, auth_bytes): @@ -74,6 +74,13 @@ def receive(self, auth_bytes): ] # add authorization identity to the response, and GSS-wrap self._next_token = self._client_ctx.wrap(b''.join(message_parts), False).message + # We need to identify the last token in auth_bytes(); + # we can't rely on client_ctx.complete because it becomes True after generating + # the second-to-last token (after calling .step(auth_bytes) for the final time) + # We could introduce an additional state variable (i.e., self._final_token), + # but instead we just set _is_authenticated. Since the plugin interface does + # not read is_authenticated() until after is_done() is True, this should be fine. + self._is_authenticated = True def is_done(self): return self._is_done diff --git a/test/sasl/test_gssapi.py b/test/sasl/test_gssapi.py new file mode 100644 index 000000000..893414e37 --- /dev/null +++ b/test/sasl/test_gssapi.py @@ -0,0 +1,42 @@ +from __future__ import absolute_import + +try: + from unittest import mock +except ImportError: + import mock + +from kafka.sasl import get_sasl_mechanism +import kafka.sasl.gssapi + + +def test_gssapi(): + config = { + 'sasl_kerberos_domain_name': 'foo', + 'sasl_kerberos_service_name': 'bar', + } + client_ctx = mock.Mock() + client_ctx.step.side_effect = [b'init', b'exchange', b'complete', b'xxxx'] + client_ctx.complete = False + def mocked_message_wrapper(msg, *args): + wrapped = mock.Mock() + type(wrapped).message = mock.PropertyMock(return_value=msg) + return wrapped + client_ctx.unwrap.side_effect = mocked_message_wrapper + client_ctx.wrap.side_effect = mocked_message_wrapper + kafka.sasl.gssapi.gssapi = mock.Mock() + kafka.sasl.gssapi.gssapi.SecurityContext.return_value = client_ctx + gssapi = get_sasl_mechanism('GSSAPI')(**config) + assert isinstance(gssapi, kafka.sasl.gssapi.SaslMechanismGSSAPI) + client_ctx.step.assert_called_with(None) + + while not gssapi.is_done(): + send_token = gssapi.auth_bytes() + receive_token = send_token # not realistic, but enough for testing + if send_token == b'\x00cbar@foo': # final wrapped message + receive_token = b'' # final message gets an empty response + gssapi.receive(receive_token) + if client_ctx.step.call_count == 3: + client_ctx.complete = True + + assert gssapi.is_done() + assert gssapi.is_authenticated() From e2b669307392a292ab5a9760a48aacc4c88d215d Mon Sep 17 00:00:00 2001 From: SeppMe <41479299+SeppMe@users.noreply.github.com> Date: Thu, 22 May 2025 22:56:52 +0200 Subject: [PATCH 190/205] Set the current host in the SASL configs (#2633) Co-authored-by: Sebastian.Meinhardt --- kafka/conn.py | 1 + test/test_conn.py | 10 ++++++++++ 2 files changed, 11 insertions(+) diff --git a/kafka/conn.py b/kafka/conn.py index c9cdd595f..b396b9778 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -256,6 +256,7 @@ def __init__(self, host, port, afi, **configs): for key in self.config: if key in configs: self.config[key] = configs[key] + self.config['host'] = host self.node_id = self.config.pop('node_id') diff --git a/test/test_conn.py b/test/test_conn.py index 037cd015e..8d56668c5 100644 --- a/test/test_conn.py +++ b/test/test_conn.py @@ -386,3 +386,13 @@ def test_maybe_throttle(conn): time.return_value = 3000 assert not conn.throttled() + + +def test_host_in_sasl_config(): + hostname = 'example.org' + port = 9092 + for security_protocol in ('SASL_PLAINTEXT', 'SASL_SSL'): + with mock.patch("kafka.conn.get_sasl_mechanism") as get_sasl_mechanism: + BrokerConnection(hostname, port, socket.AF_UNSPEC, security_protocol=security_protocol) + call_config = get_sasl_mechanism.mock_calls[1].kwargs + assert call_config['host'] == hostname From 7fceb1bd626eb781c1b7d3168e82c34525ef30b9 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 22 May 2025 13:57:49 -0700 Subject: [PATCH 191/205] Pass host as kwarg to get_sasl_mechanism --- kafka/conn.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/kafka/conn.py b/kafka/conn.py index b396b9778..64445fab0 100644 --- a/kafka/conn.py +++ b/kafka/conn.py @@ -256,7 +256,6 @@ def __init__(self, host, port, afi, **configs): for key in self.config: if key in configs: self.config[key] = configs[key] - self.config['host'] = host self.node_id = self.config.pop('node_id') @@ -314,7 +313,7 @@ def __init__(self, host, port, afi, **configs): 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'])(**self.config) + self._sasl_mechanism = get_sasl_mechanism(self.config['sasl_mechanism'])(host=self.host, **self.config) else: self._sasl_mechanism = None From 65834607d3ad2135772ec63696497500a9014c96 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 22 May 2025 14:00:29 -0700 Subject: [PATCH 192/205] Patch Release 2.2.10 --- CHANGES.md | 6 ++++++ docs/changelog.rst | 9 +++++++++ kafka/version.py | 2 +- 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index df08c71d7..b538396b5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,9 @@ +# 2.2.10 (May 22, 2025) + +Fixes +* Set the current host in the SASL configs (#2633) +* Fix sasl gssapi plugin: do not rely on `client_ctx.complete` in `auth_bytes()` (#2631) + # 2.2.9 (May 21, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index 6da6d849e..ff5910b88 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,15 @@ Changelog ========= +2.2.10 (May 22, 2025) +##################### + +Fixes +----- +* Set the current host in the SASL configs (#2633) +* Fix sasl gssapi plugin: do not rely on `client_ctx.complete` in `auth_bytes()` (#2631) + + 2.2.9 (May 21, 2025) #################### diff --git a/kafka/version.py b/kafka/version.py index 00c3889f6..dbd7bf93d 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.9' +__version__ = '2.2.10' From 5957c1ba23826523583a6669e1f8a86ec19af817 Mon Sep 17 00:00:00 2001 From: Raphael Boidol Date: Sat, 24 May 2025 17:08:18 +0200 Subject: [PATCH 193/205] Update build links in documentation (#2634) --- docs/index.rst | 4 ++-- docs/tests.rst | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/index.rst b/docs/index.rst index 823780929..0e9aa5f7b 100644 --- a/docs/index.rst +++ b/docs/index.rst @@ -7,8 +7,8 @@ kafka-python :target: https://pypi.python.org/pypi/kafka-python .. image:: https://coveralls.io/repos/dpkp/kafka-python/badge.svg?branch=master&service=github :target: https://coveralls.io/github/dpkp/kafka-python?branch=master -.. image:: https://travis-ci.org/dpkp/kafka-python.svg?branch=master - :target: https://travis-ci.org/dpkp/kafka-python +.. image:: https://img.shields.io/github/actions/workflow/status/dpkp/kafka-python/python-package.yml + :target: https://github.com/dpkp/kafka-python/actions/workflows/python-package.yml .. image:: https://img.shields.io/badge/license-Apache%202-blue.svg :target: https://github.com/dpkp/kafka-python/blob/master/LICENSE diff --git a/docs/tests.rst b/docs/tests.rst index c8adb2d76..79409887e 100644 --- a/docs/tests.rst +++ b/docs/tests.rst @@ -3,8 +3,8 @@ Tests .. image:: https://coveralls.io/repos/dpkp/kafka-python/badge.svg?branch=master&service=github :target: https://coveralls.io/github/dpkp/kafka-python?branch=master -.. image:: https://travis-ci.org/dpkp/kafka-python.svg?branch=master - :target: https://travis-ci.org/dpkp/kafka-python +.. image:: https://img.shields.io/github/actions/workflow/status/dpkp/kafka-python/python-package.yml + :target: https://github.com/dpkp/kafka-python/actions/workflows/python-package.yml The test suite is run via pytest. From e6abbbf284a1556536941fb8d99fb5ca03aa1e22 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sun, 1 Jun 2025 07:22:30 -0700 Subject: [PATCH 194/205] Add synchronized decorator; add lock to subscription state (#2636) --- kafka/consumer/subscription_state.py | 35 +++++++++++++++++++++++++++- kafka/util.py | 9 +++++++ 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/kafka/consumer/subscription_state.py b/kafka/consumer/subscription_state.py index cc3675b1d..f99f01615 100644 --- a/kafka/consumer/subscription_state.py +++ b/kafka/consumer/subscription_state.py @@ -15,6 +15,7 @@ import logging import random import re +import threading import time from kafka.vendor import six @@ -22,7 +23,7 @@ import kafka.errors as Errors from kafka.protocol.list_offsets import OffsetResetStrategy from kafka.structs import OffsetAndMetadata -from kafka.util import ensure_valid_topic_name +from kafka.util import ensure_valid_topic_name, synchronized log = logging.getLogger(__name__) @@ -84,6 +85,7 @@ def __init__(self, offset_reset_strategy='earliest'): self.assignment = OrderedDict() self.rebalance_listener = None self.listeners = [] + self._lock = threading.RLock() def _set_subscription_type(self, subscription_type): if not isinstance(subscription_type, SubscriptionType): @@ -93,6 +95,7 @@ def _set_subscription_type(self, subscription_type): elif self.subscription_type != subscription_type: raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) + @synchronized def subscribe(self, topics=(), pattern=None, listener=None): """Subscribe to a list of topics, or a topic regex pattern. @@ -147,6 +150,7 @@ def subscribe(self, topics=(), pattern=None, listener=None): raise TypeError('listener must be a ConsumerRebalanceListener') self.rebalance_listener = listener + @synchronized def change_subscription(self, topics): """Change the topic subscription. @@ -178,6 +182,7 @@ def change_subscription(self, topics): self.subscription = set(topics) self._group_subscription.update(topics) + @synchronized def group_subscribe(self, topics): """Add topics to the current group subscription. @@ -191,6 +196,7 @@ def group_subscribe(self, topics): raise Errors.IllegalStateError(self._SUBSCRIPTION_EXCEPTION_MESSAGE) self._group_subscription.update(topics) + @synchronized def reset_group_subscription(self): """Reset the group's subscription to only contain topics subscribed by this consumer.""" if not self.partitions_auto_assigned(): @@ -198,6 +204,7 @@ def reset_group_subscription(self): assert self.subscription is not None, 'Subscription required' self._group_subscription.intersection_update(self.subscription) + @synchronized def assign_from_user(self, partitions): """Manually assign a list of TopicPartitions to this consumer. @@ -222,6 +229,7 @@ def assign_from_user(self, partitions): self._set_assignment({partition: self.assignment.get(partition, TopicPartitionState()) for partition in partitions}) + @synchronized def assign_from_subscribed(self, assignments): """Update the assignment to the specified partitions @@ -258,6 +266,7 @@ def _set_assignment(self, partition_states, randomize=False): for tp in topic_partitions[topic]: self.assignment[tp] = partition_states[tp] + @synchronized def unsubscribe(self): """Clear all topic subscriptions and partition assignments""" self.subscription = None @@ -266,6 +275,7 @@ def unsubscribe(self): self.subscribed_pattern = None self.subscription_type = SubscriptionType.NONE + @synchronized def group_subscription(self): """Get the topic subscription for the group. @@ -281,6 +291,7 @@ def group_subscription(self): """ return self._group_subscription + @synchronized def seek(self, partition, offset): """Manually specify the fetch offset for a TopicPartition. @@ -298,15 +309,18 @@ def seek(self, partition, offset): raise TypeError("offset must be type in or OffsetAndMetadata") self.assignment[partition].seek(offset) + @synchronized def assigned_partitions(self): """Return set of TopicPartitions in current assignment.""" return set(self.assignment.keys()) + @synchronized def paused_partitions(self): """Return current set of paused TopicPartitions.""" return set(partition for partition in self.assignment if self.is_paused(partition)) + @synchronized def fetchable_partitions(self): """Return ordered list of TopicPartitions that should be Fetched.""" fetchable = list() @@ -315,10 +329,12 @@ def fetchable_partitions(self): fetchable.append(partition) return fetchable + @synchronized def partitions_auto_assigned(self): """Return True unless user supplied partitions manually.""" return self.subscription_type in (SubscriptionType.AUTO_TOPICS, SubscriptionType.AUTO_PATTERN) + @synchronized def all_consumed_offsets(self): """Returns consumed offsets as {TopicPartition: OffsetAndMetadata}""" all_consumed = {} @@ -327,6 +343,7 @@ def all_consumed_offsets(self): all_consumed[partition] = state.position return all_consumed + @synchronized def request_offset_reset(self, partition, offset_reset_strategy=None): """Mark partition for offset reset using specified or default strategy. @@ -338,23 +355,28 @@ def request_offset_reset(self, partition, offset_reset_strategy=None): offset_reset_strategy = self._default_offset_reset_strategy self.assignment[partition].reset(offset_reset_strategy) + @synchronized def set_reset_pending(self, partitions, next_allowed_reset_time): for partition in partitions: self.assignment[partition].set_reset_pending(next_allowed_reset_time) + @synchronized def has_default_offset_reset_policy(self): """Return True if default offset reset policy is Earliest or Latest""" return self._default_offset_reset_strategy != OffsetResetStrategy.NONE + @synchronized def is_offset_reset_needed(self, partition): return self.assignment[partition].awaiting_reset + @synchronized def has_all_fetch_positions(self): for state in six.itervalues(self.assignment): if not state.has_valid_position: return False return True + @synchronized def missing_fetch_positions(self): missing = set() for partition, state in six.iteritems(self.assignment): @@ -362,9 +384,11 @@ def missing_fetch_positions(self): missing.add(partition) return missing + @synchronized def has_valid_position(self, partition): return partition in self.assignment and self.assignment[partition].has_valid_position + @synchronized def reset_missing_positions(self): partitions_with_no_offsets = set() for tp, state in six.iteritems(self.assignment): @@ -377,6 +401,7 @@ def reset_missing_positions(self): if partitions_with_no_offsets: raise Errors.NoOffsetForPartitionError(partitions_with_no_offsets) + @synchronized def partitions_needing_reset(self): partitions = set() for tp, state in six.iteritems(self.assignment): @@ -384,25 +409,32 @@ def partitions_needing_reset(self): partitions.add(tp) return partitions + @synchronized def is_assigned(self, partition): return partition in self.assignment + @synchronized def is_paused(self, partition): return partition in self.assignment and self.assignment[partition].paused + @synchronized def is_fetchable(self, partition): return partition in self.assignment and self.assignment[partition].is_fetchable() + @synchronized def pause(self, partition): self.assignment[partition].pause() + @synchronized def resume(self, partition): self.assignment[partition].resume() + @synchronized def reset_failed(self, partitions, next_retry_time): for partition in partitions: self.assignment[partition].reset_failed(next_retry_time) + @synchronized def move_partition_to_end(self, partition): if partition in self.assignment: try: @@ -411,6 +443,7 @@ def move_partition_to_end(self, partition): state = self.assignment.pop(partition) self.assignment[partition] = state + @synchronized def position(self, partition): return self.assignment[partition].position diff --git a/kafka/util.py b/kafka/util.py index bfb9365ad..658c17d59 100644 --- a/kafka/util.py +++ b/kafka/util.py @@ -1,6 +1,7 @@ from __future__ import absolute_import, division import binascii +import functools import re import time import weakref @@ -129,3 +130,11 @@ class Dict(dict): See: https://docs.python.org/2/library/weakref.html """ pass + + +def synchronized(func): + def wrapper(self, *args, **kwargs): + with self._lock: + return func(self, *args, **kwargs) + functools.update_wrapper(wrapper, func) + return wrapper From 9f7384c2bf32f07e4ae1039fd87024fdd966c124 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 4 Jun 2025 15:17:42 -0700 Subject: [PATCH 195/205] Fix decoding bug in AWS_MSK_IAM mechanism (#2639) --- kafka/sasl/msk.py | 17 ++++++++++++++--- test/sasl/test_msk.py | 16 +++++++++++++++- 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/kafka/sasl/msk.py b/kafka/sasl/msk.py index db56b4801..7ec03215d 100644 --- a/kafka/sasl/msk.py +++ b/kafka/sasl/msk.py @@ -4,6 +4,7 @@ import hashlib import hmac import json +import logging import string # needed for AWS_MSK_IAM authentication: @@ -13,10 +14,14 @@ # no botocore available, will disable AWS_MSK_IAM mechanism BotoSession = None +from kafka.errors import KafkaConfigurationError from kafka.sasl.abc import SaslMechanism from kafka.vendor.six.moves import urllib +log = logging.getLogger(__name__) + + class SaslMechanismAwsMskIam(SaslMechanism): def __init__(self, **config): assert BotoSession is not None, 'AWS_MSK_IAM requires the "botocore" package' @@ -27,22 +32,28 @@ def __init__(self, **config): self._is_done = False self._is_authenticated = False - def auth_bytes(self): + def _build_client(self): session = BotoSession() credentials = session.get_credentials().get_frozen_credentials() - client = AwsMskIamClient( + if not session.get_config_variable('region'): + raise KafkaConfigurationError('Unable to determine region for AWS MSK cluster. Is AWS_DEFAULT_REGION set?') + return AwsMskIamClient( host=self.host, access_key=credentials.access_key, secret_key=credentials.secret_key, region=session.get_config_variable('region'), token=credentials.token, ) + + def auth_bytes(self): + client = self._build_client() + log.debug("Generating auth token for MSK scope: %s", client._scope) return client.first_message() def receive(self, auth_bytes): self._is_done = True self._is_authenticated = auth_bytes != b'' - self._auth = auth_bytes.deode('utf-8') + self._auth = auth_bytes.decode('utf-8') def is_done(self): return self._is_done diff --git a/test/sasl/test_msk.py b/test/sasl/test_msk.py index e9f1325f3..f3cc46ce8 100644 --- a/test/sasl/test_msk.py +++ b/test/sasl/test_msk.py @@ -2,7 +2,7 @@ import json import sys -from kafka.sasl.msk import AwsMskIamClient +from kafka.sasl.msk import AwsMskIamClient, SaslMechanismAwsMskIam try: from unittest import mock @@ -69,3 +69,17 @@ def test_aws_msk_iam_client_temporary_credentials(): 'x-amz-security-token': 'XXXXX', } assert actual == expected + + +def test_aws_msk_iam_sasl_mechanism(): + with mock.patch('kafka.sasl.msk.BotoSession'): + sasl = SaslMechanismAwsMskIam(security_protocol='SASL_SSL', host='localhost') + with mock.patch.object(sasl, '_build_client', return_value=client_factory(token=None)): + assert sasl.auth_bytes() != b'' + assert not sasl.is_done() + assert not sasl.is_authenticated() + sasl.receive(b'foo') + assert sasl._auth == 'foo' + assert sasl.is_done() + assert sasl.is_authenticated() + assert sasl.auth_details() From e93ab64c6d7d16d758465271d8bb3184a9aa57b1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 5 Jun 2025 09:34:36 -0700 Subject: [PATCH 196/205] Do not ignore metadata response for single topic with error (#2640) --- kafka/cluster.py | 17 +++++++++-------- kafka/producer/kafka.py | 4 +++- test/test_cluster.py | 15 +++++++++++++++ 3 files changed, 27 insertions(+), 9 deletions(-) diff --git a/kafka/cluster.py b/kafka/cluster.py index d6ec82dba..ded8c6f96 100644 --- a/kafka/cluster.py +++ b/kafka/cluster.py @@ -245,13 +245,6 @@ def update_metadata(self, metadata): Returns: None """ - # In the common case where we ask for a single topic and get back an - # error, we should fail the future - if len(metadata.topics) == 1 and metadata.topics[0][0] != Errors.NoError.errno: - error_code, topic = metadata.topics[0][:2] - error = Errors.for_code(error_code)(topic) - return self.failed_update(error) - if not metadata.brokers: log.warning("No broker metadata found in MetadataResponse -- ignoring.") return self.failed_update(Errors.MetadataEmptyBrokerList(metadata)) @@ -349,7 +342,15 @@ def update_metadata(self, metadata): self._last_successful_refresh_ms = now if f: - f.success(self) + # In the common case where we ask for a single topic and get back an + # error, we should fail the future + if len(metadata.topics) == 1 and metadata.topics[0][0] != Errors.NoError.errno: + error_code, topic = metadata.topics[0][:2] + error = Errors.for_code(error_code)(topic) + f.failure(error) + else: + f.success(self) + log.debug("Updated cluster metadata to %s", self) for listener in self._listeners: diff --git a/kafka/producer/kafka.py b/kafka/producer/kafka.py index 2416cc3ef..9401bd814 100644 --- a/kafka/producer/kafka.py +++ b/kafka/producer/kafka.py @@ -960,9 +960,11 @@ def _wait_on_metadata(self, topic, max_wait_ms): future.add_both(lambda e, *args: e.set(), metadata_event) self._sender.wakeup() metadata_event.wait(timer.timeout_ms / 1000) - if not metadata_event.is_set(): + if not future.is_done: raise Errors.KafkaTimeoutError( "Failed to update metadata after %.1f secs." % (max_wait_ms / 1000,)) + elif future.failed() and not future.retriable(): + raise future.exception elif topic in self._metadata.unauthorized_topics: raise Errors.TopicAuthorizationFailedError(set([topic])) else: diff --git a/test/test_cluster.py b/test/test_cluster.py index c57bd8f9f..730b27cb6 100644 --- a/test/test_cluster.py +++ b/test/test_cluster.py @@ -136,6 +136,21 @@ def test_metadata_v7(): assert cluster._partitions['topic-1'][0].leader_epoch == 0 +def test_unauthorized_topic(): + cluster = ClusterMetadata() + assert len(cluster.brokers()) == 0 + + cluster.update_metadata(MetadataResponse[0]( + [(0, 'foo', 12), (1, 'bar', 34)], + [(29, 'unauthorized-topic', [])])) # single topic w/ unauthorized error + + # broker metadata should get updated + assert len(cluster.brokers()) == 2 + + # topic should be added to unauthorized list + assert 'unauthorized-topic' in cluster.unauthorized_topics + + def test_collect_hosts__happy_path(): hosts = "127.0.0.1:1234,127.0.0.1" results = collect_hosts(hosts) From 3542e01aac4eefffb48b0de9da3f064f7c57be0e Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 5 Jun 2025 14:23:47 -0700 Subject: [PATCH 197/205] Patch Release 2.2.11 --- CHANGES.md | 8 ++++++++ docs/changelog.rst | 11 +++++++++++ kafka/version.py | 2 +- 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index b538396b5..75f07fbc8 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,11 @@ +# 2.2.11 (June 5, 2025) + +Fixes +* Do not ignore metadata response for single topic with error (#2640) +* Fix decoding bug in AWS_MSK_IAM mechanism (#2639) +* Add synchronized decorator; add lock to subscription state (#2636) +* Update build links in documentation (#2634) + # 2.2.10 (May 22, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index ff5910b88..ddc43ff9a 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,17 @@ Changelog ========= +2.2.11 (June 5, 2025) +##################### + +Fixes +----- +* Do not ignore metadata response for single topic with error (#2640) +* Fix decoding bug in AWS_MSK_IAM mechanism (#2639) +* Add synchronized decorator; add lock to subscription state (#2636) +* Update build links in documentation (#2634) + + 2.2.10 (May 22, 2025) ##################### diff --git a/kafka/version.py b/kafka/version.py index dbd7bf93d..0086e638d 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.10' +__version__ = '2.2.11' From 5c7cd2f21742949f89f2a5191adc4dc1aaa79d43 Mon Sep 17 00:00:00 2001 From: Emmanuel Ferdman Date: Tue, 10 Jun 2025 19:56:22 +0300 Subject: [PATCH 198/205] Throw exception on invalid bucket type (#2642) Signed-off-by: Emmanuel Ferdman --- kafka/metrics/stats/percentiles.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka/metrics/stats/percentiles.py b/kafka/metrics/stats/percentiles.py index c36543ffa..2cb2d84de 100644 --- a/kafka/metrics/stats/percentiles.py +++ b/kafka/metrics/stats/percentiles.py @@ -30,7 +30,7 @@ def __init__(self, size_in_bytes, bucketing, max_val, min_val=0.0, ' to be 0.0.') self.bin_scheme = Histogram.LinearBinScheme(self._buckets, max_val) else: - ValueError('Unknown bucket type: %s' % (bucketing,)) + raise ValueError('Unknown bucket type: %s' % (bucketing,)) def stats(self): measurables = [] From 5d83941c1faa34ffba04455dde169c108d0d4201 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 18 Jun 2025 12:34:24 -0700 Subject: [PATCH 199/205] Avoid RuntimeError on mutated _completed_fetches deque in consumer fetcher (#2646) --- 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 b083deb1a..1888d38bf 100644 --- a/kafka/consumer/fetcher.py +++ b/kafka/consumer/fetcher.py @@ -612,7 +612,8 @@ def _handle_list_offsets_response(self, future, response): def _fetchable_partitions(self): fetchable = self._subscriptions.fetchable_partitions() # do not fetch a partition if we have a pending fetch response to process - discard = {fetch.topic_partition for fetch in self._completed_fetches} + # use copy.copy to avoid runtimeerror on mutation from different thread + discard = {fetch.topic_partition for fetch in self._completed_fetches.copy()} current = self._next_partition_records if current: discard.add(current.topic_partition) From 02453523a516d63655617b9e24a764d553de791f Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 18 Jun 2025 16:01:48 -0700 Subject: [PATCH 200/205] Fix construction of final GSSAPI authentication message (#2647) --- kafka/sasl/gssapi.py | 6 ++++-- test/sasl/test_gssapi.py | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/kafka/sasl/gssapi.py b/kafka/sasl/gssapi.py index 6a4896585..c8e4f7cac 100644 --- a/kafka/sasl/gssapi.py +++ b/kafka/sasl/gssapi.py @@ -1,5 +1,7 @@ from __future__ import absolute_import +import struct + # needed for SASL_GSSAPI authentication: try: import gssapi @@ -68,8 +70,8 @@ def receive(self, auth_bytes): client_flags = self.SASL_QOP_AUTH server_flags = msg[0] message_parts = [ - bytes(client_flags & server_flags), - msg[:1], + struct.Struct('>b').pack(client_flags & server_flags), + msg[1:], self.auth_id.encode('utf-8'), ] # add authorization identity to the response, and GSS-wrap diff --git a/test/sasl/test_gssapi.py b/test/sasl/test_gssapi.py index 893414e37..aa1d86b9e 100644 --- a/test/sasl/test_gssapi.py +++ b/test/sasl/test_gssapi.py @@ -32,7 +32,7 @@ def mocked_message_wrapper(msg, *args): while not gssapi.is_done(): send_token = gssapi.auth_bytes() receive_token = send_token # not realistic, but enough for testing - if send_token == b'\x00cbar@foo': # final wrapped message + if send_token == b'\x01ompletebar@foo': # final wrapped message receive_token = b'' # final message gets an empty response gssapi.receive(receive_token) if client_ctx.step.call_count == 3: From e22e41e2e01d08115ee643efef3e3a66a021c5c7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 18 Jun 2025 16:03:41 -0700 Subject: [PATCH 201/205] Patch Release 2.2.12 --- CHANGES.md | 8 ++++++++ docs/changelog.rst | 10 ++++++++++ kafka/version.py | 2 +- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 75f07fbc8..1c3411051 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,11 @@ +# 2.2.12 (June 18, 2025) + +Fixes +* Fix construction of final GSSAPI authentication message (#2647) +* 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 ddc43ff9a..fcfe59540 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,16 @@ Changelog ========= +2.2.12 (June 18, 2025) +###################### + +Fixes +----- +* Fix construction of final GSSAPI authentication message (#2647) +* Avoid RuntimeError on mutated `_completed_fetches` deque in consumer fetcher (#2646) +* Throw exception on invalid bucket type (#2642) + + 2.2.11 (June 5, 2025) ##################### diff --git a/kafka/version.py b/kafka/version.py index 0086e638d..985a91bdc 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.11' +__version__ = '2.2.12' From e4e6fcf353184af36226397d365cce1ee88b4a3a Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Wed, 18 Jun 2025 16:08:36 -0700 Subject: [PATCH 202/205] update README kafka version badge to include 4.0 --- README.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.rst b/README.rst index b820c34eb..b11868241 100644 --- a/README.rst +++ b/README.rst @@ -1,7 +1,7 @@ Kafka Python client ------------------------ -.. image:: https://img.shields.io/badge/kafka-3.9--0.8-brightgreen.svg +.. image:: https://img.shields.io/badge/kafka-4.0--0.8-brightgreen.svg :target: https://kafka-python.readthedocs.io/en/master/compatibility.html .. image:: https://img.shields.io/pypi/pyversions/kafka-python.svg :target: https://pypi.python.org/pypi/kafka-python From cf1cd1043d91e3f7d5c00267e8dc7d79ff52f145 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 19 Jun 2025 09:20:03 -0700 Subject: [PATCH 203/205] Use client.await_ready() to simplify blocking wait and add timeout to admin client (#2648) --- kafka/admin/client.py | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/kafka/admin/client.py b/kafka/admin/client.py index 82aaa68e9..8490fdb46 100644 --- a/kafka/admin/client.py +++ b/kafka/admin/client.py @@ -17,6 +17,7 @@ from kafka.errors import ( IncompatibleBrokerVersion, KafkaConfigurationError, UnknownTopicOrPartitionError, UnrecognizedBrokerVersion, IllegalArgumentError) +from kafka.future import Future from kafka.metrics import MetricConfig, Metrics from kafka.protocol.admin import ( CreateTopicsRequest, DeleteTopicsRequest, DescribeConfigsRequest, AlterConfigsRequest, CreatePartitionsRequest, @@ -358,14 +359,11 @@ def _send_request_to_node(self, node_id, request, wakeup=True): Returns: A future object that may be polled for status and results. - - Raises: - The exception if the message could not be sent. """ - while not self._client.ready(node_id): - # poll until the connection to broker is ready, otherwise send() - # will fail with NodeNotReadyError - self._client.poll(timeout_ms=200) + try: + self._client.await_ready(node_id) + except Errors.KafkaConnectionError as e: + return Future().failure(e) return self._client.send(node_id, request, wakeup) def _send_request_to_controller(self, request): From 0735a7c0d3dfad3bb441069ffd6ff4d4146d6fcf Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Thu, 19 Jun 2025 09:20:57 -0700 Subject: [PATCH 204/205] Fixup import style in example.py --- example.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/example.py b/example.py index 9907450f6..ac89296b6 100755 --- a/example.py +++ b/example.py @@ -1,5 +1,6 @@ #!/usr/bin/env python -import threading, time +import threading +import time from kafka import KafkaAdminClient, KafkaConsumer, KafkaProducer from kafka.admin import NewTopic From f6eb0b401a3941899c04c61f195e5fa4644cb6c7 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Fri, 20 Jun 2025 15:47:12 -0700 Subject: [PATCH 205/205] Patch Release 2.2.13 --- CHANGES.md | 9 +++++++++ docs/changelog.rst | 13 +++++++++++++ kafka/version.py | 2 +- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 1c3411051..372aebfc6 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,12 @@ +# 2.2.13 (June 20, 2025) + +Fixes +* Use client.await_ready() to simplify blocking wait and add timeout to admin client (#2648) +* Fixup import style in example.py + +Documentation +* update README kafka version badge to include 4.0 + # 2.2.12 (June 18, 2025) Fixes diff --git a/docs/changelog.rst b/docs/changelog.rst index fcfe59540..430f8a512 100644 --- a/docs/changelog.rst +++ b/docs/changelog.rst @@ -1,6 +1,19 @@ Changelog ========= +2.2.13 (June 20, 2025) +###################### + +Fixes +----- +* Use client.await_ready() to simplify blocking wait and add timeout to admin client (#2648) +* Fixup import style in example.py + +Documentation +------------- +* update README kafka version badge to include 4.0 + + 2.2.12 (June 18, 2025) ###################### diff --git a/kafka/version.py b/kafka/version.py index 985a91bdc..298979870 100644 --- a/kafka/version.py +++ b/kafka/version.py @@ -1 +1 @@ -__version__ = '2.2.12' +__version__ = '2.2.13'