Skip to content

Librdkafka handling of brokers with the same host:port is inconsistent with java #4212

@dpcollins-google

Description

@dpcollins-google

Description

When a Metadata request returns brokers with different IDs but the same host:port, librdkafka will rebind the state associated with the node ID with the same internal state preventing communicating with any IDs but the last in the list, while java will handle this case without issue.

How to reproduce

This is an internal project and hard to reproduce from base principles, but a unit test could easily be constructed.

I'm developing an L7 proxy which terminates TCP streams for the kafka wire protocol at a horizontally-load-balanced server, then re-exports those streams to the underlying cluster after aggregation. When presenting multiple nodes (to avoid saturating any individual stream) to the MetadataResponse that all have the same URL (of the generic TLS proxy) but different node ids, librdkafka thrashes connections and is unable to get into a stable state.

This is due to this check in librdkafka where, if it sees a node id it does not currently have state for, will forcibly update the node id for a broker with the same host:port

} else if ((rkb = rd_kafka_broker_find(rk, proto, mdb->host,

This check does not exist in java, which just replaces the nodeId -> host:port mapping when it gets a new one https://github.com/apache/kafka/blob/27548707dd8f2324d27335edc922ae88d78d86ed/clients/src/main/java/org/apache/kafka/clients/MetadataCache.java#LL136C22-L136C22

It looks like this check dates back to when Metadata request support was first added in b09ff60 . @edenhill if you're still involved in the project.

Can we remove this inconsistency with Java and with the kafka protocol?

Checklist

IMPORTANT: We will close issues where the checklist has not been completed.

Please provide the following information:

  • librdkafka version 2.0.2
  • Apache Kafka version: See above
  • librdkafka client configuration: all
  • Operating system: all
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts: N/A

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions