Skip to content

Conversation

@chirag-wadhwa5
Copy link
Collaborator

@chirag-wadhwa5 chirag-wadhwa5 commented Nov 26, 2025

Currently, if a ListOffsets request fails in
NetworkPartitionMetadataClient for any reason, the corresponding future
is completed then and there, without any retries. But the NetworkClient
and InterbrokerSendThread are loaded lazily in the
NetworkPartitionMetadataClient on the arrival of the first request. But
when the first request comes, it is immediately enqueued in the
NetworkClient, before the connection could be established, thereby
always failing the first request. As a solution to that, this PR
introduces a retry mechanism with an upper limit on the retry attempts,
as well as exponential backoff between succesive retries.

Reviewers: Apoorv Mittal [email protected], Andrew Schofield
[email protected], Sushant Mahajan [email protected]

@github-actions github-actions bot added triage PRs from the community core Kafka Broker clients group-coordinator labels Nov 26, 2025
@apoorvmittal10 apoorvmittal10 added ci-approved and removed triage PRs from the community labels Nov 26, 2025
Copy link
Contributor

@apoorvmittal10 apoorvmittal10 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the changes, some doubts.

/**
* Manages retry attempts and exponential backoff for requests.
*/
public class ExponentialBackoffManager {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we really require a new class and can't use some existing mechanism?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The PersisterStateManager already had a subclass called BackOffManager, and I requred a similar thing. Instead of having 2 subclasses at different places, I think having a utility class is better. It could be used in future elsewhere as well.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AndrewJSchofield
Copy link
Member

@chirag-wadhwa5 Checkstyle failures to fix please.

@AndrewJSchofield AndrewJSchofield added the KIP-932 Queues for Kafka label Nov 26, 2025
/**
* Manages retry attempts and exponential backoff for requests.
*/
public class ExponentialBackoffManager {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since this is a public class now, please add a few unit tests.

shouldRetry = true;
} else if (clientResponse.wasTimedOut()) {
log.error("Response for ListOffsets for TopicPartitions: {} timed out - {}.", partitionFutures.keySet(), clientResponse);
log.warn("Response for ListOffsets for TopicPartitions: {} timed out - {}.", partitionFutures.keySet(), clientResponse);
Copy link
Collaborator

@smjn smjn Nov 27, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe remove logging from retriable requests - log if attempts exhausted only.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review. Maybe I'll change it to debug.

@smjn
Copy link
Collaborator

smjn commented Nov 27, 2025

@chirag-wadhwa5 There are thread leaks from the new reaper org.opentest4j.AssertionFailedError: Thread leak detected: network-partition-metadata-client-reaper ==>
Please see it is being closed properly in BrokerServer

@AndrewJSchofield
Copy link
Member

@chirag-wadhwa5 Please can you merge latest changes from trunk.

Copy link
Contributor

@apoorvmittal10 apoorvmittal10 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Approving given we will improve it in near future.

@AndrewJSchofield AndrewJSchofield merged commit ed3af72 into apache:trunk Nov 30, 2025
23 checks passed
AndrewJSchofield pushed a commit that referenced this pull request Nov 30, 2025
…adataClient (#21001)

Currently, if a ListOffsets request fails in
NetworkPartitionMetadataClient for any reason, the corresponding future
is completed then and there, without any retries. But the NetworkClient
and InterbrokerSendThread are loaded lazily in the
NetworkPartitionMetadataClient on the arrival of the first request. But
when the first request comes, it is immediately enqueued in the
NetworkClient, before the connection could be established, thereby
always failing the first request. As a solution to that, this PR
introduces a retry mechanism with an upper limit on the retry attempts,
as well as exponential backoff between succesive retries.

Reviewers: Apoorv Mittal <[email protected]>, Andrew Schofield
 <[email protected]>, Sushant Mahajan <[email protected]>
@AndrewJSchofield
Copy link
Member

Cherry-picked to 4.2.

TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Dec 1, 2025
…adataClient (apache#21001)

Currently, if a ListOffsets request fails in
NetworkPartitionMetadataClient for any reason, the corresponding future
is completed then and there, without any retries. But the NetworkClient
and InterbrokerSendThread are loaded lazily in the
NetworkPartitionMetadataClient on the arrival of the first request. But
when the first request comes, it is immediately enqueued in the
NetworkClient, before the connection could be established, thereby
always failing the first request. As a solution to that, this PR
introduces a retry mechanism with an upper limit on the retry attempts,
as well as exponential backoff between succesive retries.

Reviewers: Apoorv Mittal <[email protected]>, Andrew Schofield
 <[email protected]>, Sushant Mahajan <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants