librdkafka v2.3.0 is a feature release:
- KIP-516
Partial support of topic identifiers. Topic identifiers in metadata response
available through the newrd_kafka_DescribeTopics
function (#4300, #4451). - KIP-117 Add support for AdminAPI
DescribeCluster()
andDescribeTopics()
(#4240, @jainruchir). - KIP-430:
Return authorized operations in Describe Responses.
(#4240, @jainruchir). - KIP-580: Added Exponential Backoff mechanism for
retriable requests withretry.backoff.ms
as minimum backoff andretry.backoff.max.ms
as the
maximum backoff, with 20% jitter (#4422). - KIP-396: completed the implementation with
the addition of ListOffsets (#4225). - Fixed ListConsumerGroupOffsets not fetching offsets for all the topics in a group with Apache Kafka version below 2.4.0.
- Add missing destroy that leads to leaking partition structure memory when there
are partition leader changes and a stale leader epoch is received (#4429). - Fix a segmentation fault when closing a consumer using the
cooperative-sticky assignor before the first assignment (#4381). - Fix for insufficient buffer allocation when allocating rack information (@wolfchimneyrock, #4449).
- Fix for infinite loop of OffsetForLeaderEpoch requests on quick leader changes. (#4433).
- Fix to add leader epoch to control messages, to make sure they're stored
for committing even without a subsequent fetch message (#4434). - Fix for stored offsets not being committed if they lacked the leader epoch (#4442).
- Upgrade OpenSSL to v3.0.11 (while building from source) with various security fixes,
check the release notes
(#4454, started by @migarc1). - Fix to ensure permanent errors during offset validation continue being retried and
don't cause an offset reset (#4447). - Fix to ensure max.poll.interval.ms is reset when rd_kafka_poll is called with
consume_cb (#4431). - Fix for idempotent producer fatal errors, triggered after a possibly persisted message state (#4438).
- Fix
rd_kafka_query_watermark_offsets
continuing beyond timeout expiry (#4460). - Fix
rd_kafka_query_watermark_offsets
not refreshing the partition leader
after a leader change and subsequentNOT_LEADER_OR_FOLLOWER
error (#4225).
Upgrade considerations
-
retry.backoff.ms
:
If it is set greater thanretry.backoff.max.ms
which has the default value of 1000 ms then it is assumes the value ofretry.backoff.max.ms
.
To change this behaviour make sure thatretry.backoff.ms
is always less thanretry.backoff.max.ms
.
If equal then the backoff will be linear instead of exponential. -
topic.metadata.refresh.fast.interval.ms
:
If it is set greater thanretry.backoff.max.ms
which has the default value of 1000 ms then it is assumes the value ofretry.backoff.max.ms
.
To change this behaviour make sure thattopic.metadata.refresh.fast.interval.ms
is always less thanretry.backoff.max.ms
.
If equal then the backoff will be linear instead of exponential.
Fixes
General fixes
- An assertion failed with insufficient buffer size when allocating
rack information on 32bit architectures.
Solved by aligning all allocations to the maximum allowed word size (#4449). - The timeout for
rd_kafka_query_watermark_offsets
was not enforced after
making the necessary ListOffsets requests, and thus, it never timed out in
case of broker/network issues. Fixed by setting an absolute timeout (#4460).
Idempotent producer fixes
- After a possibly persisted error, such as a disconnection or a timeout, next expected sequence
used to increase, leading to a fatal error if the message wasn't persisted and
the second one in queue failed with anOUT_OF_ORDER_SEQUENCE_NUMBER
.
The error could contain the message "sequence desynchronization" with
just one possibly persisted error or "rewound sequence number" in case of
multiple errored messages.
Solved by treating the possible persisted message as not persisted,
and expecting aDUPLICATE_SEQUENCE_NUMBER
error in case it was or
NO_ERROR
in case it wasn't, in both cases the message will be considered
delivered (#4438).
Consumer fixes
- Stored offsets were excluded from the commit if the leader epoch was
less than committed epoch, as it's possible if leader epoch is the default -1.
This didn't happen in Python, Go and .NET bindings when stored position was
taken from the message.
Solved by checking only that the stored offset is greater
than committed one, if either stored or committed leader epoch is -1 (#4442). - If an OffsetForLeaderEpoch request was being retried, and the leader changed
while the retry was in-flight, an infinite loop of requests was triggered,
because we weren't updating the leader epoch correctly.
Fixed by updating the leader epoch before sending the request (#4433). - During offset validation a permanent error like host resolution failure
would cause an offset reset.
This isn't what's expected or what the Java implementation does.
Solved by retrying even in case of permanent errors (#4447). - If using
rd_kafka_poll_set_consumer
, along with a consume callback, and then
callingrd_kafka_poll
to service the callbacks, would not reset
max.poll.interval.ms.
This was because we were only checkingrk_rep
for
consumer messages, while the method to service the queue internally also
services the queue forwarded to fromrk_rep
, which isrkcg_q
.
Solved by moving themax.poll.interval.ms
check intord_kafka_q_serve
(#4431). - After a leader change a
rd_kafka_query_watermark_offsets
call would continue
trying to call ListOffsets on the old leader, if the topic wasn't included in
the subscription set, so it started querying the new leader only after
topic.metadata.refresh.interval.ms
(#4225).
Checksums
Release asset checksums:
- v2.3.0.zip SHA256
15e77455811b3e5d869d6f97ce765b634c7583da188792e2930a2098728e932b
- v2.3.0.tar.gz SHA256
2d49c35c77eeb3d42fa61c43757fcbb6a206daa560247154e60642bcdcc14d12