Kafka C/C++ Client Changelog
2.14.0
2.14.0 is a feature release:
KIP-768 Extend SASL/OAUTHBEARER to support OIDC claim mapping beyond the default
subclaim.
2.13.2
2.13.2 is a maintenance release:
The
librdkafka.redistNuGet package now includes binary for alpine-arm64.Remove CPU usage regression when a subscription matches no topics.
Fix
rd_kafka_consume_batch_queueincorrectly updating the application position on EOF or error messages.Fix compilation without
getentropy.Use a truly random seed for pseudo-random number generation whenever available.
Fix
rd_listdestroy callback type mismatch by changingrd_kafka_assignor_destroyto take avoid *argument, as expected byrd_list_init()destroy callbacks, and casting internally tord_kafka_assignor_t.
Fixes
General fixes
Fix compilation without
getentropy. glibc versions lacking support are those less than 2.25 (2017). Happening since 2.13.0.
Consumer fixes
Remove CPU usage regression when a subscription matches no topics. The increased CPU usage (~30%) was seen in particular when there are many topics in the clusters and the given subscription regex doesn’t match any. Happening since 2.10.0.
Fix
rd_kafka_consume_batch_queueincorrectly updating the application position when receiving EOF or error messages, causing the position to move forward and likely be stored and committed. When storing the application offset the leader epoch is also considered for correct offset ordering in case of log truncation. Happening since 2.2.0.
2.13.0
2.13.0 is a feature release:
KIP-482 Upgrade CreateAcls, DescribeAcls, DeleteAcls to the first version supporting this KIP.
KIP-482 Upgrade DescribeGroups, DeleteTopics, DeleteRecords, CreatePartitions, DeleteGroups to the first version supporting this KIP.
Strip trailing dot of hostname to fix SSL certificate verification issue.
Fix memory management for interceptors in rd_kafka_conf to prevent double-free errors.
Fix for the pseudo-random generator seed on Windows involving as well the uniqueness of the new consumer group protocol member ID.
Add secure random generation functionality used for UUID uniqueness and secure salt generation in
rd_kafka_UserScramCredentialUpsertionusing OpenSSL or the POSIX or WIN32 equivalent calls when it isn’t available.
Fixes
General fixes
Strip trailing dot of hostname to fix SSL certificate verification issue. Happening since 1.x.
Fix memory management for interceptors in rd_kafka_conf to prevent double-free errors. In case the client instance fails the users needs to destroy the configuration data structure, it was causing a double-free because the interceptors were already freed in the constructor. Happening since 1.x.
Fix for the pseudo-random seed on Windows. The function
rand_risn’t present on Windows and the global seed wasn’t based on the current microseconds and thread ID. Also it wasn’t called on every thread as required on this platform but only once per process. The fix allows on this platform the uniqueness of client side member ID generation in next-generation consumer group protocol. Happening since 1.x.
2.12.1
2.12.1 is a maintenance release:
Restored macOS binaries compatibility with macOS 13 and 14.
Fixes
General fixes
Fix to restore macOS 13 and 14 compatibility in prebuilt binaries present in
librdkafka.redist. Happening since 2.12.0.
2.12.0
2.12.0 is a feature release:
KIP-848 – General availability
Starting with librdkafka 2.12.0, the next generation consumer group rebalance protocol defined in KIP-848 is production-ready. Please refer the following migration guide for moving from classic to consumer protocol.
Note
The new consumer group protocol defined in KIP-848 is not enabled by default. There are few contract change associated with the new protocol and might cause breaking changes. group.protocol configuration property dictates whether to use the new consumer protocol or older classic protocol. It defaults to classic if not provided.
Enhancements and fixes
Support for OAUTHBEARER metadata based authentication types, starting with Azure IMDS. See Introduction available.
Fix compression types read issue in GetTelemetrySubscriptions response for big-endian architectures.
Fix for KIP-1102 time based re-bootstrap condition.
Fix for discarding the member epoch in a consumer group heartbeat response when leaving with an inflight HB.
Fix for an error being raised after a commit due to an existing error in the topic partition.
Fix double free of headers in
rd_kafka_producevamethod.Fix to ensure
rd_kafka_query_watermark_offsetsenforces the specified timeout and does not continue beyond timeout expiry.New walkthrough in the Wiki about configuring Kafka cross-realm authentication between Windows SSPI and MIT Kerberos.
Fixes
General fixes
Fix for KIP-1102 time based re-bootstrap condition. Re-bootstrap is now triggered only after
metadata.recovery.rebootstrap.trigger.mshave passed since first metadata refresh request after last successful metadata response. The calculation was since last successful metadata response so it’s possible it did overlap with the periodictopic.metadata.refresh.interval.msand cause a re-bootstrap even if not needed. Happening since 2.11.0.Fix to ensure
rd_kafka_query_watermark_offsetsenforces the specified timeout and does not continue beyond timeout expiry. Happening since 2.3.0.
Telemetry fixes
Fix issue in GetTelemetrySubscriptions with big-endian architectures where wrong values are read as accepted compression types causing the metrics to be sent uncompressed. Happening since 2.5.0. Since 2.10.1 unit tests are failing when run on big-endian architectures.
Consumer fixes
Fixed an issue where topic partition errors were not cleared after a successful commit. Previously, a partition could retain a stale error state even though the most recent commit succeeded, causing misleading error reporting. Now, successful commits correctly clear the error state for the affected partitions Happening since 2.4.0.
Producer fixes
Fix double free of headers in
rd_kafka_producevamethod in cases where the partition doesn’t exist. Happening since 1.x.
2.11.1
2.11.1 is a maintenance release:
Made the conditions for enabling the features future proof.
Avoid returning an all brokers down error on planned disconnections.
An “all brokers down” error isn’t returned when we haven’t tried to connect to all brokers since last successful connection.
Fixes
General fixes
Made the conditions for enabling the features future proof, allowing to remove RPC versions in a subsequent Apache Kafka® version without disabling features. The existing checks were matching a single version instead of a range and were failing if the older version was removed. Happening since 1.x.
Avoid returning an all brokers down error on planned disconnections. This is done by avoiding to count planned disconnections, such as idle disconnections, broker host change and similar as events that can cause the client to reach the “all brokers down” state, returning an error and since 2.10.0 possibly starting a re-bootstrap sequence. Happening since 1.x.
An “all brokers down” error isn’t returned when we haven’t tried to connect to all brokers since last successful connection. It happened because the down state is cached and can be stale when a connection isn’t needed to that particular broker. Solved by resetting the cached broker down state when any broker successfully connects, so that broker needs to be tried again. Happening since 1.x.
2.11.0
2.11.0 is a feature release:
KIP-1102 Enable clients to rebootstrap based on timeout or error code.
KIP-1139 Add support for OAuth jwt-bearer grant type.
Fix for poll ratio calculation in case the queues are forwarded.
Fix data race when buffer queues are being reset instead of being initialized.
Features BROKER_BALANCED_CONSUMER and SASL_GSSAPI don’t depend on JoinGroup v0 anymore, missing in AK 4.0 and CP 8.0.
Improve HTTPS CA certificates configuration by probing several paths when OpenSSL is statically linked and providing a way to customize their location or value.
Fixes
General fixes
A data race happened when emptying buffers of a failing broker, in its thread, with the statistics callback in main thread gathering the buffer counts. Solved by resetting the atomic counters instead of initializing them. Happening since 1.x.
Features BROKER_BALANCED_CONSUMER and SASL_GSSAPI don’t depend on JoinGroup v0 anymore, missing in AK 4.0 and CP 8.0. This PR partially fixes the linked issue, a complete fix for all features will follow. Rest of fixes are necessary only for a subsequent Apache Kafka® major version, such as AK 5.x. Happening since 1.x.
Telemetry fixes
Fix for poll ratio calculation in case the queues are forwarded. Poll ratio is now calculated per queue instead of per instance and it allows to avoid calculation problems linked to using the same field. Happens since 2.6.0.
2.10.1
2.10.1 is a maintenance release:
Fix to add locks when updating the metadata cache for the consumer after no broker connection is available.
Fix to the re-bootstrap case when
bootstrap.serversisNULLand brokers were added manually throughrd_kafka_brokers_add.Fix an issue where the first message to any topic produced via
producevorproducevawas delivered late by up to one second.Fix for a loop of re-bootstrap sequences in case the client reaches the
all brokers downstate.Fix for frequent disconnections on push telemetry requests with particular metric configurations.
Avoid copy outside boundaries when reading metric names in telemetry subscription.
Metrics aren’t duplicated when multiple prefixes match them.
Fixes
General fixes
Fix for a loop of re-bootstrap sequences in case the client reaches the
all brokers downstate. The client continues to select the bootstrap brokers given they have no connection attempt and doesn’t reconnect to the learned ones. In case it happens a broker restart can break the loop for the clients using the affected version. Fixed by giving a higher chance to connect to the learned brokers even if there are new ones that never tried to connect. Happens since 2.10.0.Fix to the re-bootstrap case when
bootstrap.serversisNULLand brokers were added manually throughrd_kafka_brokers_add. Avoids a segmentation fault in this case. Happens since 2.10.0.
Producer fixes
In case of
producevorproduceva, the producer did not enqueue a leader query metadata request immediately, and rather, waited for the one second timer to kick in. This could cause delays in the sending of the first message by up to one second. Happens since 1.x.
Consumer fixes
Fix to add locks when updating the metadata cache for the consumer. It can cause memory corruption or use-after-free in case there’s no broker connection and the consumer group metadata needs to be updated. Happens since 2.10.0.
Telemetry fixes
Fix for frequent disconnections on push telemetry requests with particular metric configurations. A
NULLpayload is sent in a push telemetry request when an empty one is needed. This causes disconnections every time the push is sent, only when metrics are requested and some metrics are matching the producer but none the consumer or the other way around. Happens since 2.5.0.Avoid copy outside boundaries when reading metric names in telemetry subscription. It can cause that some metrics aren’t matched. Happens since 2.5.0.
Telemetry metrics aren’t duplicated when multiple prefixes match them. Fixed by keeping track of the metrics that already matched. Happens since 2.5.0.
2.10.0
2.10.0 is a feature release:
Warning
We suggest you upgrade to 2.10.1 or later because of the possibly critical bug #5088.
KIP-848 – Now in Preview
KIP-848 has transitioned from Early Access to Preview.
Added support for regex-based subscriptions.
Implemented client-side member ID generation as per KIP-1082.
rd_kafka_DescribeConsumerGroups()now supports KIP-848-styleconsumergroups. Two new fields have been added:
Group type – Indicates whether the group is
classicorconsumer.Target assignment – Applicable only to
consumerprotocol groups (defaults toNULL).
Group configuration is now supported in
AlterConfigs,IncrementalAlterConfigs, andDescribeConfigs.Added Topic Authorization Error support in the
ConsumerGroupHeartbeatresponse.Removed usage of the
partition.assignment.strategyproperty for theconsumergroup protocol. An error will be raised if this is set withgroup.protocol=consumer.Deprecated and disallowed the following properties for the
consumergroup protocol:
session.timeout.ms
heartbeat.interval.ms
group.protocol.typeAttempting to set any of these will result in an error.
Enhanced handling for
subscribe()andunsubscribe()edge cases.
Note
The KIP-848 consumer is currently in Preview and should not be used in production environments. Implementation is feature complete but contract could have minor changes before General Availability.
Upgrade considerations
Starting from this version, brokers not reported in metadata RPC call are removed along with their threads. Brokers and their threads are added back when they appear in a metadata RPC response again. When no brokers are left or they’re not reachable, the client will start a re-bootstrap sequence by default. metadata.recovery.strategy controls this, which defaults to rebootstrap. Setting metadata.recovery.strategy to none avoids any re-bootstrapping and leaves only the broker received in last successful metadata response.
Enhancements and fixes
KIP-899 Allow producer and consumer clients to rebootstrap.
Identify brokers only by broker ID.
Remove unavailable brokers and their thread.
Commits during a cooperative incremental rebalance aren’t causing an assignment lost if the generation ID was bumped in between.
Fix for librdkafka yielding before timeouts had been reached.
Removed a 500 ms latency when a consumer partition switches to a different leader.
The mock cluster implementation removes brokers from metadata response when they’re not available. This simulates better the actual behavior of a cluster that is using KRaft.
Doesn’t remove topics from cache on temporary metadata errors but only on metadata cache expiry.
Doesn’t mark the topic as unknown if it had been marked as existent earlier and
topic.metadata.propagation.max.mshasn’t passed still.Doesn’t update partition leaders if the topic in metadata response has errors.
Only topic authorization errors in a metadata response are considered permanent and are returned to the user.
The function
rd_kafka_offsets_for_timesrefreshes leader information if the error requires it, allowing it to succeed on subsequent manual retries.Deprecated
api.version.request,api.version.fallback.msandbroker.version.fallbackconfiguration properties.When consumer is closed before destroying the client, the operations queue isn’t purged anymore as it contains operations unrelated to the consumer group.
When making multiple changes to the consumer subscription in a short time, no unknown topic error is returned for topics that are in the new subscription but weren’t in previous one.
Prevent metadata cache corruption when topic ID changes.
Fix for the case where a metadata refresh enqueued on an unreachable broker prevents refreshing the controller or the coordinator until that broker becomes reachable again.
Remove a one second wait after a partition fetch is restarted following a leader change and offset validation.
Fix Nagle’s algorithm (TCP_NODELAY) on broker sockets to not be enabled by default.
Fixes
General fixes
Brokers are now identified exclusively by broker ID, which matches Java client behavior, instead of by hostname. This prevents incorrectly reusing threads and connections when multiple brokers share the same hostname. Happens since 1.x
Remove brokers and their threads if they are not included in metadata responses. This prevents the client from attempting to connect to unavailable brokers. Because it is not clear if a broker is removed temporarily or permanently, all absent brokers are removed and are re-added automatically if is included in subsequent metadata responses. Happens since 1.x.
librdkafka code using
cnd_timedwaitwas yielding before a timeout occurred without the condition being fulfilled because of spurious wake-ups. Solved by verifying with a monotonic clock that the expected point in time was reached and calling the function again if needed. Happens since 1.x.Doesn’t remove topics from cache on temporary metadata errors but only on metadata cache expiry. It allows the client to continue working in case of temporary problems to the Kafka metadata plane. Happens since 1.x.
Doesn’t mark the topic as unknown if it had been marked as existent earlier and
topic.metadata.propagation.max.mshasn’t passed still. It achieves this property expected effect even if a different broker had previously reported the topic as existent. Happens since 1.x.Doesn’t update partition leaders if the topic in metadata response has errors. It’s in line with what Java client does and allows to avoid segmentation faults for unknown partitions. Happens since 1.x.
Only topic authorization errors in a metadata response are considered permanent and are returned to the user. It’s in line with what Java client does and avoids returning to the user an error that wasn’t meant to be permanent. Happens since 1.x.
Prevent metadata cache corruption when topic ID for the same topic name changes. Solved by correctly removing the entry with the old topic ID from metadata cache to prevent subsequent use-after-free. Happens since 2.4.0.
Fix for the case where a metadata refresh enqueued on an unreachable broker prevents refreshing the controller or the coordinator until that broker becomes reachable again. Given the request continues to be retried on that broker, the counter for refreshing complete broker metadata doesn’t reach zero and prevents the client from obtaining the new controller or group or transactional coordinator. It causes a series of debug messages like: “Skipping metadata request: … full request already in-transit”, until the broker the request is enqueued on is up again. Solved by not retrying these kinds of metadata requests. Happens since 1.x.
The Nagle’s algorithm (TCP_NODELAY) is now disabled by default. It caused a large increase in latency for some use cases, for example, when using an SSL connection. For efficient batching, the application should use
linger.ms,batch.sizeetc. Happens since: 0.x.
Consumer fixes
Commits during a cooperative incremental rebalance could cause an assignment lost if the generation ID was bumped by a second join group request. Solved by not rejoining the group in case an illegal generation error happens during a rebalance. Happening since v1.6.0.
When switching to a different leader a consumer could wait 500 ms (
fetch.error.backoff.ms) before starting to fetch again. The fetch backoff wasn’t reset when joining the new broker. Solved by resetting it, given it’s not needed to backoff the first fetch on a different node. This way faster leader switches are possible. Happens since 1.x.The function
rd_kafka_offsets_for_timesrefreshes leader information if the error requires it, allowing it to succeed on subsequent manual retries. Similar to the fix done in 2.3.0 inrd_kafka_query_watermark_offsets. Additionally, the partition current leader epoch is taken from metadata cache instead of from passed partitions. Happens since 1.x.When consumer is closed before destroying the client, the operations queue isn’t purged anymore as it contains operations unrelated to the consumer group. Happens since 1.x.
When making multiple changes to the consumer subscription in a short time, no unknown topic error is returned for topics that are in the new subscription but weren’t in previous one. This was due to the metadata request relative to previous subscription. Happens since 1.x.
Remove a one second wait after a partition fetch is restarted following a leader change and offset validation. This is done by resetting the fetch error backoff and waking up the delegated broker if present. Happens since 2.1.0.
2.8.0
2.8.0 is a maintenance release:
Socket options are now all set before connection.
Client certificate chain is now sent when using
ssl.certificate.pemorssl_certificateorssl.keystore.locationAvoid sending client certificates whose chain doesn’t match with broker trusted root certificates.
Fixes to allow to migrate partitions to leaders with same leader epoch, or NULL leader epoch.
Support versions of OpenSSL without the ENGINE component.
Fixes
General fixes
Socket options are now all set before connection, as documentation says it’s needed for socket buffers to take effect, even if in some cases they could have effect even after connection. Happening since v0.9.0.
Client certificate chain is now sent when using
ssl.certificate.pemorssl_certificateorssl.keystore.location. Without that, broker must explicitly add any intermediate certification authority certificate to its truststore to be able to accept client certificate. Happens since: 1.x.
Consumer fixes
Fix to allow to migrate partitions to leaders with NULL leader epoch. NULL leader epoch can happen during a cluster roll with an upgrade to a version supporting KIP-320. Happening since v2.1.0.
Fix to allow to migrate partitions to leaders with same leader epoch. Same leader epoch can happen when partition is temporarily migrated to the internal broker, or if broker implementation never bumps it, as it’s not needed to validate the offsets. Happening since v2.4.0.
2.6.1
2.6.1 is a maintenance release:
Fix for a fetch regression when connecting to Apache Kafka® versions earlier than 2.7.
Fix for an infinite loop happening with cooperative-sticky assignor under some particular conditions.
Fix for retrieving offset commit metadata when it contains zeros and configured with
strndupFix for a loop of
ListOffsetrequests, happening in a fetch-from-follower scenario, if such request is made to the follower.Fix to remove fetch queue messages that blocked the destroy of rdkafka instances.
Upgrade Linux dependencies: OpenSSL 3.0.15, CURL 8.10.1.
Upgrade Windows dependencies: MSVC runtime to 14.40.338160.0, zstd 1.5.6, zlib 1.3.1, OpenSSL 3.3.2, CURL 8.10.1.
SASL and SCRAM authentication fix: avoid concatenating client side nonce once more, as it’s already prepended in server sent nonce.
Allow retrying for status code 429 (‘Too Many Requests’) in HTTP requests for OAUTHBEARER OIDC.
Fixes
General fixes
SASL and SCRAM authentication fix: avoid concatenating client side nonce once more, as it’s already prepended in server sent nonce. librdkafka was incorrectly concatenating the client side nonce again, leading to this fix being made on AK side, released with 3.8.1, with
endsWithinstead ofequals. Happening since v0.0.99.
Consumer fixes
Fix for a fetch regression when connecting to Apache Kafka® versions earlier than 2.7, causing fetches to fail. Happening since v2.6.0.
A consumer configured with the
cooperative-stickypartition assignment strategy could get stuck in an infinite loop, with corresponding spike of main thread CPU usage. That happened with some particular orders of members and potential assignable partitions. Solved by removing the infinite loop cause. Happening since: 1.6.0.When retrieving offset metadata, if the binary value contained zeros and librdkafka was configured with
strndup, part of the buffer after first zero contained uninitialized data instead of rest of metadata. Solved by avoiding to usestrndupfor copying metadata. Happening since: 0.9.0.When an out of range on a follower caused an offset reset, the corresponding ListOffsets request is made to the follower, causing a repeated “Not leader for partition” error. Fixed by sending the request always to the leader. Happening since 1.5.0 (tested version) or previous ones.
Issues: Fix to remove fetch queue messages that blocked the destroy of rdkafka instances. Circular dependencies from a partition fetch queue message to the same partition blocked the destroy of an instance, that happened in case the partition was removed from the cluster while it was being consumed. Solved by purging internal partition queue, after being stopped and removed, to allow reference count to reach zero and trigger a destroy. Happening since 2.0.2.
2.6.0
2.6.0 is a feature release:
KIP-460 Admin Leader Election RPC.
[KIP-714] Complete consumer metrics support.
[KIP-714] Produce latency average and maximum metrics support for parity with Java client.
KIP-848 ListConsumerGroups Admin API now has an optional filter to return only groups of given types.
Added transactional ID resource type for ACL operations.
Fix for permanent fetch errors when using a newer fetch RPC version with an older inter broker protocol.
Fixes
Consumer fixes
Fix for permanent fetch errors when brokers support a fetch RPC version greater than 12 but cluster is configured to use an inter broker protocol that is less than 2.8. In this case returned topic IDs are zero valued and fetch has to fall back to version 12, using topic names. Happening since v2.5.0.
2.5.3
2.5.3 is a feature release.
Fix an assert being triggered during push telemetry call when no metrics matched on the client side.
Fixes
Telemetry fixes
Issue: #4833 Fix a regression introduced with KIP-714 support in which an assert is triggered during PushTelemetry call. This happens when no metric is matched on the client side among those requested by broker subscription. Happening since 2.5.0.
2.5.0
Warning
This version has introduced a regression in which an assert is triggered during PushTelemetry call. This happens when no metric is matched on the client side among those requested by broker subscription.
You won’t face any problem if:
Broker doesn’t support KIP-714.
KIP-714 feature is disabled on the broker side.
KIP-714 feature is disabled on the client side. This is enabled by default. Set configuration
enable.metrics.pushtofalse.If KIP-714 is enabled on the broker side and there is no subscription configured there.
If KIP-714 is enabled on the broker side with subscriptions that match the KIP-714 metrics defined on the client.
Having said this, we strongly recommend using v2.5.3 and above to not face this regression at all.
2.5.0 is a feature release.
KIP-951 Leader discovery optimizations for the client.
Fix segfault when using long client ID because of erased segment when using FlexVer.
Fix for an idempotent producer error, with a message batch not reconstructed identically when retried.
Removed support for CentOS 6 and CentOS 7.
KIP-714 Client metrics and observability.
Upgrade considerations
CentOS 6 and CentOS 7 support was removed as they reached EOL and security patches aren’t publicly available anymore. ABI compatibility from CentOS 8 on is maintained through pypa/manylinux, AlmaLinux based. See also Confluent supported OSs page
Enhancements
Update bundled lz4 (used when
./configure --disable-lz4-ext) to v1.9.4, which contains bugfixes and performance improvements.KIP-951 With this KIP leader updates are received through produce and fetch responses in case of errors corresponding to leader changes and a partition migration happens before refreshing the metadata cache.
Fixes
General fixes
Issues: confluentinc/confluent-kafka-dotnet#2084 Fix segfault when a segment is erased and more data is written to the buffer. Happens since 1.x when a portion of the buffer (segment) is erased for FlexVer or compression. More likely to happen since 2.1.0, because of the upgrades to FlexVer, with certain string sizes like a long client ID.
Idempotent producer fixes
Fix for an idempotent producer error, with a message batch not reconstructed identically when retried. Caused the error message “Local: Inconsistent state: Unable to reconstruct MessageSet”. Happening on large batches. Solved by using the same backoff baseline for all messages in the batch. Happens since 2.2.0.
2.4.0
2.4.0 is a feature release:
KIP-848: The new version of the Consumer Rebalance Protocol. Early Access: This should be used only for evaluation and must not be used in production. Features and contract of this KIP might change in future.
KIP-467: Augment ProduceResponse error messaging for specific culprit records.
KIP-516 Continue partial implementation by adding a metadata cache by topic ID and updating the topic ID corresponding to the partition name.
Upgrade OpenSSL to v3.0.12 (while building from source) with various security fixes, check the release notes.
Integration tests can be started in KRaft mode and run against any GitHub Kafka branch other than the released versions.
Fix pipeline inclusion of static binaries.
Fix to main loop timeout calculation leading to a tight loop for a max period of one ms.
Fixed a bug causing duplicate message consumption from a stale fetch start offset in some particular cases.
Fix to metadata cache expiration on full metadata refresh.
Fix for a wrong error returned on full metadata refresh before joining a consumer group.
Fix to metadata refresh interruption.
Fix for an undesired partition migration with stale leader epoch.
Fix hang in cooperative consumer mode if an assignment is processed while closing the consumer.
Upgrade OpenSSL to v3.0.13 (while building from source) with various security fixes, check the release notes
Upgrade zstd to v1.5.6, zlib to v1.3.1, and curl to v8.8.0.
Upgrade considerations
With KIP 467, INVALID_MSG (Java: CorruptRecordExpection) will be retried automatically. INVALID_RECORD (Java: InvalidRecordException) instead is not retriable and will be set only to the records that caused the error. Rest of records in the batch will fail with the new error code _INVALID_DIFFERENT_RECORD (Java: KafkaException) and can be retried manually, depending on the application logic.
Early access
KIP-848: The next version of the Consumer Rebalance Protocol
With this new protocol the role of the Group Leader (a member) is removed and the assignment is calculated by the Group Coordinator (a broker) and sent to each member through heartbeats.
The feature is still not production ready. It’s possible to try it in a non-production environment.
Fixes
General fixes
Issues: confluentinc/confluent-kafka-go#981. In librdkafka release pipeline a static build containing libsasl2 could be chosen instead of the alternative one without it. That caused the libsasl2 dependency to be required in confluent-kafka-go v2.1.0-linux-musl-arm64 and v2.3.0-linux-musl-arm64. Solved by correctly excluding the binary configured with that library, when targeting a static build. Happening since v2.0.2, with specified platforms, when using static binaries.
When the main thread loop was awakened less than one ms before the expiration of a timeout, it was serving with a zero timeout, leading to increased CPU usage until the timeout was reached. Happening since 1.x.
Metadata cache was cleared on full metadata refresh, leading to unnecessary refreshes and occasional
UNKNOWN_TOPIC_OR_PARTerrors. Solved by updating cache for existing or hinted entries instead of clearing them. Happening since 2.1.0.A metadata call before member joins consumer group, could lead to an
UNKNOWN_TOPIC_OR_PARTerror. Solved by updating the consumer group following a metadata refresh only in safe states. Happening since 2.1.0.Metadata refreshes without partition leader change could lead to a loop of metadata calls at fixed intervals. Solved by stopping metadata refresh when all existing metadata is non-stale. Happening since 2.3.0.
A partition migration could happen, using stale metadata, when the partition was undergoing a validation and being retried because of an error. Solved by doing a partition migration only with a non-stale leader epoch. Happening since 2.1.0.
Consumer fixes
In case of subscription change with a consumer using the cooperative assignor it could resume fetching from a previous position. That could also happen if resuming a partition that wasn’t paused. Fixed by ensuring that a resume operation is completely a no-op when the partition isn’t paused. Happening since 1.x.
While using the cooperative assignor, given an assignment is received while closing the consumer it’s possible that it gets stuck in state WAIT_ASSIGN_CALL, while the method is converted to a full unassign. Solved by changing state from WAIT_ASSIGN_CALL to WAIT_UNASSIGN_CALL while doing this conversion. Happening since 1.x.
2.3.0
2.3.0 is a feature release:
KIP-516 Partial support of topic identifiers. Topic identifiers in metadata response available through the new
rd_kafka_DescribeTopicsfunction.KIP-117 Add support for AdminAPI
DescribeCluster()andDescribeTopics().KIP-430: Return authorized operations in
describeresponses.KIP-580: Added exponential backoff mechanism for retriable requests with
retry.backoff.msas minimum backoff andretry.backoff.max.msas the maximum backoff, with 20% jitter.KIP-396: completed the implementation with the addition of ListOffsets.
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.
Fix a segmentation fault when closing a consumer using the cooperative-sticky assignor before the first assignment.
Fix for insufficient buffer allocation when allocating rack information.
Fix for infinite loop of OffsetForLeaderEpoch requests on quick leader changes.
Fix to add leader epoch to control messages, to make sure they’re stored for committing even without a subsequent fetch message.
Fix for stored offsets not being committed if they lacked the leader epoch.
While building from source, upgrade OpenSSL to v3.0.11 with various security fixes. Check the release notes
Fix to ensure permanent errors during offset validation continue being retried and don’t cause an offset reset.
Fix to ensure max.poll.interval.ms is reset when rd_kafka_poll is called with consume_cb.
Fix for idempotent producer fatal errors, triggered after a possibly persisted message state.
Fix
rd_kafka_query_watermark_offsetscontinuing beyond timeout expiry.Fix
rd_kafka_query_watermark_offsetsnot refreshing the partition leader after a leader change and subsequentNOT_LEADER_OR_FOLLOWERerror.
Upgrade considerations
retry.backoff.ms: If it is set greater thanretry.backoff.max.mswhich 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.msis 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.mswhich 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.msis 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 32-bit architectures. Solved by aligning all allocations to the maximum allowed word size.
The timeout for
rd_kafka_query_watermark_offsetswas not enforced after making the necessary ListOffsets requests, and thus, it never timed out in case of broker or network issues. Fixed by setting an absolute timeout.
Idempotent producer fixes
After a potentially persisted error, such as a disconnection or timeout, the producer incorrectly incremented the next expected sequence number. This caused a fatal
OUT_OF_ORDER_SEQUENCE_NUMBERerror when the failed message wasn’t actually persisted and a subsequent message was sent.This issue showed up as “sequence desynchronization” after a single potentially persisted error or “rewound sequence number” after multiple errored messages.
Fixed by treating potentially persisted messages as not persisted. The producer now expects a
DUPLICATE_SEQUENCE_NUMBERerror if the message was actually persisted orNO_ERRORif the message wasn’t. In both cases, the message is correctly marked as delivered.
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.
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.
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.
If using
rd_kafka_poll_set_consumer, along with a consume callback, and then callingrd_kafka_pollto service the callbacks, would not resetmax.poll.interval.ms.This was because we were only checkingrk_repfor 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.mscheck intord_kafka_q_serve.After a leader change a
rd_kafka_query_watermark_offsetscall 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 aftertopic.metadata.refresh.interval.ms.
2.2.0
2.2.0 is a feature release:
Fix a segmentation fault when subscribing to non-existent topics and using the consume batch functions.
Store offset commit metadata in
rd_kafka_offsets_store.Fix a bug that happens when skipping tags, causing buffer underflow in metadata API.