Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

producer continually refreshing client metadata after broker disconnection #4577

Closed
5 tasks done
jpiper opened this issue Jan 9, 2024 · 8 comments
Closed
5 tasks done

Comments

@jpiper
Copy link

jpiper commented Jan 9, 2024

Description

We are producing into a topic with 84 partitions over 8 brokers. We are seeing the issue where a recoverable error condition (difficult to reproduce, but an example would be transport failure causing a broker connection to fail) will trigger the underlying librdkafka library to continually refresh the metadata for this topic at the retry.backoff.max.ms, even though the metadata request is successful and all partitions have leaders.

e.g. here is the error condition happening which is triggered by a new connection to a broker being broken just ~8s after entering UP state

%7|1704816898.865|CONNECT|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Connected to ipv4#10.255.253.3:443
%7|1704816898.865|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state CONNECT -> SSL_HANDSHAKE
%7|1704816898.865|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 6 message(s) queued but broker not up
%7|1704816898.886|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 6 message(s) queued but broker not up
%7|1704816898.888|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 6 message(s) queued but broker not up
%7|1704816898.907|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 6 message(s) queued but broker not up
%7|1704816898.907|CONNECTED|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Connected (#13)
%7|1704816898.907|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state SSL_HANDSHAKE -> APIVERSION_QUERY
%7|1704816898.907|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 6 message(s) queued but broker not up
%7|1704816898.926|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state APIVERSION_QUERY -> UP
%7|1704816898.926|METADATA|producer.host#producer-2| [thrd:ssl://broker-6:4]: Skipping metadata refresh of 1 topic(s): connected: already being requested
%7|1704816906.827|FAIL|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Disconnected (after 7900ms in state UP) (_TRANSPORT)
%6|1704816906.827|FAIL|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Disconnected (after 7900ms in state UP)
%7|1704816906.827|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state UP -> DOWN
%7|1704816906.827|REQERR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: ProduceRequest failed: Local: Broker transport failure: explicit actions Refresh,MsgPossiblyPersisted
%7|1704816906.827|BROKERUA|producer.host#producer-2| [thrd:ssl://broker-6:4]: produce_topic [51]: broker unavailable: produce: Local: Broker transport failure
%7|1704816906.827|FASTQUERY|producer.host#producer-2| [thrd:ssl://broker-6:4]: Starting fast leader query
%7|1704816906.827|METADATA|producer.host#producer-2| [thrd:ssl://broker-6:4]: Requesting metadata for 1/1 topics: broker down
%7|1704816906.827|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query
%7|1704816906.827|METADATA|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-5:443/5: Request metadata for 1 topic(s): broker down
%7|1704816906.827|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: Request metadata for 1 topic(s): partition leader query
%7|1704816906.827|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state DOWN -> INIT
%7|1704816906.827|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 4 message(s) queued but broker not up
%7|1704816906.827|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state INIT -> TRY_CONNECT
%7|1704816906.827|CONNECT|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: broker in state TRY_CONNECT connecting
%7|1704816906.827|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state TRY_CONNECT -> CONNECT
%7|1704816906.843|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: ===== Received metadata (for 1 requested topics): broker down =====
%7|1704816906.858|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816906.859|CONNECT|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Connected to ipv4#10.255.253.3:443
%7|1704816906.859|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state CONNECT -> SSL_HANDSHAKE
%7|1704816906.859|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 4 message(s) queued but broker not up
%7|1704816906.878|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 4 message(s) queued but broker not up
%7|1704816906.880|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 4 message(s) queued but broker not up
%7|1704816906.897|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 4 message(s) queued but broker not up
%7|1704816906.897|CONNECTED|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Connected (#14)
%7|1704816906.897|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state SSL_HANDSHAKE -> APIVERSION_QUERY
%7|1704816906.897|TOPPAR|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: produce_topic [51] 4 message(s) queued but broker not up
%7|1704816906.912|STATE|producer.host#producer-2| [thrd:ssl://broker-6:4]: ssl://broker-6:443/6: Broker changed state APIVERSION_QUERY -> UP
%7|1704816906.912|METADATA|producer.host#producer-2| [thrd:ssl://broker-6:4]: Skipping metadata refresh of 1 topic(s): connected: already being requested
%7|1704816906.927|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query

and here we can see that even though every one of these metadata requests is returning successfully (I've truncated the response for readability) and all the partitions have leaders, the client is still refreshing the metadata as if something were broken. Note that during this the producer is working fine and messages are getting delivered fine, but we have noticed on our servers that the number of metadata requests from our clients is huge.

%7|1704816906.843|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816906.858|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: ===== Received metadata (for 1 requested topics): partition leader query =====
%7|1704816906.858|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816906.927|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query <-- 100ms later
%7|1704816906.943|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816907.115|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query <-- 200ms later
%7|1704816907.131|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816907.475|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query <-- 400ms later
%7|1704816907.491|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816908.227|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query <-- 800ms later
%7|1704816908.243|METADATA|producer.host#producer-2| [thrd:main]: ssl://broker-5:443/5: 1/1 requested topic(s) seen in metadata
%7|1704816909.227|METADATA|producer.host#producer-2| [thrd:main]: Requesting metadata for 1/1 topics: partition leader query <-- 1s later

It looks to me like there could be some sort of race condition around broker disconnects/reconnects and metadata refreshing?

Checklist

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

Please provide the following information:

  • librdkafka version (release number or git tag): 2.3.0
  • Apache Kafka version: 3.4.0
  • librdkafka client configuration: message.max.bytes=25165824;socket.timeout.ms=10000;socket.keepalive.enable=true;debug=metadata,broker,topic
  • Operating system: centos7
  • Provide logs (with debug=.. as necessary) from librdkafka
@jpiper
Copy link
Author

jpiper commented Jan 10, 2024

Reverting to 2.2.0 has fixed the issue so I assume this is a regression introduced in 2.3.0

@richardartoul
Copy link

I can confirm this as well, combined with the very low value of retry.backoff.max.ms (1s) the latest version of librdkafka can easily turn into a DDOS cannon at any moment once a retriable error is encountered.

@mensfeld
Copy link

I can confirm this as well, combined with the very low value of retry.backoff.max.ms (1s) the latest version of librdkafka can easily turn into a DDOS cannon at any moment once a retriable error is encountered.

Wouldn't increasing retry.backoff.max.ms mitigate this?

@jpiper
Copy link
Author

jpiper commented Jan 29, 2024

I can confirm this as well, combined with the very low value of retry.backoff.max.ms (1s) the latest version of librdkafka can easily turn into a DDOS cannon at any moment once a retriable error is encountered.

Wouldn't increasing retry.backoff.max.ms mitigate this?

ehh... if by mitigate you mean "reduce the impact of the bug on the kafka cluster" - it doesn't actually solve the issue at hand.

@richardartoul
Copy link

Yeah that's a viable work-around in the interim, but this behavior is clearly a bug and there is a lot of software out there that isn't gonna increase that value. It's really common for people to have thousands of Kafka clients/producers and one day they're going to upgrade and everything will work fine until one of their connections has an intermittent error in the middle of the night and their cluster gets melted :/

@mensfeld
Copy link

mensfeld commented Jan 29, 2024

@jpiper sorry, this is what I meant. I am a maintainer of the ruby bindings and I usually do my best to ship intermediate fixes when stuff like this occurs. This is why I asked. I may consider changing the defaults for my lib until patch is released.

// edit - I see default for librdkafka is 1 second (1000ms) so as long as not altered, it should be ok)

@jpiper
Copy link
Author

jpiper commented Jan 29, 2024

... until one of their connections has an intermittent error in the middle of the night and their cluster gets melted :/

I feel seen

@emasab
Copy link
Collaborator

emasab commented Feb 27, 2024

There is a problem in v2.3.0 when metadata is refreshed without leader changes that avoids that fast metadata refresh is stopped. We'll include this fix in next release.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

4 participants