AndrewJSchofield commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1687645000


##########
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##########
@@ -754,6 +752,49 @@ public void subscribe(Pattern pattern) {
         delegate.subscribe(pattern);
     }
 
+    /**
+     * Subscribe to all topics matching specified pattern to get dynamically 
assigned partitions.
+     * The pattern matching will be done on the broker against topics existing 
at the time of check
+     * and matching topic(s) will be returned to the client.
+     * <p>
+     * See {@link #subscribe(Collection, ConsumerRebalanceListener)} for 
details on the
+     * use of the {@link ConsumerRebalanceListener}.
+     *
+     * @param pattern SubscriptionPattern to subscribe to
+     * @param listener Non-null listener instance to get notifications on 
partition assignment/revocation for the
+     *                 subscribed topics
+     * @throws IllegalArgumentException If pattern or listener is null
+     * @throws IllegalStateException If {@code subscribe()} is called 
previously with topics, or assign is called
+     *                               previously (without a subsequent call to 
{@link #unsubscribe()}), or if not
+     *                               configured at-least one partition 
assignment strategy
+     */
+    @Override
+    public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {
+        delegate.subscribe(pattern, callback);
+    }
+
+    /**
+     * Subscribe to all topics matching specified pattern to get dynamically 
assigned partitions.
+     * The pattern matching will be done on the broker against topics existing 
at the time of check
+     * and matching topic(s) will be returned to the client.
+     * <p>
+     * This is a short-hand for {@link #subscribe(SubscriptionPattern, 
ConsumerRebalanceListener)}, which

Review Comment:
   Should be "shorthand".



##########
clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java:
##########
@@ -141,6 +141,16 @@ public synchronized void subscribe(Pattern pattern) {
         subscribe(pattern, Optional.empty());
     }
 
+    @Override
+    public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {
+

Review Comment:
   It would be very easy to adapt the implementation of `subscribe(Pattern 
pattern, ConsumerRebalanceListener callback)` for this rather than leaving it 
blank.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java:
##########
@@ -141,6 +141,16 @@ public synchronized void subscribe(Pattern pattern) {
         subscribe(pattern, Optional.empty());
     }
 
+    @Override
+    public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {
+
+    }
+
+    @Override
+    public void subscribe(SubscriptionPattern pattern) {
+

Review Comment:
   And here.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Represents a regular expression used to subscribe to topics. The pattern
+ * must be a Google RE2/J compatible pattern. Visit
+ * 
+ * @see <a href="https://github.com/google/re2j";>RE2/J regular expression 
engine</a>
+ */
+
+public class SubscriptionPattern {
+    final private String pattern;
+    public SubscriptionPattern(final String pattern) {
+        if (pattern == null || pattern.equals("")) {

Review Comment:
   I suggest using `String.isEmpty()` rather than `equals("")`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##########
@@ -754,6 +752,49 @@ public void subscribe(Pattern pattern) {
         delegate.subscribe(pattern);
     }
 
+    /**
+     * Subscribe to all topics matching specified pattern to get dynamically 
assigned partitions.
+     * The pattern matching will be done on the broker against topics existing 
at the time of check
+     * and matching topic(s) will be returned to the client.
+     * <p>
+     * See {@link #subscribe(Collection, ConsumerRebalanceListener)} for 
details on the
+     * use of the {@link ConsumerRebalanceListener}.
+     *
+     * @param pattern SubscriptionPattern to subscribe to
+     * @param listener Non-null listener instance to get notifications on 
partition assignment/revocation for the
+     *                 subscribed topics
+     * @throws IllegalArgumentException If pattern or listener is null
+     * @throws IllegalStateException If {@code subscribe()} is called 
previously with topics, or assign is called
+     *                               previously (without a subsequent call to 
{@link #unsubscribe()}), or if not
+     *                               configured at-least one partition 
assignment strategy
+     */
+    @Override
+    public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {
+        delegate.subscribe(pattern, callback);
+    }
+
+    /**
+     * Subscribe to all topics matching specified pattern to get dynamically 
assigned partitions.
+     * The pattern matching will be done on the broker against topics existing 
at the time of check

Review Comment:
   Better would be "at the time of the check and matching topics will be 
returned".



##########
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##########
@@ -754,6 +752,49 @@ public void subscribe(Pattern pattern) {
         delegate.subscribe(pattern);
     }
 
+    /**
+     * Subscribe to all topics matching specified pattern to get dynamically 
assigned partitions.
+     * The pattern matching will be done on the broker against topics existing 
at the time of check

Review Comment:
   Better would be "at the time of the check and matching topics will be 
returned". No need to (s) which is a bit messy.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -553,7 +553,7 @@ public ConsumerGroupHeartbeatRequestData buildRequestData() 
{
                 }
             } else {
                 // SubscribedTopicRegex - only sent if has changed since the 
last heartbeat
-                //                      - not supported yet
+

Review Comment:
   Why not go ahead and implement the `SentFields` code and call 
`data.setSubscribedTopicRegex()`?



##########
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##########
@@ -13,6 +13,8 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+//Version 1 added SubscribedTopicRegex to the request for KIP-848

Review Comment:
   I would expect you to actually define v1 including the extra field, and also 
v1 of the response (which I suppose is structurally identical to v0).



##########
clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Represents a regular expression used to subscribe to topics. The pattern
+ * must be a Google RE2/J compatible pattern. Visit
+ * 
+ * @see <a href="https://github.com/google/re2j";>RE2/J regular expression 
engine</a>
+ */
+
+public class SubscriptionPattern {
+    final private String pattern;
+    public SubscriptionPattern(final String pattern) {
+        if (pattern == null || pattern.equals("")) {
+            throw new IllegalArgumentException("Topic pattern to subscribe to 
cannot be " + (pattern == null ?
+                    "null" : "empty"));
+        } else {
+            this.pattern = pattern;
+        }
+

Review Comment:
   nit: Extraneous blank line



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java:
##########
@@ -84,6 +85,9 @@ private enum SubscriptionType {
     /* the pattern user has requested */
     private Pattern subscribedPattern;
 
+    /* RE2J compatible regex */
+    private SubscriptionPattern subscriptionPattern;

Review Comment:
   I agree with @cadonna. It seems very odd to me having `subscribedPattern` 
and `subscriptionPattern`. I understand that there are old patterns and new 
patterns, but really they are achieving the same thing. Hiding the difference 
inside `subscribedPattern` seems sensible to me.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to