hachikuji commented on code in PR #12862:
URL: https://github.com/apache/kafka/pull/12862#discussion_r1038423837


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.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.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.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+
+/**
+ * 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 boolean wakeup = false;
+    private final Queue<UnsentRequest> unsentRequests;
+
+    public NetworkClientDelegate(
+            final Time time,
+            final LogContext logContext,
+            final KafkaClient client) {
+        this.time = time;
+        this.client = client;
+        this.log = logContext.logger(getClass());
+        this.unsentRequests = new ArrayDeque<>();
+    }
+
+    public List<ClientResponse> poll(Timer timer, boolean disableWakeup) {
+        client.wakeup();
+        if (!disableWakeup) {
+            // trigger wakeups after checking for disconnects so that the 
callbacks will be ready
+            // to be fired on the next call to poll()
+            maybeTriggerWakeup();
+        }
+
+        trySend();
+        return this.client.poll(timer.timeoutMs(), time.milliseconds());
+    }
+
+    private void trySend() {
+        while (unsentRequests.size() > 0) {
+            UnsentRequest unsent = unsentRequests.poll();
+            if (unsent.timer.isExpired()) {
+                // TODO: expired request should be marked
+                unsent.callback.ifPresent(c -> c.onFailure(new 
TimeoutException(
+                        "Failed to send request after " + 
unsent.timer.timeoutMs() + " " + "ms.")));
+                continue;
+            }
+
+            if (!doSend(unsent)) {
+                log.debug("No broker available to send the request: {}", 
unsent);
+                unsent.callback.ifPresent(v -> v.onFailure(
+                        new IllegalThreadStateException("No node available in 
the kafka cluster to send the request")));
+            }
+        }
+    }
+
+    static boolean isReady(KafkaClient client, Node node, long currentTime) {
+        client.poll(0, currentTime);
+        return client.isReady(node, currentTime);
+    }
+
+    public boolean doSend(UnsentRequest r) {
+        long now = time.milliseconds();
+        Node node = r.node.orElse(client.leastLoadedNode(now));
+        if (node == null) {
+            return false;
+        }
+        ClientRequest request = makeClientRequest(r, node);
+        // TODO: Sounds like we need to check disconnections for each node and 
complete the request with
+        //  authentication error
+        if (isReady(client, node, now)) {
+            client.send(request, now);
+        }
+        return true;
+    }
+
+    private ClientRequest makeClientRequest(UnsentRequest unsent, Node node) {
+        return client.newClientRequest(
+                node.idString(),
+                unsent.abstractBuilder,
+                time.milliseconds(),
+                true,
+                // TODO: Determine if we want the actual request timeout here 
to be requestTimeoutMs - timeInUnsentQueue
+                (int) unsent.timer.remainingMs(),
+                unsent.callback.orElse(new 
DefaultRequestFutureCompletionHandler()));
+    }
+
+    public void maybeTriggerWakeup() {
+        if (this.wakeup) {
+            this.wakeup = false;
+            throw new WakeupException();
+        }
+    }
+
+    public void wakeup() {
+        this.wakeup = true;
+        this.client.wakeup();
+    }
+
+    public Node leastLoadedNode() {
+        return this.client.leastLoadedNode(time.milliseconds());
+    }
+
+    public void add(UnsentRequest r) {
+        unsentRequests.add(r);
+    }
+
+    public void ready(Node node) {
+        client.ready(node, time.milliseconds());
+    }
+
+    /**
+     * Check if the code is disconnected and unavailable for immediate 
reconnection (i.e. if it is in
+     * reconnect backoff window following the disconnect).
+     */
+    public boolean nodeUnavailable(Node node) {
+        return client.connectionFailed(node) && client.connectionDelay(node, 
time.milliseconds()) > 0;
+    }
+
+    public void tryDisconnect(Optional<Node> coordinator) {
+        coordinator.ifPresent(node -> client.disconnect(node.idString()));
+    }
+
+    public void close() throws IOException {
+        this.client.close();
+    }
+
+    public void addAll(List<UnsentRequest> unsentRequests) {
+        unsentRequests.forEach(this::add);
+    }
+
+    public static class PollResult {
+        public final long timeMsTillNextPoll;
+        public final List<UnsentRequest> unsentRequests;
+
+        public PollResult(final long timeMsTillNextPoll, final 
List<UnsentRequest> unsentRequests) {
+            this.timeMsTillNextPoll = timeMsTillNextPoll;
+            this.unsentRequests = Collections.unmodifiableList(unsentRequests);
+        }
+    }
+
+    public static class UnsentRequest {
+        private final Optional<AbstractRequestFutureCompletionHandler> 
callback;
+        private final AbstractRequest.Builder abstractBuilder;
+        private final Optional<Node> node; // empty if random node can be 
choosen
+        private final Timer timer;
+
+        public UnsentRequest(final Timer timer,
+                             final AbstractRequest.Builder abstractBuilder,
+                             final AbstractRequestFutureCompletionHandler 
callback) {
+            this(timer, abstractBuilder, callback, null);
+        }
+
+        public UnsentRequest(final Timer timer,
+                             final AbstractRequest.Builder abstractBuilder,
+                             final AbstractRequestFutureCompletionHandler 
callback,
+                             final Node node) {
+            Objects.requireNonNull(abstractBuilder);
+            this.abstractBuilder = abstractBuilder;
+            this.node = Optional.ofNullable(node);
+            this.callback = Optional.ofNullable(callback);
+            this.timer = timer;
+        }
+
+        public static UnsentRequest makeUnsentRequest(
+                final Timer timeoutTimer,
+                final AbstractRequest.Builder<?> requestBuilder,
+                final AbstractRequestFutureCompletionHandler callback) {
+            return new UnsentRequest(
+                    timeoutTimer,
+                    requestBuilder,
+                    callback);
+        }
+
+        @Override
+        public String toString() {
+            return abstractBuilder.toString();
+        }
+    }
+
+    public static class DefaultRequestFutureCompletionHandler extends 
AbstractRequestFutureCompletionHandler {
+        @Override
+        public void handleResponse(ClientResponse r, Throwable t) {}
+    }
+
+    public abstract static class AbstractRequestFutureCompletionHandler 
implements RequestCompletionHandler {
+        private final RequestFuture<ClientResponse> future;
+        private ClientResponse response;
+        private RuntimeException e;
+
+        AbstractRequestFutureCompletionHandler() {
+            this.future = new RequestFuture<>();
+        }
+
+        public void fireCompletion() {
+            if (e != null) {
+                future.raise(e);
+            } else if (response.authenticationException() != null) {
+                future.raise(response.authenticationException());
+            } else if (response.wasDisconnected()) {

Review Comment:
   Check if the disconnect was to the current coordinator. If so, then 
`markCoordinatorUnknown()`.
   
   Alternatively (better), we can let custom callbacks call 
`markCoordinatorUnknown()`. This would mean the offset commit/join group 
handlers need a reference to the `CoordinatorManager`. We will need this anyway 
because we have to check for `NOT_COORDINATOR` errors in each request type sent 
to the coordinator.



-- 
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

Reply via email to