junrao commented on code in PR #14359:
URL: https://github.com/apache/kafka/pull/14359#discussion_r1323771865


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegate.java:
##########
@@ -63,6 +68,31 @@ public NetworkClientDelegate(
         this.unsentRequests = new ArrayDeque<>();
         this.requestTimeoutMs = 
config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
         this.retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        this.tryConnectNodes = new HashSet<>();
+    }
+
+    @Override
+    public boolean isUnavailable(Node node) {
+        return NetworkClientUtils.isUnavailable(client, node, time);
+    }
+
+    @Override
+    public void maybeThrowAuthFailure(Node node) {
+        NetworkClientUtils.maybeThrowAuthFailure(client, node);
+    }
+
+    @Override
+    public void tryConnect(Node node) {

Review Comment:
   `tryConnect` and `maybeTryConnect` seem highly coupled. Why is only the 
former part of the `NodeStatusDetector` interface? Also, it seems that we have 
used `maybeTryConnect`, but not `tryConnect` in this class.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchUtils.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.common.TopicPartition;
+
+/**
+ * {@code FetchUtils} provides a place for disparate parts of the fetch logic 
to live.
+ */
+public class FetchUtils {
+
+    /**
+     * Performs two combined actions based on the state related to the {@link 
TopicPartition}:
+     *
+     * <ol>
+     *     <li>
+     *         Invokes {@link ConsumerMetadata#requestUpdate(boolean)} to 
signal that the metadata is incorrect and
+     *         needs to be updated
+     *     </li>
+     *     <li>
+     *         Invokes {@link 
SubscriptionState#clearPreferredReadReplica(TopicPartition)} to clear out any 
read replica
+     *         information that may be present.
+     *     </li>
+     * </ol>
+     *
+     * This utility method should be invoked if the client detects (or is told 
by a node in the broker) that an
+     * attempt was made to fetch from a node that isn't the leader or 
preferred replica.
+     *
+     * @param metadata {@link ConsumerMetadata} for which to request an update
+     * @param subscriptions {@link SubscriptionState} to clear any internal 
read replica node
+     * @param topicPartition {@link TopicPartition} for which this state 
change is related
+     */
+    static void requestMetadataUpdate(final ConsumerMetadata metadata,
+                                      final SubscriptionState subscriptions,
+                                      final TopicPartition topicPartition) {
+        metadata.requestUpdate(true);

Review Comment:
   This was set to `false`. Why is it changed to `true`? I think `false` is 
correct since we don't want to reset the metadata retry count when receiving 
retriable error due to stale metadata.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NodeStatusDetector.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.KafkaClient;
+import org.apache.kafka.clients.NetworkClientUtils;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.errors.AuthenticationException;
+import org.apache.kafka.common.utils.Time;
+
+/**
+ * Use {@code NodeStatusDetector} to determine the status of a given broker 
{@link Node}. It's also
+ * possible to check for previous authentication errors if the node isn't 
available.
+ *
+ * @see ConsumerNetworkClient
+ * @see NetworkClientDelegate
+ */
+public interface NodeStatusDetector {
+
+    /**
+     * Check if the node is disconnected and unavailable for immediate 
reconnection (i.e. if it is in
+     * reconnect backoff window following the disconnect).
+     *
+     * @param node {@link Node} to check for availability
+     * @see NetworkClientUtils#isUnavailable(KafkaClient, Node, Time)
+     */
+    boolean isUnavailable(Node node);
+
+    /**
+     * Checks for an authentication error on a given node and throws the 
exception if it exists.
+     *
+     * @param node {@link Node} to check for a previous {@link 
AuthenticationException}; if found it is thrown
+     * @see NetworkClientUtils#maybeThrowAuthFailure(KafkaClient, Node)
+     */
+    void maybeThrowAuthFailure(Node node);
+
+    /**
+     * Initiate a connection if currently possible. This is only really useful 
for resetting

Review Comment:
   Hmm, the implementation in ConsumerNetworkClient initiates a connection, but 
the implementation in NetworkClientDelegate doesn't. So, is this the right 
abstraction to capture both?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.internals.IdempotentCloser;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Predicate;
+
+/**
+ * {@code FetchBuffer} buffers up the results from the broker responses as 
they are received. It is essentially a
+ * wrapper around a {@link java.util.Queue} of {@link CompletedFetch}.
+ *
+ * <p/>
+ *
+ * <em>Note</em>: this class is not thread-safe and is intended to only be 
used from a single thread.
+ */
+public class FetchBuffer implements Closeable {
+
+    private final Logger log;
+    private final ConcurrentLinkedQueue<CompletedFetch> completedFetches;
+    private final IdempotentCloser idempotentCloser = new IdempotentCloser();
+
+    private CompletedFetch nextInLineFetch;
+
+    public FetchBuffer(final LogContext logContext) {
+        this.log = logContext.logger(FetchBuffer.class);
+        this.completedFetches = new ConcurrentLinkedQueue<>();
+    }
+
+    /**
+     * Returns {@code true} if there are no completed fetches pending to 
return to the user.
+     *
+     * @return {@code true} if the buffer is empty, {@code false} otherwise
+     */
+    boolean isEmpty() {
+        return completedFetches.isEmpty();
+    }
+
+    /**
+     * Return whether we have any completed fetches pending return to the 
user. This method is thread-safe. Has
+     * visibility for testing.
+     *
+     * @return {@code true} if there are completed fetches that match the 
{@link Predicate}, {@code false} otherwise
+     */
+    boolean hasCompletedFetches(Predicate<CompletedFetch> predicate) {
+        return completedFetches.stream().anyMatch(predicate);
+    }
+
+    void add(CompletedFetch completedFetch) {
+        completedFetches.add(completedFetch);
+    }
+
+    void addAll(Collection<CompletedFetch> completedFetches) {
+        this.completedFetches.addAll(completedFetches);
+    }
+
+    CompletedFetch nextInLineFetch() {
+        return nextInLineFetch;
+    }
+
+    void setNextInLineFetch(CompletedFetch completedFetch) {
+        this.nextInLineFetch = completedFetch;
+    }
+
+    CompletedFetch peek() {
+        return completedFetches.peek();
+    }
+
+    CompletedFetch poll() {
+        return completedFetches.poll();
+    }
+
+    /**
+     * Updates the buffer to retain only the fetch data that corresponds to 
the given partitions. Any previously
+     * {@link CompletedFetch fetched data} is removed if its partition is not 
in the given set of partitions.
+     *
+     * @param partitions {@link Set} of {@link TopicPartition}s for which any 
buffered data should be kept
+     */
+    void retainAll(final Set<TopicPartition> partitions) {
+        completedFetches.removeIf(cf -> maybeDrain(partitions, cf));
+
+        if (maybeDrain(partitions, nextInLineFetch))
+            nextInLineFetch = null;
+    }
+
+    boolean maybeDrain(final Set<TopicPartition> partitions, final 
CompletedFetch completedFetch) {
+        if (completedFetch != null && 
!partitions.contains(completedFetch.partition)) {
+            log.debug("Removing {} from buffered fetch data as it is not in 
the set of partitions to retain ({})", completedFetch.partition, partitions);
+            completedFetch.drain();
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    Set<TopicPartition> partitions() {

Review Comment:
   Perhaps bufferedPartitions?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchBuffer.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.common.TopicPartition;
+import org.apache.kafka.common.internals.IdempotentCloser;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.io.Closeable;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Predicate;
+
+/**
+ * {@code FetchBuffer} buffers up the results from the broker responses as 
they are received. It is essentially a
+ * wrapper around a {@link java.util.Queue} of {@link CompletedFetch}.
+ *
+ * <p/>
+ *
+ * <em>Note</em>: this class is not thread-safe and is intended to only be 
used from a single thread.
+ */
+public class FetchBuffer implements Closeable {
+
+    private final Logger log;
+    private final ConcurrentLinkedQueue<CompletedFetch> completedFetches;

Review Comment:
   Has this been addressed?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java:
##########
@@ -256,138 +258,23 @@ protected FetchRequest.Builder createFetchRequest(final 
Node fetchTarget,
     }
 
     /**
-     * Return the fetched records, empty the record buffer and update the 
consumed position.
-     *
-     * </p>
+     * Return the list of <em>fetchable</em> partitions, which are the set of 
partitions to which we are subscribed,
+     * but <em>excluding</em> any partitions for which we still have buffered 
data. The idea is that since the user
+     * has yet to process the data for the partition that has already been 
fetch, we should not go send for more data

Review Comment:
   been fetch => been fetched



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##########
@@ -91,23 +105,70 @@ public void onFailure(RuntimeException e) {
                 }
             };
 
-            final RequestFuture<ClientResponse> future = 
client.send(fetchTarget, request);
+            final RequestFuture<ClientResponse> future = 
nodeStatusDetector.send(fetchTarget, request);
             future.addListener(listener);
         }
 
         return fetchRequestMap.size();
     }
 
-    public void close(final Timer timer) {
-        if (!isClosed.compareAndSet(false, true)) {
-            log.info("Fetcher {} is already closed.", this);
-            return;
+    public Fetch<K, V> collectFetch() {
+        return fetchCollector.collectFetch(fetchBuffer);
+    }
+
+    protected void maybeCloseFetchSessions(final Timer timer) {

Review Comment:
   Hmm, I don't quite understand this refactoring now. This method only depends 
on `nodeStatusDetector`, which is in `AbstractFetch`. Why is this method moved 
here instead of remaining in `AbstractFetch`?



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