KafkaClient

class kafka.KafkaClient(**configs)[source]

A network client for asynchronous request/response network I/O.

This is an internal class used to implement the user-facing producer and consumer clients.

This class is not thread-safe!

Attributes:
cluster (ClusterMetadata): Local cache of cluster metadata, retrieved

via MetadataRequests during poll().

Keyword Arguments:
bootstrap_servers: ‘host[:port]’ string (or list of ‘host[:port]’

strings) that the client should contact to bootstrap initial cluster metadata. This does not have to be the full node list. It just needs to have at least one broker that will respond to a Metadata API Request. Default port is 9092. If no servers are specified, will default to localhost:9092.

client_id (str): a name for this client. This string is passed in

each request to servers and can be used to identify specific 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}’

reconnect_backoff_ms (int): The amount of time in milliseconds to

wait before attempting to reconnect to a given host. Default: 50.

reconnect_backoff_max_ms (int): The maximum amount of time in

milliseconds to backoff/wait when reconnecting to a broker that has repeatedly failed to connect. If provided, the backoff per host will increase exponentially for each consecutive connection failure, up to this maximum. Once the maximum is reached, reconnection attempts will continue periodically with this fixed rate. To avoid connection storms, a randomization factor of 0.2 will be applied to the backoff resulting in a random range between 20% below and 20% above the computed value. Default: 1000.

request_timeout_ms (int): Client request timeout in milliseconds.

Default: 30000.

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 unexpected socket disconnected errors on the client. Default: 540000

retry_backoff_ms (int): Milliseconds to backoff when retrying on

errors. Default: 100.

max_in_flight_requests_per_connection (int): Requests are pipelined

to kafka brokers up to this number of maximum requests per broker connection. Default: 5.

receive_buffer_bytes (int): The size of the TCP receive buffer

(SO_RCVBUF) to use when reading data. Default: None (relies on system defaults). Java client defaults to 32768.

send_buffer_bytes (int): The size of the TCP send buffer

(SO_SNDBUF) to use when sending data. Default: None (relies on system defaults). Java client defaults to 131072.

socket_options (list): List of tuple-arguments to socket.setsockopt

to apply to broker connection sockets. Default: [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)]

metadata_max_age_ms (int): The period of time in milliseconds after

which we force a refresh of metadata even if we haven’t seen any partition leadership changes to proactively discover any new brokers or partitions. Default: 300000

security_protocol (str): Protocol used to communicate with brokers.

Valid values are: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL. Default: PLAINTEXT.

ssl_context (ssl.SSLContext): Pre-configured SSLContext for wrapping

socket connections. If provided, all other ssl_* configurations will be ignored. Default: None.

ssl_check_hostname (bool): Flag to configure whether SSL handshake

should verify that the certificate matches the broker’s hostname. Default: True.

ssl_cafile (str): Optional filename of CA file to use in certificate

verification. Default: None.

ssl_certfile (str): Optional filename of file in PEM format containing

the client certificate, as well as any CA certificates needed to establish the certificate’s authenticity. Default: None.

ssl_keyfile (str): Optional filename containing the client private key.

Default: None.

ssl_password (str): Optional password to be used when loading the

certificate chain. Default: None.

ssl_crlfile (str): Optional filename containing the CRL to check for

certificate expiration. By default, no CRL check is done. When providing a file, only the leaf certificate will be checked against this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+. Default: None.

ssl_ciphers (str): optionally set the available ciphers for ssl

connections. It should be a string in the OpenSSL cipher list format. If no cipher can be selected (because compile-time options or other configuration forbids use of all the specified ciphers), an ssl.SSLError will be raised. See ssl.SSLContext.set_ciphers

api_version (tuple): Specify which Kafka API version to use. If set

to None, KafkaClient will attempt to infer the broker version by probing various APIs. Example: (0, 10, 2). Default: None

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

selector (selectors.BaseSelector): Provide a specific selector

implementation to use for I/O multiplexing. Default: selectors.DefaultSelector

metrics (kafka.metrics.Metrics): Optionally provide a metrics

instance for capturing network IO stats. Default: None.

metric_group_prefix (str): Prefix for metric names. Default: ‘’ sasl_mechanism (str): Authentication mechanism when security_protocol

is configured for SASL_PLAINTEXT or SASL_SSL. Valid values are: PLAIN, GSSAPI, OAUTHBEARER, SCRAM-SHA-256, SCRAM-SHA-512.

sasl_plain_username (str): username for sasl PLAIN and SCRAM authentication.

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_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

sasl mechanism handshake. Default: one of bootstrap servers

sasl_oauth_token_provider (AbstractTokenProvider): OAuthBearer token provider

instance. (See kafka.oauth.abstract). Default: None

add_topic(topic)[source]

Add a topic to the list of topics tracked via metadata.

Arguments:

topic (str): topic to track

Returns:

Future: resolves after metadata request/response

bootstrap_connected()[source]

Return True if a bootstrap node is connected

check_version(node_id=None, timeout=2, strict=False)[source]

Attempt to guess the version of a Kafka broker.

Note: It is possible that this method blocks longer than the

specified timeout. This can happen if the entire cluster is down and the client enters a bootstrap backoff sleep. This is only possible if node_id is None.

Returns: version tuple, i.e. (0, 10), (0, 9), (0, 8, 2), …

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!

close(node_id=None)[source]

Close one or all broker connections.

Arguments:

node_id (int, optional): the id of the node to close

connected(node_id)[source]

Return True iff the node_id is connected.

connection_delay(node_id)[source]

Return the number of milliseconds to wait, based on the connection state, before attempting to send data. When disconnected, this respects the reconnect backoff time. When connecting, returns 0 to allow non-blocking connect to finish. When connected, returns a very large number to handle slow/stalled connections.

Arguments:

node_id (int): The id of the node to check

Returns:

int: The number of milliseconds to wait.

get_api_versions()[source]

Return the ApiVersions map, if available.

Note: A call to check_version must previously have succeeded and returned 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.

in_flight_request_count(node_id=None)[source]

Get the number of in-flight requests for a node or all nodes.

Arguments:
node_id (int, optional): a specific node to check. If unspecified,

return the total for all nodes

Returns:

int: pending in-flight requests for the node, or all nodes if None

is_disconnected(node_id)[source]

Check whether the node connection has been disconnected or failed.

A disconnected node has either been closed or has failed. Connection failures are usually transient and can be resumed in the next ready() call, but there are cases where transient failures need to be caught and re-acted upon.

Arguments:

node_id (int): the id of the node to check

Returns:

bool: True iff the node exists and is disconnected

is_ready(node_id, metadata_priority=True)[source]

Check whether a node is ready to send more requests.

In addition to connection-level checks, this method also is used to block additional requests from being sent during a metadata refresh.

Arguments:

node_id (int): id of the node to check metadata_priority (bool): Mark node as not-ready if a metadata

refresh is required. Default: True

Returns:

bool: True if the node is ready and metadata is not refreshing

least_loaded_node()[source]

Choose the node with fewest outstanding requests, with fallbacks.

This method will prefer a node with an existing connection and no in-flight-requests. If no such node is found, a node will be chosen randomly from disconnected nodes that are not “blacked out” (i.e., are not subject to a reconnect backoff). If no node metadata has been obtained, will return a bootstrap node (subject to exponential backoff).

Returns:

node_id or None if no suitable node was found

maybe_connect(node_id, wakeup=True)[source]

Queues a node for asynchronous connection during the next .poll()

poll(timeout_ms=None, future=None)[source]

Try to read and write to sockets.

This method will also attempt to complete node connections, refresh stale metadata, and run previously-scheduled tasks.

Arguments:
timeout_ms (int, optional): maximum amount of time to wait (in ms)

for at least one response. Must be non-negative. The actual timeout will be the minimum of timeout, request timeout and metadata timeout. Default: request_timeout_ms

future (Future, optional): if provided, blocks until future.is_done

Returns:

list: responses received (can be empty)

ready(node_id, metadata_priority=True)[source]

Check whether a node is connected and ok to send more requests.

Arguments:

node_id (int): the id of the node to check metadata_priority (bool): Mark node as not-ready if a metadata

refresh is required. Default: True

Returns:

bool: True if we are ready to send to the given node

send(node_id, request, wakeup=True)[source]

Send a request to a specific node. Bytes are placed on an internal per-connection send-queue. Actual network I/O will be triggered in a subsequent call to .poll()

Arguments:

node_id (int): destination node request (Struct): request object (not-encoded) wakeup (bool): optional flag to disable thread-wakeup

Raises:

AssertionError: if node_id is not in current cluster metadata

Returns:

Future: resolves to Response struct or Error

set_topics(topics)[source]

Set specific topics to track for metadata.

Arguments:

topics (list of str): topics to check for metadata

Returns:

Future: resolves after metadata request/response