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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##########
@@ -636,42 +857,148 @@ public void assign(Collection<TopicPartition> 
partitions) {
     }
 
     @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().equals(""))
+            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);
+    private void sendFetches() {
+        FetchEvent event = new FetchEvent();
+        eventHandler.add(event);
+
+        event.future().whenComplete((completedFetches, error) -> {
+            if (completedFetches != null && !completedFetches.isEmpty()) {
+                fetchBuffer.addAll(completedFetches);
+            }
+        });
+    }
+
+    /**
+     * @throws KafkaException if the rebalance callback throws exception

Review Comment:
   Not yet, no. I'll remove the comment because we haven't implemented the 
callback mechanism here. It's in a draft PR #14357. We do have the _general_ 
mechanism for how we'll end up calling them, which is via the 
`BackgroundEventProcessor`.
   
   I did notice a difference between `KafkaConsumer` and 
`PrototypeAsyncConsumer`—the former is potentially invoking the rebalance 
callback on each iteration of the loop inside `poll()` whereas the latter 
implementation is only calling it once at the top of `poll()`. I'll change ours 
to work in a similar fashion.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##########
@@ -636,42 +857,148 @@ public void assign(Collection<TopicPartition> 
partitions) {
     }
 
     @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().equals(""))
+            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);
+    private void sendFetches() {
+        FetchEvent event = new FetchEvent();
+        eventHandler.add(event);
+
+        event.future().whenComplete((completedFetches, error) -> {
+            if (completedFetches != null && !completedFetches.isEmpty()) {
+                fetchBuffer.addAll(completedFetches);
+            }
+        });
+    }
+
+    /**
+     * @throws KafkaException if the rebalance callback throws exception

Review Comment:
   Not yet, no. I'll remove the comment because we haven't implemented the 
callback mechanism here. It's in a draft PR #14357. We do have the _general_ 
mechanism for how we'll end up calling them, which is via the 
`BackgroundEventProcessor`.
   
   I did notice a difference between `KafkaConsumer` and 
`PrototypeAsyncConsumer`—the former is potentially invoking the rebalance 
callback on each iteration of the loop inside `poll()` whereas the latter 
implementation is only calling it once at the top of `poll()`. I'll change ours 
to work in a similar fashion.



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