rajinisivaram commented on a change in pull request #9406: URL: https://github.com/apache/kafka/pull/9406#discussion_r509293443
########## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ########## @@ -444,10 +444,25 @@ private boolean maybeSendAndPollTransactionalRequest() { AbstractRequest.Builder<?> requestBuilder = nextRequestHandler.requestBuilder(); Node targetNode = null; try { - targetNode = awaitNodeReady(nextRequestHandler.coordinatorType()); - if (targetNode == null) { + FindCoordinatorRequest.CoordinatorType coordinatorType = nextRequestHandler.coordinatorType(); + targetNode = coordinatorType != null ? + transactionManager.coordinator(coordinatorType) : + client.leastLoadedNode(time.milliseconds()); + if (targetNode != null) { + if (!awaitNodeReady(targetNode, coordinatorType)) { + log.trace("Target node {} not ready within request timeout, will retry when node is ready.", targetNode); + maybeFindCoordinatorAndRetry(nextRequestHandler); + return true; + } + } else if (coordinatorType != null) { + log.trace("Coordinator not known for {}, will retry {} after finding coordinator.", coordinatorType, requestBuilder.apiKey()); maybeFindCoordinatorAndRetry(nextRequestHandler); return true; + } else { + log.trace("No nodes available to send requests, will poll and retry when until a node is ready."); + transactionManager.retry(nextRequestHandler); + client.poll(retryBackoffMs, time.milliseconds()); + return true; Review comment: @dajac Yes, I was thinking about this earlier when adding the retry. We get to this `else` path when leastLoadedNode is null. For the other cases, we would have taken one of the other paths, which retain existing behaviour (unless I missed something, again!) When least loaded node is null, the old behaviour didn't quite work because you have to poll to change that state. And you don't have anywhere to send metadata requests to. So just polling seemed to be sufficient for this case. What do you think? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org