UNPKG

@confluentinc/kafka-javascript

Version:
1,164 lines (892 loc) 131 kB
<a name="introduction-to-librdkafka---the-apache-kafka-cc-client-library"></a> # Introduction to librdkafka - the Apache Kafka C/C++ client library librdkafka is a high performance C implementation of the Apache Kafka client, providing a reliable and performant client for production use. librdkafka also provides a native C++ interface. <!-- markdown-toc start - Don't edit this section. Run M-x markdown-toc-refresh-toc --> **Table of Contents** - [Introduction to librdkafka - the Apache Kafka C/C++ client library](#introduction-to-librdkafka---the-apache-kafka-cc-client-library) - [Performance](#performance) - [High throughput](#high-throughput) - [Low latency](#low-latency) - [Latency measurement](#latency-measurement) - [Compression](#compression) - [Message reliability](#message-reliability) - [Producer message delivery success](#producer-message-delivery-success) - [Producer message delivery failure](#producer-message-delivery-failure) - [Error: Timed out in transmission queue](#error-timed-out-in-transmission-queue) - [Error: Timed out in flight to/from broker](#error-timed-out-in-flight-tofrom-broker) - [Error: Temporary broker-side error](#error-temporary-broker-side-error) - [Error: Temporary errors due to stale metadata](#error-temporary-errors-due-to-stale-metadata) - [Error: Local time out](#error-local-time-out) - [Error: Permanent errors](#error-permanent-errors) - [Producer retries](#producer-retries) - [Reordering](#reordering) - [Idempotent Producer](#idempotent-producer) - [Guarantees](#guarantees) - [Ordering and message sequence numbers](#ordering-and-message-sequence-numbers) - [Partitioner considerations](#partitioner-considerations) - [Message timeout considerations](#message-timeout-considerations) - [Leader change](#leader-change) - [Error handling](#error-handling) - <a href="#rd-kafka-resp-err-out-of-order-sequence-number">RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER</a> - <a href="#rd-kafka-resp-err-duplicate-sequence-number">RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER</a> - <a href="#rd-kafka-resp-err-unknown-producer-id">RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID</a> - [Standard errors](#standard-errors) - [Message persistence status](#message-persistence-status) - [Transactional Producer](#transactional-producer) - [Error handling](#error-handling-1) - [Old producer fencing](#old-producer-fencing) - [Configuration considerations](#configuration-considerations) - [Exactly Once Semantics (EOS) and transactions](#exactly-once-semantics-eos-and-transactions) - [Usage](#usage) - [Documentation](#documentation) - [Initialization](#initialization) - [Configuration](#configuration) - [Example](#example) - [Termination](#termination) - [High-level KafkaConsumer](#high-level-kafkaconsumer) - [Producer](#producer) - [Admin API client](#admin-api-client) - [Speeding up termination](#speeding-up-termination) - [Threads and callbacks](#threads-and-callbacks) - [Brokers](#brokers) - [SSL](#ssl) - [OAUTHBEARER with support for OIDC](#oauthbearer-with-support-for-oidc) - [JWT bearer grant type (KIP-1139)](#jwt-bearer-grant-type-kip-1139) - [Metadata based authentication](#metadata-based-authentication) - [Azure IMDS](#azure-imds) - [Sparse connections](#sparse-connections) - [Random broker selection](#random-broker-selection) - [Persistent broker connections](#persistent-broker-connections) - [Connection close](#connection-close) - [Fetch From Follower](#fetch-from-follower) - [Logging](#logging) - [Debug contexts](#debug-contexts) - [Feature discovery](#feature-discovery) - [Producer API](#producer-api) - [Simple Consumer API (legacy)](#simple-consumer-api-legacy) - [Offset management](#offset-management) - [Auto offset commit](#auto-offset-commit) - [At-least-once processing](#at-least-once-processing) - [Auto offset reset](#auto-offset-reset) - [Consumer groups](#consumer-groups) - [Static consumer groups](#static-consumer-groups) - [Next Generation Consumer Group Protocol (KIP-848)](#next-generation-consumer-group-protocol-kip-848) - [Overview](#overview) - [Available Features](#available-features) - [Contract Changes](#contract-changes) - [Client Configuration changes](#client-configuration-changes) - [Rebalance Callback Changes](#rebalance-callback-changes) - [Static Group Membership](#static-group-membership) - [Session Timeout \& Fetching](#session-timeout--fetching) - [Closing / Auto-Commit](#closing--auto-commit) - [Error Handling Changes](#error-handling-changes) - [Summary of Key Differences (Classic vs Next-Gen)](#summary-of-key-differences-classic-vs-next-gen) - [Minimal Example Config](#minimal-example-config) - [Classic Protocol](#classic-protocol) - [Next-Gen Protocol / KIP-848](#next-gen-protocol--kip-848) - [Rebalance Callback Migration](#rebalance-callback-migration) - [Range Assignor (Classic)](#range-assignor-classic) - [Incremental Assignor (Including Range in Consumer / KIP-848, Any Protocol)](#incremental-assignor-including-range-in-consumer--kip-848-any-protocol) - [Upgrade and Downgrade](#upgrade-and-downgrade) - [Migration Checklist (Next-Gen Protocol / KIP-848)](#migration-checklist-next-gen-protocol--kip-848) - [Note on Batch consume APIs](#note-on-batch-consume-apis) - [Topics](#topics) - [Unknown or unauthorized topics](#unknown-or-unauthorized-topics) - [Topic metadata propagation for newly created topics](#topic-metadata-propagation-for-newly-created-topics) - [Topic auto creation](#topic-auto-creation) - [Metadata](#metadata) - [\< 0.9.3](#lt093) - [\> 0.9.3](#gt093-1) - [Query reasons](#query-reasons) - [Caching strategy](#caching-strategy) - [Fatal errors](#fatal-errors) - [Fatal producer errors](#fatal-producer-errors) - [Fatal consumer errors](#fatal-consumer-errors) - [Compatibility](#compatibility) - [Broker version compatibility](#broker-version-compatibility) - [Broker version \>= 0.10.0.0 (or trunk)](#broker-version--01000-or-trunk) - [Broker versions 0.9.0.x](#broker-versions-090x) - [Broker versions 0.8.x.y](#broker-versions-08xy) - [Detailed description](#detailed-description) - [Supported KIPs](#supported-kips) - [Supported protocol versions](#supported-protocol-versions) - [Recommendations for language binding developers](#recommendations-for-language-binding-developers) - [Expose the configuration interface pass-thru](#expose-the-configuration-interface-pass-thru) - [Error constants](#error-constants) - [Reporting client software name and version to broker](#reporting-client-software-name-and-version-to-broker) - [Documentation reuse](#documentation-reuse) - [Community support](#community-support) <!-- markdown-toc end --> <a name="performance"></a> ## Performance librdkafka is a multi-threaded library designed for use on modern hardware and it attempts to keep memory copying to a minimum. The payload of produced or consumed messages may pass through without any copying (if so desired by the application) putting no limit on message sizes. librdkafka allows you to decide if high throughput is the name of the game, or if a low latency service is required, or a balance between the two, all through the configuration property interface. The single most important configuration properties for performance tuning is `linger.ms` - how long to wait for `batch.num.messages` or `batch.size` to fill up in the local per-partition queue before sending the batch of messages to the broker. In low throughput scenarios, a lower value improves latency. As throughput increases, the cost of each broker request becomes significant impacting both maximum throughput and latency. For higher throughput applications, latency will typically be lower using a higher `linger.ms` due to larger batches resulting in a lesser number of requests, yielding decreased per-message load on the broker. A good general purpose setting is 5ms. For applications seeking maximum throughput, the recommended value is >= 50ms. <a name="high-throughput"></a> ### High throughput The key to high throughput is message batching - waiting for a certain amount of messages to accumulate in the local queue before sending them off in one large message set or batch to the peer. This amortizes the messaging overhead and eliminates the adverse effect of the round trip time (rtt). `linger.ms` (also called `queue.buffering.max.ms`) allows librdkafka to wait up to the specified amount of time to accumulate up to `batch.num.messages` or `batch.size` in a single batch (MessageSet) before sending to the broker. The larger the batch the higher the throughput. Enabling `msg` debugging (set `debug` property to `msg`) will emit log messages for the accumulation process which lets you see what batch sizes are being produced. Example using `linger.ms=1`: ``` ... test [0]: MessageSet with 1514 message(s) delivered ... test [3]: MessageSet with 1690 message(s) delivered ... test [0]: MessageSet with 1720 message(s) delivered ... test [3]: MessageSet with 2 message(s) delivered ... test [3]: MessageSet with 4 message(s) delivered ... test [0]: MessageSet with 4 message(s) delivered ... test [3]: MessageSet with 11 message(s) delivered ``` Example using `linger.ms=1000`: ``` ... test [0]: MessageSet with 10000 message(s) delivered ... test [0]: MessageSet with 10000 message(s) delivered ... test [0]: MessageSet with 4667 message(s) delivered ... test [3]: MessageSet with 10000 message(s) delivered ... test [3]: MessageSet with 10000 message(s) delivered ... test [3]: MessageSet with 4476 message(s) delivered ``` The default setting of `linger.ms=5` is not suitable for high throughput, it is recommended to set this value to >50ms, with throughput leveling out somewhere around 100-1000ms depending on message produce pattern and sizes. These setting are set globally (`rd_kafka_conf_t`) but applies on a per topic+partition basis. <a name="low-latency"></a> ### Low latency When low latency messaging is required the `linger.ms` should be tuned to the maximum permitted producer-side latency. Setting `linger.ms` to 0 or 0.1 will make sure messages are sent as soon as possible. Lower buffering time leads to smaller batches and larger per-message overheads, increasing network, memory and CPU usage for producers, brokers and consumers. See [How to decrease message latency](https://github.com/confluentinc/librdkafka/wiki/How-to-decrease-message-latency) for more info. <a name="latency-measurement"></a> #### Latency measurement End-to-end latency is preferably measured by synchronizing clocks on producers and consumers and using the message timestamp on the consumer to calculate the full latency. Make sure the topic's `log.message.timestamp.type` is set to the default `CreateTime` (Kafka topic configuration, not librdkafka topic). Latencies are typically incurred by the producer, network and broker, the consumer effect on end-to-end latency is minimal. To break down the end-to-end latencies and find where latencies are adding up there are a number of metrics available through librdkafka statistics on the producer: * `brokers[].int_latency` is the time, per message, between produce() and the message being written to a MessageSet and ProduceRequest. High `int_latency` indicates CPU core contention: check CPU load and, involuntary context switches (`/proc/<..>/status`). Consider using a machine/instance with more CPU cores. This metric is only relevant on the producer. * `brokers[].outbuf_latency` is the time, per protocol request (such as ProduceRequest), between the request being enqueued (which happens right after int_latency) and the time the request is written to the TCP socket connected to the broker. High `outbuf_latency` indicates CPU core contention or network congestion: check CPU load and socket SendQ (`netstat -anp | grep :9092`). * `brokers[].rtt` is the time, per protocol request, between the request being written to the TCP socket and the time the response is received from the broker. High `rtt` indicates broker load or network congestion: check broker metrics, local socket SendQ, network performance, etc. * `brokers[].throttle` is the time, per throttled protocol request, the broker throttled/delayed handling of a request due to usage quotas. The throttle time will also be reflected in `rtt`. * `topics[].batchsize` is the size of individual Producer MessageSet batches. See below. * `topics[].batchcnt` is the number of messages in individual Producer MessageSet batches. Due to Kafka protocol overhead a batch with few messages will have a higher relative processing and size overhead than a batch with many messages. Use the `linger.ms` client configuration property to set the maximum amount of time allowed for accumulating a single batch, the larger the value the larger the batches will grow, thus increasing efficiency. When producing messages at a high rate it is recommended to increase linger.ms, which will improve throughput and in some cases also latency. See [STATISTICS.md](STATISTICS.md) for the full definition of metrics. A JSON schema for the statistics is available in [statistics-schema.json](src/statistics-schema.json). <a name="compression"></a> ### Compression Producer message compression is enabled through the `compression.codec` configuration property. Compression is performed on the batch of messages in the local queue, the larger the batch the higher likelyhood of a higher compression ratio. The local batch queue size is controlled through the `batch.num.messages`, `batch.size`, and `linger.ms` configuration properties as described in the **High throughput** chapter above. <a name="message-reliability"></a> ## Message reliability Message reliability is an important factor of librdkafka - an application can rely fully on librdkafka to deliver a message according to the specified configuration (`request.required.acks` and `message.send.max.retries`, etc). If the topic configuration property `request.required.acks` is set to wait for message commit acknowledgements from brokers (any value but 0, see [`CONFIGURATION.md`](CONFIGURATION.md) for specifics) then librdkafka will hold on to the message until all expected acks have been received, gracefully handling the following events: * Broker connection failure * Topic leader change * Produce errors signaled by the broker * Network problems We recommend `request.required.acks` to be set to `all` to make sure produced messages are acknowledged by all in-sync replica brokers. This is handled automatically by librdkafka and the application does not need to take any action at any of the above events. The message will be resent up to `message.send.max.retries` times before reporting a failure back to the application. The delivery report callback is used by librdkafka to signal the status of a message back to the application, it will be called once for each message to report the status of message delivery: * If `error_code` is non-zero the message delivery failed and the error_code indicates the nature of the failure (`rd_kafka_resp_err_t` enum). * If `error_code` is zero the message has been successfully delivered. See Producer API chapter for more details on delivery report callback usage. The delivery report callback is optional but highly recommended. <a name="producer-message-delivery-success"></a> ### Producer message delivery success When a ProduceRequest is successfully handled by the broker and a ProduceResponse is received (also called the ack) without an error code the messages from the ProduceRequest are enqueued on the delivery report queue (if a delivery report callback has been set) and will be passed to the application on the next invocation rd_kafka_poll(). <a name="producer-message-delivery-failure"></a> ### Producer message delivery failure The following sub-chapters explains how different produce errors are handled. If the error is retryable and there are remaining retry attempts for the given message(s), an automatic retry will be scheduled by librdkafka, these retries are not visible to the application. Only permanent errors and temporary errors that have reached their maximum retry count will generate a delivery report event to the application with an error code set. The application should typically not attempt to retry producing the message on failure, but instead configure librdkafka to perform these retries using the `retries`, `retry.backoff.ms` and `retry.backoff.max.ms` configuration properties. <a name="error-timed-out-in-transmission-queue"></a> #### Error: Timed out in transmission queue Internal error ERR__TIMED_OUT_QUEUE. The connectivity to the broker may be stalled due to networking contention, local or remote system issues, etc, and the request has not yet been sent. The producer can be certain that the message has not been sent to the broker. This is a retryable error, but is not counted as a retry attempt since the message was never actually transmitted. A retry by librdkafka at this point will not cause duplicate messages. <a name="error-timed-out-in-flight-tofrom-broker"></a> #### Error: Timed out in flight to/from broker Internal error ERR__TIMED_OUT, ERR__TRANSPORT. Same reasons as for `Timed out in transmission queue` above, with the difference that the message may have been sent to the broker and might be stalling waiting for broker replicas to ack the message, or the response could be stalled due to networking issues. At this point the producer can't know if the message reached the broker, nor if the broker wrote the message to disk and replicas. This is a retryable error. A retry by librdkafka at this point may cause duplicate messages. <a name="error-temporary-broker-side-error"></a> #### Error: Temporary broker-side error Broker errors ERR_REQUEST_TIMED_OUT, ERR_NOT_ENOUGH_REPLICAS, ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND. These errors are considered temporary and librdkafka is will retry them if permitted by configuration. <a name="error-temporary-errors-due-to-stale-metadata"></a> #### Error: Temporary errors due to stale metadata Broker errors ERR_LEADER_NOT_AVAILABLE, ERR_NOT_LEADER_FOR_PARTITION. These errors are considered temporary and a retry is warranted, a metadata request is automatically sent to find a new leader for the partition. A retry by librdkafka at this point will not cause duplicate messages. <a name="error-local-time-out"></a> #### Error: Local time out Internal error ERR__MSG_TIMED_OUT. The message could not be successfully transmitted before `message.timeout.ms` expired, typically due to no leader being available or no broker connection. The message may have been retried due to other errors but those error messages are abstracted by the ERR__MSG_TIMED_OUT error code. Since the `message.timeout.ms` has passed there will be no more retries by librdkafka. <a name="error-permanent-errors"></a> #### Error: Permanent errors Any other error is considered a permanent error and the message will fail immediately, generating a delivery report event with the distinctive error code. The full list of permanent errors depend on the broker version and will likely grow in the future. Typical permanent broker errors are: * ERR_CORRUPT_MESSAGE * ERR_MSG_SIZE_TOO_LARGE - adjust client's or broker's `message.max.bytes`. * ERR_UNKNOWN_TOPIC_OR_PART - topic or partition does not exist, automatic topic creation is disabled on the broker or the application is specifying a partition that does not exist. * ERR_RECORD_LIST_TOO_LARGE * ERR_INVALID_REQUIRED_ACKS * ERR_TOPIC_AUTHORIZATION_FAILED * ERR_UNSUPPORTED_FOR_MESSAGE_FORMAT * ERR_CLUSTER_AUTHORIZATION_FAILED <a name="producer-retries"></a> ### Producer retries The ProduceRequest itself is not retried, instead the messages are put back on the internal partition queue by an insert sort that maintains their original position (the message order is defined at the time a message is initially appended to a partition queue, i.e., after partitioning). A backoff time (`retry.backoff.ms`) is set on the retried messages which effectively blocks retry attempts until the backoff time has expired. <a name="reordering"></a> ### Reordering As for all retries, if `max.in.flight` > 1 and `retries` > 0, retried messages may be produced out of order, since a sub-sequent message in a sub-sequent ProduceRequest may already be in-flight (and accepted by the broker) by the time the retry for the failing message is sent. Using the Idempotent Producer prevents reordering even with `max.in.flight` > 1, see [Idempotent Producer](#idempotent-producer) below for more information. <a name="idempotent-producer"></a> ### Idempotent Producer librdkafka supports the idempotent producer which provides strict ordering and and exactly-once producer guarantees. The idempotent producer is enabled by setting the `enable.idempotence` configuration property to `true`, this will automatically adjust a number of other configuration properties to adhere to the idempotency requirements, see the documentation of `enable.idempotence` in [CONFIGURATION.md](CONFIGURATION.md) for more information. Producer instantiation will fail if the user supplied an incompatible value for any of the automatically adjusted properties, e.g., it is an error to explicitly set `acks=1` when `enable.idempotence=true` is set. <a name="guarantees"></a> #### Guarantees There are three types of guarantees that the idempotent producer can satisfy: * Exactly-once - a message is only written to the log once. Does NOT cover the exactly-once consumer case. * Ordering - a series of messages are written to the log in the order they were produced. * Gap-less - **EXPERIMENTAL** a series of messages are written once and in order without risk of skipping messages. The sequence of messages may be cut short and fail before all messages are written, but may not fail individual messages in the series. This guarantee is disabled by default, but may be enabled by setting `enable.gapless.guarantee` if individual message failure is a concern. Messages that fail due to exceeded timeout (`message.timeout.ms`), are permitted by the gap-less guarantee and may cause gaps in the message series without raising a fatal error. See **Message timeout considerations** below for more info. **WARNING**: This is an experimental property subject to change or removal. All three guarantees are in effect when idempotence is enabled, only gap-less may be disabled individually. <a name="ordering-and-message-sequence-numbers"></a> #### Ordering and message sequence numbers librdkafka maintains the original produce() ordering per-partition for all messages produced, using an internal per-partition 64-bit counter called the msgid which starts at 1. This msgid allows messages to be re-inserted in the partition message queue in the original order in the case of retries. The Idempotent Producer functionality in the Kafka protocol also has a per-message sequence number, which is a signed 32-bit wrapping counter that is reset each time the Producer's ID (PID) or Epoch changes. The librdkafka msgid is used, along with a base msgid value stored at the time the PID/Epoch was bumped, to calculate the Kafka protocol's message sequence number. With Idempotent Producer enabled there is no risk of reordering despite `max.in.flight` > 1 (capped at 5). **Note**: "MsgId" in log messages refer to the librdkafka msgid, while "seq" refers to the protocol message sequence, "baseseq" is the seq of the first message in a batch. MsgId starts at 1, while message seqs start at 0. The producer statistics also maintain two metrics for tracking the next expected response sequence: * `next_ack_seq` - the next sequence to expect an acknowledgement for, which is the last successfully produced MessageSet's last sequence + 1. * `next_err_seq` - the next sequence to expect an error for, which is typically the same as `next_ack_seq` until an error occurs, in which case the `next_ack_seq` can't be incremented (since no messages were acked on error). `next_err_seq` is used to properly handle sub-sequent errors due to a failing first request. **Note**: Both are exposed in partition statistics. <a name="partitioner-considerations"></a> #### Partitioner considerations Strict ordering is guaranteed on a **per partition** basis. An application utilizing the idempotent producer should not mix producing to explicit partitions with partitioner-based partitions since messages produced for the latter are queued separately until a topic's partition count is known, which would insert these messages after the partition-explicit messages regardless of produce order. <a name="message-timeout-considerations"></a> #### Message timeout considerations If messages time out (due to `message.timeout.ms`) while in the producer queue there will be gaps in the series of produced messages. E.g., Messages 1,2,3,4,5 are produced by the application. While messages 2,3,4 are transmitted to the broker the connection to the broker goes down. While the broker is down the message timeout expires for message 2 and 3. As the connection comes back up messages 4, 5 are transmitted to the broker, resulting in a final written message sequence of 1, 4, 5. The producer gracefully handles this case by draining the in-flight requests for a given partition when one or more of its queued (not transmitted) messages are timed out. When all requests are drained the Epoch is bumped and the base sequence number is reset to the first message in the queue, effectively skipping the timed out messages as if they had never existed from the broker's point of view. The message status for timed out queued messages will be `RD_KAFKA_MSG_STATUS_NOT_PERSISTED`. If messages time out while in-flight to the broker (also due to `message.timeout.ms`), the protocol request will fail, the broker connection will be closed by the client, and the timed out messages will be removed from the producer queue. In this case the in-flight messages may be written to the topic log by the broker, even though a delivery report with error `ERR__MSG_TIMED_OUT` will be raised, since the producer timed out the request before getting an acknowledgement back from the broker. The message status for timed out in-flight messages will be `RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED`, indicating that the producer does not know if the messages were written and acked by the broker, or dropped in-flight. An application may inspect the message status by calling `rd_kafka_message_status()` on the message in the delivery report callback, to see if the message was (possibly) persisted (written to the topic log) by the broker or not. Despite the graceful handling of timeouts, we recommend to use a large `message.timeout.ms` to minimize the risk of timeouts. **Warning**: `enable.gapless.guarantee` does not apply to timed-out messages. **Note**: `delivery.timeout.ms` is an alias for `message.timeout.ms`. <a name="leader-change"></a> #### Leader change There are corner cases where an Idempotent Producer has outstanding ProduceRequests in-flight to the previous leader while a new leader is elected. A leader change is typically triggered by the original leader failing or terminating, which has the risk of also failing (some of) the in-flight ProduceRequests to that broker. To recover the producer to a consistent state it will not send any ProduceRequests for these partitions to the new leader broker until all responses for any outstanding ProduceRequests to the previous partition leader has been received, or these requests have timed out. This drain may take up to `min(socket.timeout.ms, message.timeout.ms)`. If the connection to the previous broker goes down the outstanding requests are failed immediately. <a name="error-handling"></a> #### Error handling Background: The error handling for the Idempotent Producer, as initially proposed in the [EOS design document](https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8), missed some corner cases which are now being addressed in [KIP-360](https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Improve+handling+of+unknown+producer). There were some intermediate fixes and workarounds prior to KIP-360 that proved to be incomplete and made the error handling in the client overly complex. With the benefit of hindsight the librdkafka implementation will attempt to provide correctness from the lessons learned in the Java client and provide stricter and less complex error handling. The follow sections describe librdkafka's handling of the Idempotent Producer specific errors that may be returned by the broker. <a name="rd-kafka-resp-err-out-of-order-sequence-number"></a> ##### RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER This error is returned by the broker when the sequence number in the ProduceRequest is larger than the expected next sequence for the given PID+Epoch+Partition (last BaseSeq + msgcount + 1). Note: sequence 0 is always accepted. If the failed request is the head-of-line (next expected sequence to be acked) it indicates desynchronization between the client and broker: the client thinks the sequence number is correct but the broker disagrees. There is no way for the client to recover from this scenario without risking message loss or duplication, and it is not safe for the application to manually retry messages. A fatal error (`RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER`) is raised. When the request is not head-of-line the previous request failed (for any reason), which means the messages in the current request can be retried after waiting for all outstanding requests for this partition to drain and then reset the Producer ID and start over. **Java Producer behaviour**: Fail the batch, reset the pid, and then continue producing (and retrying sub-sequent) messages. This will lead to gaps in the message series. <a name="rd-kafka-resp-err-duplicate-sequence-number"></a> ##### RD_KAFKA_RESP_ERR_DUPLICATE_SEQUENCE_NUMBER Returned by broker when the request's base sequence number is less than the expected sequence number (which is the last written sequence + msgcount). Note: sequence 0 is always accepted. This error is typically benign and occurs upon retrying a previously successful send that was not acknowledged. The messages will be considered successfully produced but will have neither timestamp or offset set. **Java Producer behaviour:** Treats the message as successfully delivered. <a name="rd-kafka-resp-err-unknown-producer-id"></a> ##### RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID Returned by broker when the PID+Epoch is unknown, which may occur when the PID's state has expired (due to topic retention, DeleteRecords, or compaction). The Java producer added quite a bit of error handling for this case, extending the ProduceRequest protocol to return the logStartOffset to give the producer a chance to differentiate between an actual UNKNOWN_PRODUCER_ID or topic retention having deleted the last message for this producer (effectively voiding the Producer ID cache). This workaround proved to be error prone (see explanation in KIP-360) when the partition leader changed. KIP-360 suggests removing this error checking in favour of failing fast, librdkafka follows suite. If the response is for the first ProduceRequest in-flight and there are no messages waiting to be retried nor any ProduceRequests unaccounted for, then the error is ignored and the epoch is incremented, this is likely to happen for an idle producer who's last written message has been deleted from the log, and thus its PID state. Otherwise the producer raises a fatal error (RD_KAFKA_RESP_ERR_UNKNOWN_PRODUCER_ID) since the delivery guarantees can't be satisfied. **Java Producer behaviour:** Retries the send in some cases (but KIP-360 will change this). Not a fatal error in any case. <a name="standard-errors"></a> ##### Standard errors All the standard Produce errors are handled in the usual way, permanent errors will fail the messages in the batch, while temporary errors will be retried (if retry count permits). If a permanent error is returned for a batch in a series of in-flight batches, the sub-sequent batches will fail with RD_KAFKA_RESP_ERR_OUT_OF_ORDER_SEQUENCE_NUMBER since the sequence number of the failed batched was never written to the topic log and next expected sequence thus not incremented on the broker. A fatal error (RD_KAFKA_RESP_ERR__GAPLESS_GUARANTEE) is raised to satisfy the gap-less guarantee (if `enable.gapless.guarantee` is set) by failing all queued messages. <a name="message-persistence-status"></a> ##### Message persistence status To help the application decide what to do in these error cases, a new per-message API is introduced, `rd_kafka_message_status()`, which returns one of the following values: * `RD_KAFKA_MSG_STATUS_NOT_PERSISTED` - the message has never been transmitted to the broker, or failed with an error indicating it was not written to the log. Application retry will risk ordering, but not duplication. * `RD_KAFKA_MSG_STATUS_POSSIBLY_PERSISTED` - the message was transmitted to the broker, but no acknowledgement was received. Application retry will risk ordering and duplication. * `RD_KAFKA_MSG_STATUS_PERSISTED` - the message was written to the log by the broker and fully acknowledged. No reason for application to retry. This method should be called by the application on delivery report error. <a name="transactional-producer"></a> ### Transactional Producer <a name="error-handling-1"></a> #### Error handling Using the transactional producer simplifies error handling compared to the standard or idempotent producer, a transactional application will only need to care about these different types of errors: * Retriable errors - the operation failed due to temporary problems, such as network timeouts, the operation may be safely retried. Use `rd_kafka_error_is_retriable()` to distinguish this case. * Abortable errors - if any of the transactional APIs return a non-fatal error code the current transaction has failed and the application must call `rd_kafka_abort_transaction()`, rewind its input to the point before the current transaction started, and attempt a new transaction by calling `rd_kafka_begin_transaction()`, etc. Use `rd_kafka_error_txn_requires_abort()` to distinguish this case. * Fatal errors - the application must cease operations and destroy the producer instance. Use `rd_kafka_error_is_fatal()` to distinguish this case. * For all other errors returned from the transactional API: the current recommendation is to treat any error that has neither retriable, abortable, or fatal set, as a fatal error. While the application should log the actual fatal or abortable errors, there is no need for the application to handle the underlying errors specifically. <a name="old-producer-fencing"></a> #### Old producer fencing If a new transactional producer instance is started with the same `transactional.id`, any previous still running producer instance will be fenced off at the next produce, commit or abort attempt, by raising a fatal error with the error code set to `RD_KAFKA_RESP_ERR__FENCED`. <a name="configuration-considerations"></a> #### Configuration considerations To make sure messages time out (in case of connectivity problems, etc) within the transaction, the `message.timeout.ms` configuration property must be set lower than the `transaction.timeout.ms`, this is enforced when creating the producer instance. If `message.timeout.ms` is not explicitly configured it will be adjusted automatically. <a name="exactly-once-semantics-eos-and-transactions"></a> ### Exactly Once Semantics (EOS) and transactions librdkafka supports Exactly One Semantics (EOS) as defined in [KIP-98](https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging). For more on the use of transactions, see [Transactions in Apache Kafka](https://www.confluent.io/blog/transactions-apache-kafka/). See [examples/transactions.c](examples/transactions.c) for an example transactional EOS application. **Warning** If the broker version is older than Apache Kafka 2.5.0 then one transactional producer instance per consumed input partition is required. For 2.5.0 and later a single producer instance may be used regardless of the number of input partitions. See KIP-447 for more information. <a name="usage"></a> ## Usage <a name="documentation"></a> ### Documentation The librdkafka API is documented in the [`rdkafka.h`](src/rdkafka.h) header file, the configuration properties are documented in [`CONFIGURATION.md`](CONFIGURATION.md) <a name="initialization"></a> ### Initialization The application needs to instantiate a top-level object `rd_kafka_t` which is the base container, providing global configuration and shared state. It is created by calling `rd_kafka_new()`. It also needs to instantiate one or more topics (`rd_kafka_topic_t`) to be used for producing to or consuming from. The topic object holds topic-specific configuration and will be internally populated with a mapping of all available partitions and their leader brokers. It is created by calling `rd_kafka_topic_new()`. Both `rd_kafka_t` and `rd_kafka_topic_t` comes with a configuration API which is optional. Not using the API will cause librdkafka to use its default values which are documented in [`CONFIGURATION.md`](CONFIGURATION.md). **Note**: An application may create multiple `rd_kafka_t` objects and they share no state. **Note**: An `rd_kafka_topic_t` object may only be used with the `rd_kafka_t` object it was created from. <a name="configuration"></a> ### Configuration To ease integration with the official Apache Kafka software and lower the learning curve, librdkafka implements identical configuration properties as found in the official clients of Apache Kafka. Configuration is applied prior to object creation using the `rd_kafka_conf_set()` and `rd_kafka_topic_conf_set()` APIs. **Note**: The `rd_kafka.._conf_t` objects are not reusable after they have been passed to `rd_kafka.._new()`. The application does not need to free any config resources after a `rd_kafka.._new()` call. <a name="example"></a> #### Example ```c rd_kafka_conf_t *conf; rd_kafka_conf_res_t res; rd_kafka_t *rk; char errstr[512]; conf = rd_kafka_conf_new(); res = rd_kafka_conf_set(conf, "compression.codec", "snappy", errstr, sizeof(errstr)); if (res != RD_KAFKA_CONF_OK) fail("%s\n", errstr); res = rd_kafka_conf_set(conf, "batch.num.messages", "100", errstr, sizeof(errstr)); if (res != RD_KAFKA_CONF_OK) fail("%s\n", errstr); rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); if (!rk) { rd_kafka_conf_destroy(rk); fail("Failed to create producer: %s\n", errstr); } /* Note: librdkafka takes ownership of the conf object on success */ ``` Configuration properties may be set in any order (except for interceptors) and may be overwritten before being passed to `rd_kafka_new()`. `rd_kafka_new()` will verify that the passed configuration is consistent and will fail and return an error if incompatible configuration properties are detected. It will also emit log warnings for deprecated and problematic configuration properties. <a name="termination"></a> ### Termination librdkafka is asynchronous in its nature and performs most operation in its background threads. Calling the librdkafka handle destructor tells the librdkafka background threads to finalize their work, close network connections, clean up, etc, and may thus take some time. The destructor (`rd_kafka_destroy()`) will block until all background threads have terminated. If the destructor blocks indefinitely it typically means there is an outstanding object reference, such as a message or topic object, that was not destroyed prior to destroying the client handle. All objects except for the handle (C: `rd_kafka_t`, C++: `Consumer,KafkaConsumer,Producer`), such as topic objects, messages, `topic_partition_t`, `TopicPartition`, events, etc, **MUST** be destroyed/deleted prior to destroying or closing the handle. For C, make sure the following objects are destroyed prior to calling `rd_kafka_consumer_close()` and `rd_kafka_destroy()`: * `rd_kafka_message_t` * `rd_kafka_topic_t` * `rd_kafka_topic_partition_t` * `rd_kafka_topic_partition_list_t` * `rd_kafka_event_t` * `rd_kafka_queue_t` For C++ make sure the following objects are deleted prior to calling `KafkaConsumer::close()` and delete on the Consumer, KafkaConsumer or Producer handle: * `Message` * `Topic` * `TopicPartition` * `Event` * `Queue` <a name="high-level-kafkaconsumer"></a> #### High-level KafkaConsumer Proper termination sequence for the high-level KafkaConsumer is: ```c /* 1) Leave the consumer group, commit final offsets, etc. */ rd_kafka_consumer_close(rk); /* 2) Destroy handle object */ rd_kafka_destroy(rk); ``` **NOTE**: There is no need to unsubscribe prior to calling `rd_kafka_consumer_close()`. **NOTE**: Any topic objects created must be destroyed prior to rd_kafka_destroy() Effects of not doing the above, for: 1. Final offsets are not committed and the consumer will not actively leave the group, it will be kicked out of the group after the `session.timeout.ms` expires. It is okay to omit the `rd_kafka_consumer_close()` call in case the application does not want to wait for the blocking close call. 2. librdkafka will continue to operate on the handle. Actual memory leaks. <a name="producer"></a> #### Producer The proper termination sequence for Producers is: ```c /* 1) Make sure all outstanding requests are transmitted and handled. */ rd_kafka_flush(rk, 60*1000); /* One minute timeout */ /* 2) Destroy the topic and handle objects */ rd_kafka_topic_destroy(rkt); /* Repeat for all topic objects held */ rd_kafka_destroy(rk); ``` Effects of not doing the above, for: 1. Messages in-queue or in-flight will be dropped. 2. librdkafka will continue to operate on the handle. Actual memory leaks. <a name="admin-api-client"></a> #### Admin API client Unlike the Java Admin client, the Admin APIs in librdkafka are available on any type of client instance and can be used in combination with the client type's main functionality, e.g., it is perfectly fine to call `CreateTopics()` in your running producer, or `DeleteRecords()` in your consumer. If you need a client instance to only perform Admin API operations the recommendation is to create a producer instance since it requires less configuration (no `group.id`) than the consumer and is generally more cost efficient. We do recommend that you set `allow.auto.create.topics=false` to avoid topic metadata lookups to unexpectedly have the broker create topics. <a name="speeding-up-termination"></a> #### Speeding up termination To speed up the termination of librdkafka an application can set a termination signal that will be used internally by librdkafka to quickly cancel any outstanding I/O waits. Make sure you block this signal in your application. ```c char tmp[16]; snprintf(tmp, sizeof(tmp), "%i", SIGIO); /* Or whatever signal you decide */ rd_kafka_conf_set(rk_conf, "internal.termination.signal", tmp, errstr, sizeof(errstr)); ``` <a name="threads-and-callbacks"></a> ### Threads and callbacks librdkafka uses multiple threads internally to fully utilize modern hardware. The API is completely thread-safe and the calling application may call any of the API functions from any of its own threads at any time. A poll-based API is used to provide signaling back to the application, the application should call rd_kafka_poll() at regular intervals. The poll API will call the following configured callbacks (optional): * `dr_msg_cb` - Message delivery report callback - signals that a message has been delivered or failed delivery, allowing the application to take action and to release any application resources used in the message. * `error_cb` - Error callback - signals an error. These errors are usually of an informational nature, i.e., failure to connect to a broker, and the application usually does not need to take any action. The type of error is passed as a rd_kafka_resp_err_t enum value, including both remote broker errors as well as local failures. An application typically does not have to perform any action when an error is raised through the error callback, the client will automatically try to recover from all errors, given that the client and cluster is correctly configured. In some specific cases a fatal error may occur which will render the client more or less inoperable for further use: if the error code in the error callback is set to `RD_KAFKA_RESP_ERR__FATAL` the application should retrieve the underlying fatal error and reason using the `rd_kafka_fatal_error()` call, and then begin terminating the instance. The Event API's EVENT_ERROR has a `rd_kafka_event_error_is_fatal()` function, and the C++ EventCb has a `fatal()` method, to help the application determine if an error is fatal or not. * `stats_cb` - Statistics callback - triggered if `statistics.interval.ms` is configured to a non-zero value, emitting metrics and internal state in JSON format, see [STATISTICS.md]. * `throttle_cb` - Throttle callback - triggered whenever a broker has throttled (delayed) a request. These callbacks will also be triggered by `rd_kafka_flush()`, `rd_kafka_consumer_poll()`, and any other functions that serve queues. Optional callbacks not triggered by poll, these may be called spontaneously from any thread at any time: * `log_cb` - Logging callback - allows the application to output log messages generated by librdkafka. * `partitioner_cb` - Partitioner callback - application provided message partitioner. The partitioner may be called in any thread at any time, it may be called multiple times for the same key. Partitioner function contraints: - MUST NOT call any rd_kafka_*() functions - MUST NOT block or execute for prolonged periods of time. - MUST return a value between 0 and partition_cnt-1, or the special RD_KAFKA_PARTITION_UA value if partitioning could not be performed. <a name="brokers"></a> ### Brokers On initialization, librdkafka only needs a partial list of brokers (at least one), called the bootstrap brokers. The client will connect to the bootstrap brokers specified by the `bootstrap.servers` configuration property and query cluster Metadata information which contains the full list of brokers, topic, partitions and their leaders in the Kafka cluster. Broker names are specified as `host[:port]` where the port is optional (default 9092) and the host is either a resolvable hostname or an IPv4 or IPv6 address. If host resolves to multiple addresses librdkafka will round-robin the addresses for each connection attempt. A DNS record containing all broker address can thus be used to provide a reliable bootstrap broker. <a name="ssl"></a> #### SSL If the client is to connect to a broker's SSL endpoints/listeners the client needs to be configured with `security.protocol=SSL` for just SSL transport or `security.protocol=SASL_SSL` for SASL authentication and SSL transport. The client will try to verify the broker's certificate by checking the CA root certificates, if the broker's certificate can't be verified the connection is closed (and retried). This is to protect the client from connecting to rogue brokers. The CA root certificate defaults are system specific: * On Linux, Mac OSX, and other Unix-like system the OpenSSL default CA path will be used, also called the OPENSSLDIR, which is typically `/etc/ssl/certs` (on Linux, typcially in the `ca-certificates` package) and `/usr/local/etc/openssl` on Mac OSX (Homebrew). * On Windows the Root certificate store is used, unless `ssl.ca.certificate.stores` is configured in which case certificates are read from the specified stores. * If OpenSSL is linked statically, librdkafka will set the default CA location to the first of a series of probed paths (see below). If the system-provided default CA root certificates are not sufficient to verify the broker's certificate, such as when a self-signed certificate or a local CA authority is used, the CA certificate must be specified explicitly so that the client can find it. This can be done either by providing a PEM file (e.g., `cacert.pem`) as the `ssl.ca.location` configuration property, or by passing an in-memory PEM, X.509/DER or PKCS#12 certificate to `rd_kafka_conf_set_ssl_cert()`. It is also possible to disable broker certificate verification completely by setting `enable.ssl.certificate.verification=false`, but this is not recommended since it allows for rogue brokers and man-in-the-middle attacks, and should only be used for testing and troubleshooting purposes. CA location probe paths (see [rdkafka_ssl.c](src/rdkafka_ssl.c) for full list) used when OpenSSL is statically linked: "/etc/pki/tls/certs/ca-bun