philipnee commented on code in PR #12862: URL: https://github.com/apache/kafka/pull/12862#discussion_r1047823219
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java: ########## @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; + +/** + * A wrapper around the {@link org.apache.kafka.clients.NetworkClient} to handle poll and send operations. + */ +public class NetworkClientDelegate implements AutoCloseable { + private final KafkaClient client; + private final Time time; + private final Logger log; + private final int requestTimeoutMs; + private boolean wakeup = false; + private final Queue<UnsentRequest> unsentRequests; + private final Set<Node> activeNodes; + + public NetworkClientDelegate( + final Time time, + final ConsumerConfig config, + final LogContext logContext, + final KafkaClient client) { + this.time = time; + this.client = client; + this.log = logContext.logger(getClass()); + this.unsentRequests = new ArrayDeque<>(); + this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + this.activeNodes = new HashSet<>(); + } + + /** + * Returns the responses of the sent requests. This methods will try to send the unsent requests, poll for responses, + * and check the disconnected nodes. + * @param timeoutMs + * @return + */ + public List<ClientResponse> poll(final long timeoutMs) { + final long currentTimeMs = time.milliseconds(); + trySend(currentTimeMs); + List<ClientResponse> res = this.client.poll(timeoutMs, currentTimeMs); + checkDisconnects(); + maybeTriggerWakeup(); + return res; + } + + /** + * Iterates through the unsentRequests queue and tries to send the unsent requests. If the request doesn't have an + * assigned node, it will find the leastLoadedOne. + * Here it also stores all the nodes in the {@code activeNodes}, which will then be used to check for the disconnection. + */ + void trySend(final long currentTimeMs) { + Queue<UnsentRequest> unsentAndUnreadyRequests = new LinkedList<>(); + while (!unsentRequests.isEmpty()) { + UnsentRequest unsent = unsentRequests.poll(); + unsent.timer.update(currentTimeMs); + if (unsent.timer.isExpired()) { + unsent.callback.ifPresent(c -> c.onFailure(new TimeoutException( + "Failed to send request after " + unsent.timer.timeoutMs() + " " + "ms."))); + continue; + } + + if (!doSend(unsent, currentTimeMs, unsentAndUnreadyRequests)) { + log.debug("No broker available to send the request: {}", unsent); + unsent.callback.ifPresent(v -> v.onFailure(Errors.NETWORK_EXCEPTION.exception( + "No node available in the kafka cluster to send the request"))); + } + } + + if (!unsentAndUnreadyRequests.isEmpty()) { + // Handle the unready requests in the next event loop + unsentRequests.addAll(unsentAndUnreadyRequests); + } + } + + // Visible for testing + boolean doSend(final UnsentRequest r, + final long currentTimeMs, + final Queue<UnsentRequest> unsentAndUnreadyRequests) { + Node node = r.node.orElse(client.leastLoadedNode(currentTimeMs)); + if (node == null || nodeUnavailable(node)) { + return false; + } + ClientRequest request = makeClientRequest(r, node); + if (client.isReady(node, currentTimeMs)) { + activeNodes.add(node); + client.send(request, currentTimeMs); + } else { + // enqueue the request again if the node isn't ready yet. The request will be handled in the next iteration + // of the event loop + log.debug("Node is not ready, handle the request in the next event loop: node={}, request={}", node, r); + unsentAndUnreadyRequests.add(r); + } + return true; + } + + private void checkDisconnects() { + // Check the connection status by all the nodes that are active. Disconnect the disconnected node if it is + // unable to be connected. + Iterator<Node> iter = activeNodes.iterator(); + while (iter.hasNext()) { + Node node = iter.next(); + iter.remove(); + if (client.connectionFailed(node)) { + client.disconnect(node.idString()); + } + } + } + + private ClientRequest makeClientRequest(final UnsentRequest unsent, final Node node) { + return client.newClientRequest( + node.idString(), + unsent.abstractBuilder, + time.milliseconds(), Review Comment: We do want to timing here to be fairly accurate I think, would it be better to re-request the time in the trySend and pass it down here? -- 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. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org