Skip to content

Conversation

@sinhashubham95
Copy link

@sinhashubham95 sinhashubham95 commented Dec 8, 2025

Add persistent broker reconnection failure detection and DNS re-resolution

Problem

When a Kafka broker node restarts or updates—particularly with AWS IAM authentication via SASL over private links in cross-account AWS MSK setups—librdkafka silently fails to reconnect without surfacing actionable errors to the application. This forces users to completely destroy and recreate consumers/producers to recover, as there's no signal indicating the persistent failure.

Root causes:

  1. Identical errors are suppressed after 30 seconds and not delivered to the application
  2. No mechanism exists to detect when reconnection attempts have been failing persistently
  3. DNS cache is not reset on authentication failures, causing stale IP addresses to be used

Solution

1. New Error Code: RD_KAFKA_RESP_ERR__BROKER_PERSISTENT_FAILURE (-136)

Signals when broker reconnection has been failing persistently, delivered via the error callback.

2. New Configuration: reconnect.failure.report.ms

Property Range Default Description
reconnect.failure.report.ms 0-86400000 60000 Time (ms) after which persistent failures are reported. Set to 0 to disable.

3. DNS Re-resolution on Auth Failures

Address cache is automatically cleared on authentication failures to force DNS re-resolution, addressing scenarios where broker IPs change after restarts.

Usage

void error_cb(rd_kafka_t *rk, int err, const char *reason, void *opaque) {
    if (err == RD_KAFKA_RESP_ERR__BROKER_PERSISTENT_FAILURE) {
        // Broker unreachable for > reconnect.failure.report.ms
        // Take action: alert, recreate client, fail fast, etc.
        fprintf(stderr, "CRITICAL: %s\n", reason);
    }
}

rd_kafka_conf_set(conf, "reconnect.failure.report.ms", "30000", NULL, 0);
rd_kafka_conf_set_error_cb(conf, error_cb);

Behavior

  • Each broker reports failures independently (up to N events for N bootstrap servers)
  • Error is reported once per broker per failure streak; resets on successful reconnection
  • DNS re-resolution is automatic on authentication failures
  • Setting reconnect.failure.report.ms=0 disables the feature
  • Existing ALL_BROKERS_DOWN behavior is unchanged

Files Changed

  • src/rdkafka.h - New error code
  • src/rdkafka.c - Error description
  • src/rdkafka_conf.h / src/rdkafka_conf.c - New configuration option
  • src/rdkafka_broker.h / src/rdkafka_broker.c - Failure tracking and DNS re-resolution logic
  • CONFIGURATION.md - Documentation
  • tests/0154-persistent_broker_failure.c - Integration tests

Testing

  • Unit tests for failure tracking and configuration validation
  • Integration tests for persistent failure detection and disabled mode
  • All tests pass

Breaking Changes

None. Additive change with backward-compatible defaults.

…n on authentication errors

This commit introduces a mechanism to track persistent broker reconnection failures and report them to the application if a specified threshold is reached. It also forces DNS re-resolution on authentication failures to handle scenarios where broker IPs may change after restarts. Additionally, new configuration options and error codes are added to support these features.
…istent failure tracking

This commit introduces the `reconnect.failure.report.ms` configuration option, which allows applications to receive notifications of persistent broker reconnection failures after a specified duration. It includes unit tests to validate the functionality of persistent failure tracking, ensuring correct behavior for failure detection, reporting, and configuration validation. Additionally, a new integration test is added to verify the error handling when brokers are unreachable for extended periods.
@sinhashubham95 sinhashubham95 requested a review from a team as a code owner December 8, 2025 04:04
@confluent-cla-assistant
Copy link

confluent-cla-assistant bot commented Dec 8, 2025

🎉 All Contributor License Agreements have been signed. Ready to merge.
✅ sinhashubham95
Please push an empty commit if you would like to re-run the checks to verify CLA status for all contributors.

@sinhashubham95 sinhashubham95 changed the title Fix: Broker Reconnection Persistent Failure Event Handling Fix: Persistent broker reconnection failure detection and DNS re-resolution Dec 18, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant