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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##########
@@ -621,56 +825,174 @@ 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()));
+        // make sure the offsets of topic partitions the consumer is 
unsubscribing from
+        // are committed since there will be no following rebalance
+        applicationEventHandler.add(new 
AssignmentChangeApplicationEvent(this.subscriptions.allConsumed(), 
time.milliseconds()));
 
         log.info("Assigned to partition(s): {}", join(partitions, ", "));
         if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
-            eventHandler.add(new NewTopicsMetadataUpdateRequestEvent());
+            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);
+        this.subscriptions.subscribe(pattern, listener);
+        this.updatePatternSubscription(metadata.fetch());
+        this.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());
+        this.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);
+    /**
+     * Send the requests for fetch data to the background thread and set up to 
collect the results in
+     * {@link #fetchResults}.
+     */
+    private void sendFetches() {
+        FetchEvent event = new FetchEvent();
+        applicationEventHandler.add(event);
+
+        event.future().whenComplete((completedFetches, error) -> {
+            if (error != null)
+                log.warn("An error occurred during poll: {}", 
error.getMessage(), error);
+            else
+                fetchResults.addAll(completedFetches);
+        });
+    }
+
+    private Fetch<K, V> pollForFetches(Timer timer) {
+        long pollTimeout = timer.remainingMs();
+
+        // if data is available already, return it immediately
+        final Fetch<K, V> fetch = fetchCollector.collectFetch(fetchBuffer);
+        if (!fetch.isEmpty()) {
+            return fetch;
+        }
+
+        // send any new fetches (won't resend pending fetches)
+        sendFetches();
+
+        // 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
+
+        // 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);
+
+        // Attempt to fetch any data. It's OK if we don't have any waiting 
data here; it's a 'best case' effort. The
+        // data may not be immediately available, but the calling method 
(poll) will correctly
+        // handle the overall timeout.
+        try {
+            while (pollTimer.notExpired()) {
+                CompletedFetch completedFetch = 
fetchResults.poll(pollTimer.remainingMs(), TimeUnit.MILLISECONDS);
+
+                if (completedFetch != null)
+                    fetchBuffer.add(completedFetch);
+
+                pollTimer.update();
+            }
+        } catch (InterruptedException e) {
+            log.trace("Timeout during fetch", e);
+        } finally {
+            timer.update(pollTimer.currentTimeMs());
+        }
+
+        return fetchCollector.collectFetch(fetchBuffer);
+    }
+
+    /**
+     * Set the fetch position to the committed position (if there is one)
+     * or reset it using the offset reset policy the user has configured.
+     *
+     * @throws org.apache.kafka.common.errors.AuthenticationException if 
authentication fails. See the exception for more details
+     * @throws NoOffsetForPartitionException If no offset is stored for a 
given partition and no offset reset policy is
+     *             defined
+     * @return true iff the operation completed without timing out
+     */
+    private boolean updateFetchPositions(final Timer timer) {
+        // If any partitions have been truncated due to a leader change, we 
need to validate the offsets
+        applicationEventHandler.add(new ValidatePositionsApplicationEvent());
+
+        cachedSubscriptionHasAllFetchPositions = 
subscriptions.hasAllFetchPositions();
+        if (cachedSubscriptionHasAllFetchPositions) return true;
 
-        clusterResourceListeners.maybeAdd(keyDeserializer);
-        clusterResourceListeners.maybeAdd(valueDeserializer);
-        return clusterResourceListeners;
+        // If there are any partitions which do not have a valid position and 
are not
+        // awaiting reset, then we need to fetch committed offsets. We will 
only do a
+        // coordinator lookup if there are partitions which have missing 
positions, so
+        // a consumer with manually assigned partitions can avoid a 
coordinator dependence
+        // by always ensuring that assigned partitions have an initial 
position.
+        if (isCommittedOffsetsManagementEnabled() && 
!refreshCommittedOffsetsIfNeeded(timer))

Review Comment:
   Regardless of the return value of `updateFetchPositions()`, `poll()` will 
still go on to call `pollForFetches()` which will block for data availability 
or timeout expiration. 



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