kirktrue commented on code in PR #13425:
URL: https://github.com/apache/kafka/pull/13425#discussion_r1146976691


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractFetch.java:
##########
@@ -0,0 +1,791 @@
+/*
+ * 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.ClientResponse;
+import org.apache.kafka.clients.FetchSessionHandler;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.OffsetOutOfRangeException;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.errors.RecordTooLargeException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.message.FetchResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.FetchRequest;
+import org.apache.kafka.common.requests.FetchResponse;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.helpers.MessageFormatter;
+
+import java.io.Closeable;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * {@code AbstractFetch} represents the basic state and logic for record 
fetching processing.
+ * @param <K> Type for the message key
+ * @param <V> Type for the message value
+ */
+public abstract class AbstractFetch<K, V> implements Closeable {
+
+    private final Logger log;
+    protected final LogContext logContext;
+    protected final ConsumerNetworkClient client;
+    protected final ConsumerMetadata metadata;
+    protected final SubscriptionState subscriptions;
+    protected final FetchConfig<K, V> fetchConfig;
+    protected final Time time;
+    protected final FetchMetricsManager metricsManager;
+
+    private final BufferSupplier decompressionBufferSupplier;
+    private final ConcurrentLinkedQueue<CompletedFetch<K, V>> completedFetches;
+    private final Map<Integer, FetchSessionHandler> sessionHandlers;
+    private final Set<Integer> nodesWithPendingFetchRequests;
+
+    private CompletedFetch<K, V> nextInLineFetch;
+
+    public AbstractFetch(final LogContext logContext,
+                         final ConsumerNetworkClient client,
+                         final ConsumerMetadata metadata,
+                         final SubscriptionState subscriptions,
+                         final FetchConfig<K, V> fetchConfig,
+                         final FetchMetricsManager metricsManager,
+                         final Time time) {
+        this.log = logContext.logger(AbstractFetch.class);
+        this.logContext = logContext;
+        this.client = client;
+        this.metadata = metadata;
+        this.subscriptions = subscriptions;
+        this.fetchConfig = fetchConfig;
+        this.decompressionBufferSupplier = BufferSupplier.create();
+        this.completedFetches = new ConcurrentLinkedQueue<>();
+        this.sessionHandlers = new HashMap<>();
+        this.nodesWithPendingFetchRequests = new HashSet<>();
+        this.metricsManager = metricsManager;
+        this.time = time;
+    }
+
+    /**
+     * Return whether we have any completed fetches pending return to the 
user. This method is thread-safe. Has
+     * visibility for testing.
+     *
+     * @return true if there are completed fetches, false otherwise
+     */
+    boolean hasCompletedFetches() {

Review Comment:
   I intentionally did my best to keep the code I moved from `Fetcher` into 
`AbstractFetch` identical. There was some minor refactoring which could 
technically introduce bugs.



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