@confluentinc/kafka-javascript
Version:
Node.js bindings for librdkafka
1,171 lines (895 loc) • 121 kB
Markdown
<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)
- [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 of the consumer group protocol: KIP 848](#next-generation-of-the-consumer-group-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-bundle.crt",
"/etc/ssl/certs/ca-bundle.crt",
"/etc/pki/tls/certs/ca-bundle.trust.crt",
"/etc/pki/ca-trust/extracted/pem/tls-ca-bundle.pem",
"/etc/ssl/ca-bundle.pem",
"/etc/pki/tls/cacert.pem",
"/etc/ssl/cert.pem",
"/etc/ssl/cacert.pem",
"/etc/certs/ca-certificates.crt",
"/etc/ssl/certs/ca-certificates.crt",
"/etc/ssl/certs",
"/usr/local/etc/ssl/cert.pem",
"/usr/local/etc/ssl/cacert.pem",
"/usr/local/etc/ssl/certs/cert.pem",
"/usr/local/etc/ssl/certs/cacert.pem",
etc..
On **Windows** the Root certificate store is read by default, but any number
of certificate stores can be read by setting the `ssl.ca.certificate.stores`
configuration property to a comma-separated list of certificate store names.
The predefined system store names are:
* `MY` - User certificates
* `Root` - System CA certificates (default)
* `CA` - Intermediate CA certificates
* `Trust` - Trusted publishers
For example, to read b