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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.events;
+
+import org.apache.kafka.clients.consumer.internals.ConsumerNetworkThread;
+import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate;
+import org.apache.kafka.clients.consumer.internals.RequestManagers;
+import org.apache.kafka.common.internals.IdempotentCloser;
+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 java.io.Closeable;
+import java.time.Duration;
+import java.util.Objects;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+/**
+ * An event handler that receives {@link ApplicationEvent application events} 
from the application thread which
+ * are then readable from the {@link ApplicationEventProcessor} in the {@link 
ConsumerNetworkThread network thread}.
+ */
+public class ApplicationEventHandler implements Closeable {
+
+    private final Logger log;
+    private final BlockingQueue<ApplicationEvent> applicationEventQueue;
+    private final ConsumerNetworkThread networkThread;
+    private final IdempotentCloser closer = new IdempotentCloser();
+
+    public ApplicationEventHandler(final LogContext logContext,
+                                   final Time time,
+                                   final BlockingQueue<ApplicationEvent> 
applicationEventQueue,
+                                   final Supplier<ApplicationEventProcessor> 
applicationEventProcessorSupplier,
+                                   final Supplier<NetworkClientDelegate> 
networkClientDelegateSupplier,
+                                   final Supplier<RequestManagers> 
requestManagersSupplier) {
+        this.log = logContext.logger(ApplicationEventHandler.class);
+        this.applicationEventQueue = applicationEventQueue;
+        this.networkThread = new ConsumerNetworkThread(logContext,
+                time,
+                applicationEventProcessorSupplier,
+                networkClientDelegateSupplier,
+                requestManagersSupplier);
+        this.networkThread.start();
+    }
+
+    /**
+     * Add an {@link ApplicationEvent} to the handler and then internally 
invoke {@link #wakeup} to alert the
+     * network I/O thread that it has something to process.
+     *
+     * @param event An {@link ApplicationEvent} created by the application 
thread
+     */
+    public void add(final ApplicationEvent event) {
+        Objects.requireNonNull(event, "ApplicationEvent provided to add must 
be non-null");
+        log.trace("Enqueued event: {}", event);
+        applicationEventQueue.add(event);
+        wakeup();
+    }
+
+    /**
+     * Wakeup the {@link ConsumerNetworkThread network I/O thread} to pull the 
event from the queue.
+     */
+    public void wakeup() {

Review Comment:
   `wakeup` => `wakeupNetworkThread` ?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##########
@@ -633,56 +829,178 @@ public void assign(Collection<TopicPartition> 
partitions) {
                 throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
         }
 
-        // TODO: implementation of refactored Fetcher will be included in 
forthcoming commits.
-        // fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+        // Clear the buffered data which are not a part of newly assigned 
topics
+        final Set<TopicPartition> currentTopicPartitions = new HashSet<>();
+
+        for (TopicPartition tp : subscriptions.assignedPartitions()) {
+            if (partitions.contains(tp))
+                currentTopicPartitions.add(tp);
+        }
+
+        fetchBuffer.retainAll(currentTopicPartitions);
 
         // assignment change event will trigger autocommit if it is configured 
and the group id is specified. This is
         // to make sure offsets of topic partitions the consumer is 
unsubscribing from are committed since there will
-        // be no following rebalance
-        eventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
+        // be no following rebalance.
+        //
+        // See the ApplicationEventProcessor.process() method that handles 
this event for more detail.
+        applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(subscriptions.allConsumed(), 
time.milliseconds()));
 
         log.info("Assigned to partition(s): {}", join(partitions, ", "));
-        if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-            eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+        if (subscriptions.assignFromUser(new HashSet<>(partitions)))
+            applicationEventHandler.add(new 
NewTopicsMetadataUpdateRequestEvent());
     }
 
     @Override
-    public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) 
{
-        throw new KafkaException("method not implemented");
+    public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) 
{
+        maybeThrowInvalidGroupIdException();
+        if (pattern == null || pattern.toString().isEmpty())
+            throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+                    "null" : "empty"));
+
+        throwIfNoAssignorsConfigured();
+        log.info("Subscribed to pattern: '{}'", pattern);
+        subscriptions.subscribe(pattern, listener);
+        updatePatternSubscription(metadata.fetch());
+        metadata.requestUpdateForNewTopics();
+    }
+
+    /**
+     * TODO: remove this when we implement the KIP-848 protocol.
+     *
+     * <p>
+     * The contents of this method are shamelessly stolen from
+     * {@link ConsumerCoordinator#updatePatternSubscription(Cluster)} and are 
used here because we won't have access
+     * to a {@link ConsumerCoordinator} in this code. Perhaps it could be 
moved to a ConsumerUtils class?
+     *
+     * @param cluster Cluster from which we get the topics
+     */
+    private void updatePatternSubscription(Cluster cluster) {
+        final Set<String> topicsToSubscribe = cluster.topics().stream()
+                .filter(subscriptions::matchesSubscribedPattern)
+                .collect(Collectors.toSet());
+        if (subscriptions.subscribeFromPattern(topicsToSubscribe))
+            metadata.requestUpdateForNewTopics();
     }
 
     @Override
     public void subscribe(Pattern pattern) {
-        throw new KafkaException("method not implemented");
+        subscribe(pattern, new NoOpConsumerRebalanceListener());
     }
 
     @Override
     public void unsubscribe() {
-        throw new KafkaException("method not implemented");
+        fetchBuffer.retainAll(Collections.emptySet());
+        subscriptions.unsubscribe();
     }
 
     @Override
     @Deprecated
-    public ConsumerRecords<K, V> poll(long timeout) {
-        throw new KafkaException("method not implemented");
+    public ConsumerRecords<K, V> poll(final long timeoutMs) {
+        return poll(Duration.ofMillis(timeoutMs));
     }
 
     // Visible for testing
     WakeupTrigger wakeupTrigger() {
         return wakeupTrigger;
     }
 
-    private static <K, V> ClusterResourceListeners 
configureClusterResourceListeners(
-            final Deserializer<K> keyDeserializer,
-            final Deserializer<V> valueDeserializer,
-            final List<?>... candidateLists) {
-        ClusterResourceListeners clusterResourceListeners = new 
ClusterResourceListeners();
-        for (List<?> candidateList: candidateLists)
-            clusterResourceListeners.maybeAddAll(candidateList);
+    private Fetch<K, V> pollForFetches(Timer timer) {
+        long pollTimeout = timer.remainingMs();
+
+        // if data is available already, return it immediately
+        final Fetch<K, V> fetch = collectFetch();
+        if (!fetch.isEmpty()) {
+            return fetch;
+        }
+
+        // We do not want to be stuck blocking in poll if we are missing some 
positions
+        // since the offset lookup may be backing off after a failure
 
-        clusterResourceListeners.maybeAdd(keyDeserializer);
-        clusterResourceListeners.maybeAdd(valueDeserializer);
-        return clusterResourceListeners;
+        // NOTE: the use of cachedSubscriptionHasAllFetchPositions means we 
MUST call
+        // updateAssignmentMetadataIfNeeded before this method.
+        if (!cachedSubscriptionHasAllFetchPositions && pollTimeout > 
retryBackoffMs) {
+            pollTimeout = retryBackoffMs;
+        }
+
+        log.trace("Polling for fetches with timeout {}", pollTimeout);
+
+        Timer pollTimer = time.timer(pollTimeout);
+
+        // Wait a bit for some fetched data to arrive, as there may not be 
anything immediately available. Note the
+        // use of a shorter, dedicated "pollTimer" here which updates "timer" 
so that calling method (poll) will
+        // correctly handle the overall timeout.
+        try {
+            fetchBuffer.awaitNotEmpty(pollTimer);
+        } catch (InterruptException e) {
+            log.trace("Timeout during fetch", e);
+        } finally {
+            timer.update(pollTimer.currentTimeMs());
+        }
+
+        return collectFetch();
+    }
+
+    /**
+     * Perform the "{@link FetchCollector#collectFetch(FetchBuffer) fetch 
collection}" step by reading raw data out
+     * of the {@link #fetchBuffer}, converting it to a well-formed {@link 
CompletedFetch}, validating that it and
+     * the internal {@link SubscriptionState state} are correct, and then 
converting it all into a {@link Fetch}
+     * for returning.
+     *
+     * <p/>
+     *
+     * Note that if the {@link Fetch#isEmpty() is not empty}, this method will
+     * {@link ApplicationEventHandler#wakeup() wake up} the {@link 
ConsumerNetworkThread} before retuning. This is
+     * done as an optimization so that the <em>next round of data can be 
pre-fetched</em>.
+     */
+    private Fetch<K, V> collectFetch() {
+        final Fetch<K, V> fetch = fetchCollector.collectFetch(fetchBuffer);
+
+        if (!fetch.isEmpty()) {
+            // Notify the network thread to wake up and start the next round 
of fetching.
+            applicationEventHandler.wakeup();

Review Comment:
   This doesn't seem quite right. It's possible for a fetchRequest to return no 
data. In that case, if we don't wake up the the network thread, it may not be 
able to send the next fetch request for a long time.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManager.java:
##########
@@ -188,7 +194,14 @@ public CompletableFuture<Map<TopicPartition, 
OffsetAndTimestamp>> fetchOffsets(
      * this function (ex. {@link 
org.apache.kafka.common.errors.TopicAuthorizationException})
      */
     public void resetPositionsIfNeeded() {
-        Map<TopicPartition, Long> offsetResetTimestamps = 
offsetFetcherUtils.getOffsetResetTimestamp();
+        Map<TopicPartition, Long> offsetResetTimestamps;
+
+        try {
+            offsetResetTimestamps = 
offsetFetcherUtils.getOffsetResetTimestamp();

Review Comment:
   
[auto.offset.reset](https://kafka.apache.org/documentation/#consumerconfigs_auto.offset.reset)
 has 3 options: `latest`, `earliest`, `none`. It's possible for a user to 
choose `none`, which means "throw exception to the consumer if no previous 
offset is found for the consumer's group". But in that case, 
`offsetFetcherUtils.getOffsetResetTimestamp` just ignores that partition and 
won't reset it forever. We need to throw an exception so that the user knows.
   
   It's a bit of weird configuration since it means a consumer can't really 
consume for the very first time with this option. However, I tested it out with 
the existing consumer and it does throw an exception.
   ```
   bin/kafka-console-consumer.sh --consumer-property auto.offset.reset=none 
--bootstrap-server localhost:9092 --topic test
   [2023-10-20 15:30:07,088] ERROR Error processing message, terminating 
consumer process:  (kafka.tools.ConsoleConsumer$)
   org.apache.kafka.clients.consumer.NoOffsetForPartitionException: Undefined 
offset with no reset policy for partitions: [test-0]
        at 
org.apache.kafka.clients.consumer.internals.SubscriptionState.resetInitializingPositions(SubscriptionState.java:711)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.updateFetchPositions(KafkaConsumer.java:2459)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1243)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1198)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1178)
        at 
kafka.tools.ConsoleConsumer$ConsumerWrapper.receive(ConsoleConsumer.scala:473)
        at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:103)
        at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:77)
        at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:54)
        at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)
   ```



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