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


##########
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:
   Or use `org.apache.kafka.common.utils.Utils.isBlank()` which also catches 
the whitespace case:
   
   ```suggestion
           if (Utils.isBlank(pattern)) {
   ```



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java:
##########
@@ -328,6 +343,14 @@ public synchronized Set<String> subscription() {
         return Collections.emptySet();
     }
 
+    public synchronized SubscriptionPattern subscriptionPattern() {
+        return this.subscriptionPattern;
+    }
+
+    public synchronized Pattern subscribedPattern() {
+        return this.subscribedPattern;
+    }
+

Review Comment:
   Super nit: I've seen committers request that we drop unnecessary `this` 
qualifiers:
   
   ```suggestion
       public synchronized SubscriptionPattern subscriptionPattern() {
           return subscriptionPattern;
       }
   
       public synchronized Pattern subscribedPattern() {
           return subscribedPattern;
       }
   
   ```



##########
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:
   Agreed. This is necessary for full coverage of the client-side changes.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java:
##########
@@ -495,6 +496,16 @@ public void subscribe(Pattern pattern) {
         subscribeInternal(pattern, Optional.empty());
     }
 
+    @Override
+    public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {
+        throw new IllegalArgumentException("Operation not supported in the 
classic group protocol");

Review Comment:
   Throwing an `IllegalArgumentException` seems just a tiny bit weird. What 
about `UnsupportedOperationException` instead?



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