Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-09-01 Thread via GitHub


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


##
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:
   I don't mind about the details of the implementation here, but I expect you 
do need a mocked implemented of this new method in order to complete the PR.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-08-31 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1739981694


##
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:
   @AndrewJSchofield, just a question, why do we need to adapt the 
implementation of `subscribe(Pattern pattern, ConsumerRebalanceListener 
callback)`  while the `SubscriptionPattern` is to be resolved on the broker, 
not locally like `Pattern.` If we go that route the `MockConsumer` won't really 
simulate how the `AsyncConsumer` works



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-08-27 Thread via GitHub


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 https://github.com/google/re2j";>RE2/J regular expression 
engine
+ */
+
+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 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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-08-23 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1729748670


##
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:
   @AndrewJSchofield I understand, will change the `subscribedPattern` 
accordingly



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-08-23 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1729747540


##
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:
   @AndrewJSchofield Understood, I will change the `subscribedPattern` 
accordingly



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-08-23 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1729747540


##
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:
   @AndrewJSchofield Understood, I will change the `subscribedPattern` 
accordingly



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-07-23 Thread via GitHub


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.
+ * 
+ * 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.
+ * 
+ * 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 https://github.com/google/re2j";>RE2/J regular expression 
engine
+ */
+
+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.
+ * 
+ * See {@link #subscribe(Collection, ConsumerRebalanceListener)} fo

Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-06-12 Thread via GitHub


github-actions[bot] commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-2164295049

   This PR is being marked as stale since it has not had any activity in 90 
days. If you would like to keep this PR alive, please ask a committer for 
review. If the PR has  merge conflicts, please update it with the latest from 
trunk (or appropriate release branch)  If this PR is no longer valid or 
desired, please feel free to close it. If no activity occurs in the next 30 
days, it will be automatically closed.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-12 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1993206506

   Regex validity check will be included in the next pull request, I'll try to 
get it done by this weekend.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-11 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1519707860


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##
@@ -1730,6 +1744,21 @@ private void subscribeInternal(Pattern pattern, 
Optional listener) {
+acquireAndEnsureOpen();
+try {
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.pattern().isEmpty())
+throw new IllegalArgumentException("Topic pattern to subscribe 
to cannot be " + (pattern == null ?
+"null" : "empty"));
+throwIfNoAssignorsConfigured();

Review Comment:
   The function call is not needed, I'll remove it. From the KIP, I don't see 
the new pattern subscription can be used with a consumer-side assignor.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-10 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1517714052


##
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:
   Nvm, I think adding this class would be a bit much when it only encapsulate 
the logic to check whether pattern or subscriptionPattern is set



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-10 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1518849655


##
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java:
##
@@ -494,6 +501,14 @@ public void testSubscriptionOnEmptyPattern(GroupProtocol 
groupProtocol) {
 () -> consumer.subscribe(Pattern.compile("")));
 }
 
+@ParameterizedTest
+@EnumSource(value = GroupProtocol.class, names = "CONSUMER")

Review Comment:
   nvm, please ignore my previous comment.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-09 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1518786653


##
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##
@@ -35,6 +38,8 @@
   "about": "-1 if it didn't change since the last heartbeat; the maximum 
time in milliseconds that the coordinator will wait on the member to revoke its 
partitions otherwise." },
 { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", 
"nullableVersions": "0+", "default": "null", "entityType": "topicName",
   "about": "null if it didn't change since the last heartbeat; the 
subscribed topic names otherwise." },
+{ "name": "SubscribedTopicRegex", "type": "string", "versions": "1+", 
"nullableVersions": "1+", "default": "null",
+  "about": "null if it didn't change since the last heartbeat; the 
subscribed topic regex otherwise" },

Review Comment:
   Will change this to a separate PR



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-08 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1517714052


##
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:
   Nvm, I think adding this class would be a bit much when it only encapsulate 
the logic to check whether patter or subscriptionPattern is set



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-08 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1517646778


##
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java:
##
@@ -494,6 +501,14 @@ public void testSubscriptionOnEmptyPattern(GroupProtocol 
groupProtocol) {
 () -> consumer.subscribe(Pattern.compile("")));
 }
 
+@ParameterizedTest
+@EnumSource(value = GroupProtocol.class, names = "CONSUMER")

Review Comment:
   I got it wrong here. The subscriptionPattern is not supported in the classic 
protocol.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-08 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1517636440


##
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:
   On second thought, we should adopt this class.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-05 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1513877668


##
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 think it is a bit overkill to have an abstraction like this, the Pattern 
and SubscriptionPattern are already abstractions for the underneath regex 
string. What do you guys think @lianetm @dajac?



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-05 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1513877668


##
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 think it is a bit overkill to have an abstraction like this. What do you 
guys think @lianetm @dajac?



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-04 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1511957999


##
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java:
##
@@ -494,6 +501,14 @@ public void testSubscriptionOnEmptyPattern(GroupProtocol 
groupProtocol) {
 () -> consumer.subscribe(Pattern.compile("")));
 }
 
+@ParameterizedTest
+@EnumSource(value = GroupProtocol.class, names = "CONSUMER")

Review Comment:
   Got it.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-03-04 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1510987276


##
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) {
+log.warn("Operation not supported in new consumer group protocol");

Review Comment:
   @cadonna This method is not supported for the classic protocol iirc. Also I 
will fix the message, it should be "classic group protocol".  



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-23 Thread via GitHub


lianetm commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1961610927

   Regarding the validation of the regex, I lean towards having a new error, 
like @dajac suggested. Just to clearly tell the user that it is using an 
invalid regex, without overcomplicating the experience with an InvalidRequest 
that would have the user wondering what exactly went wrong, probably hinting a 
a much larger surface area, since it indicates that something wen't wrong in 
the client/server interaction. 
   
   This means I think it's better to validate it on the client side too. I 
could be missing something, but to my understanding, the point of avoiding the 
new regex engine on the client was not to avoid the dependency itself, it was 
more a position we still keep, that the broker is the sole responsible for 
resolving the topics to include in a subscription when a regex is used. We 
would still keep that principle if we introduce the dependency on the client 
just to validate the regex.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##
@@ -1730,6 +1744,21 @@ private void subscribeInternal(Pattern pattern, 
Optional listener) {
+acquireAndEnsureOpen();
+try {
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.pattern().isEmpty())

Review Comment:
   I think we should move the check if the pattern is non-empty (and maybe some 
other validity checks) to class `SubscriptionPattern`. It should not be 
possible to create an instance of `SubscriptionPattern` with an empty pattern.



##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -754,6 +754,51 @@ 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 periodically against all topics 
existing at the time of check.
+ * This can be controlled through the {@code metadata.max.age.ms} 
configuration: by lowering
+ * the max metadata age, the consumer will refresh metadata more often and 
check for matching topics.

Review Comment:
   I think this is not correct with the new pattern subscription since the 
assignment is done broker-side.



##
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:
   What about having a class that contains either a `SubscriptionPattern` or a 
`Pattern`?
   Something like :
   
   ```java
   public class JavaPatternOrSubscriptionPattern {
   
   private final Pattern javaPattern;
   
   private final SubscriptionPattern subscriptionPattern;
   
   private JavaPatternOrSubscriptionPattern(final Pattern pattern) {
   this.pattern = pattern;
   } 
   
   private JavaPatternOrSubscriptionPattern(final SubscriptionPattern 
pattern) {
   this.subscriptionPattern = pattern;
   } 
   
   public static JavaPatternOrSubscriptionPattern javaPattern(final Pattern 
pattern) {
   return new JavaPatternOrSubscriptionPattern(pattern);
   }
   
   public static JavaPatternOrSubscriptionPattern subscriptionPattern(final 
SubscriptionPattern pattern) {
   return new JavaPatternOrSubscriptionPattern(pattern);
   }
   
   public String pattern() {
   return subscriptionPattern != null ? subscriptionPattern.pattern() : 
javaPattern.pattern();
   }
   
   public String toString() {
  return "pattern = " + pattern();
   } 
   
  ...
   } 
   ```
   
   In such a way we would encapsulate the code that ensures that there is only 
of both set.  



##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -754,6 +754,51 @@ 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 periodically against all topics 
existing at the time of check.
+ * This can be controlled through the {@code metadata.max.age.ms} 
configuration: by lowering
+ * the max metadata age, the consumer will refresh metadata more often and 
check for matching topics.
+ * 
+ * See {@link #subscribe(Collection, ConsumerRebalanceListener)} for 
details on the
+ * use of the {@link ConsumerRebalanceListener}. Generally rebalances are 
triggered when there
+ * is a change to the topics matching the provided pattern and when 
consumer group membership changes.
+ * Group rebalances only take place during an active call to {@link 
#poll(Duration)}.

Review Comment:
   This should also not correct anymore with KP-848 which introduces this 
method.



##
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) {
+log.warn("Operation not supported in new consumer group protocol");
+}
+
+@Override
+public void subscribe(SubscriptionPattern pattern) {
+log.warn("Operation not supported in new consumer group protocol");

Review Comment:
   See above



##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java:
##

Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953901380

   @cadonna I'll see if there is a way to go around with not using the Google 
library to check regex validity (finger-crossed!)


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953893744

   > > > I was wondering whether we should introduce a new error code to signal 
to the user that the regular expression is invalid. Otherwise, we would have to 
use the invalid request exception and it does not seem user friendly. @cadonna 
@lianetm What do you think about this?
   > > 
   > > 
   > > @dajac Is it also possible to verify the validity of the regular 
expression client-side?
   > > If we assume that the clients send valid regular expressions to the 
brokers, I think it would be OK to return an invalid request exception and log 
the error broker-side. Sending invalid regular expressions should than just be 
a mistake that happens during development of the clients and not something that 
happens during usage of the clients.
   > > The benefit would be to find the mistakes in regular expressions without 
a request to the brokers.
   > > The downside of it is that we need some way to validate the regular 
expressions client-side like the corresponding Google library in Java and I do 
not know what dependency are needed for clients in other languages.
   > 
   > I don't think we can include the Google library in the client code. I saw 
the comment about it on the pull request for the implementation of the regex 
logic on the broker. Will find it again and quote it here.
   > 
   > Edit: here is the comment [#14327 
(review)](https://github.com/apache/kafka/pull/14327#pullrequestreview-1622366023)
   
   Ah, I see! Thanks for the link!
   Nevertheless, there it says the resolution of the regular expression 
regarding the existing topics on the brokers. It does not say anything about 
checking the general validity of the regular expression.
   I definitely agree that finding topics that match the regular expression is 
something that needs to be strictly done on the brokers.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953879987

   > > I was wondering whether we should introduce a new error code to signal 
to the user that the regular expression is invalid. Otherwise, we would have to 
use the invalid request exception and it does not seem user friendly. @cadonna 
@lianetm What do you think about this?
   > 
   > 
   > 
   > @dajac Is it also possible to verify the validity of the regular 
expression client-side?
   > 
   > If we assume that the clients send valid regular expressions to the 
brokers, I think it would be OK to return an invalid request exception and log 
the error broker-side. Sending invalid regular expressions should than just be 
a mistake that happens during development of the clients and not something that 
happens during usage of the clients.
   > 
   > 
   > 
   > The benefit would be to find the mistakes in regular expressions without a 
request to the brokers.  
   > 
   >  
   > 
   > The downside of it is that we need some way to validate the regular 
expressions client-side like the corresponding Google library in Java and I do 
not know what dependency are needed for clients in other languages.
   > 
   > 
   
   I don't think we can include the Google library in the client code. I saw 
the comment about it on the pull request for the implementation of the regex 
logic on the broker. Will find it again and quote it here.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953872828

   > I was wondering whether we should introduce a new error code to signal to 
the user that the regular expression is invalid. Otherwise, we would have to 
use the invalid request exception and it does not seem user friendly. @cadonna 
@lianetm What do you think about this?
   
   @dajac Is it also possible to verify the validity of the regular expression 
client-side?
   If we assume that the clients send valid regular expressions to the brokers, 
I think it would be OK to return an invalid request exception and log the error 
broker-side. Sending invalid regular expressions should than just be a mistake 
that happens during development of the clients and not something that happens 
during usage of the clients.
   
   The benefit would be to find the mistakes in regular expressions without a 
request to the brokers.  

   The downside of it is that we need some way to validate the regular 
expressions client-side like the corresponding Google library in Java and I do 
not know what dependency are needed for clients in other languages.
   


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-20 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1953815656

   > > > @cadonna @lianetm, since we're supporting for both subscribe method 
using java.util.regex.Pattern and SubscriptionPattern, I think we should throw 
a illegal heartbeat exeption when user try to use both method at the same time 
and inform the user to use once at a time, since the field SubscribedRegex is 
used for java.util.regex.Pattern as well as SubscriptionPattern. What do you 
guys think?
   > > 
   > > 
   > > IMO, we must support the deprecated pattern subscriptions with 
`java.util.regex.Pattern` to ensure backwards compatibility, but we do not need 
to support mixed usage of `java.util.regex.Pattern` and Google regex patterns. 
I think this is a blind spot in the KIP. I propose to throw an 
`IllegalStateException` if `subscribe(java.util.regex.Pattern)` is called after 
`subscribe(SubscriptionPattern)` (and vice versa) without calling 
`unsubscribe()` in between. That is similar to the restrictions between 
pattern, topic, and partition subscriptions @lianetm linked above. I do not 
think it is worth to consider the edge case of mixed usage of the two pattern 
types. Does this make sense to you? \cc @dajac What do you as the original 
author of the KIP think? Should we update the KIP to make this clear?
   > 
   > @cadonna I would rather follow what we already do with `subscribe` today. 
The last one called takes precedence.
   
   @dajac The javadocs of `subscribe()` say:
   
   ```
* @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
   ```
   
https://github.com/apache/kafka/blob/f0087ac6a8a7b1005e9588e42b3679146bd3eb13/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L727
   
   One could argue that both subscriptions are pattern subscriptions, but they 
are quite different internally. I am wondering how complex it is to allow mixed 
usage.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-19 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1952492560

   > > > @cadonna @lianetm, since we're supporting for both subscribe method 
using java.util.regex.Pattern and SubscriptionPattern, I think we should throw 
a illegal heartbeat exeption when user try to use both method at the same time 
and inform the user to use once at a time, since the field SubscribedRegex is 
used for java.util.regex.Pattern as well as SubscriptionPattern. What do you 
guys think?
   > > 
   > > 
   > > IMO, we must support the deprecated pattern subscriptions with 
`java.util.regex.Pattern` to ensure backwards compatibility, but we do not need 
to support mixed usage of `java.util.regex.Pattern` and Google regex patterns. 
I think this is a blind spot in the KIP. I propose to throw an 
`IllegalStateException` if `subscribe(java.util.regex.Pattern)` is called after 
`subscribe(SubscriptionPattern)` (and vice versa) without calling 
`unsubscribe()` in between. That is similar to the restrictions between 
pattern, topic, and partition subscriptions @lianetm linked above. I do not 
think it is worth to consider the edge case of mixed usage of the two pattern 
types. Does this make sense to you? \cc @dajac What do you as the original 
author of the KIP think? Should we update the KIP to make this clear?
   > 
   > @cadonna I would rather follow what we already do with `subscribe` today. 
The last one called takes precedence.
   
   I have a question. The subscribe method that use Pattern override the 
subscription with topic(s) that match the Pattern. When user choose to use 
SubscriptionPattern, but already used Pattern beforehand, should we clear out 
the old subscription?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-16 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1492453933


##
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##
@@ -35,6 +35,8 @@
   "about": "-1 if it didn't change since the last heartbeat; the maximum 
time in milliseconds that the coordinator will wait on the member to revoke its 
partitions otherwise." },
 { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", 
"nullableVersions": "0+", "default": "null", "entityType": "topicName",
   "about": "null if it didn't change since the last heartbeat; the 
subscribed topic names otherwise." },
+{ "name": "SubscribedTopicRegex", "type": "string", "versions": "0+", 
"nullableVersions": "0+", "default": "null",
+  "about": "null if it didn't change since the last heartbeat; the 
subscribed topic regex otherwise" },

Review Comment:
   I understand



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-16 Thread via GitHub


dajac commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1948347606

   I was wondering whether we should introduce a new error code to signal to 
the user that the regular expression is invalid. Otherwise, we would have to 
use the invalid request exception and it does not seem user friendly. @cadonna 
@lianetm What do you think about this?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-16 Thread via GitHub


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


##
clients/src/main/resources/common/message/ConsumerGroupHeartbeatRequest.json:
##
@@ -35,6 +35,8 @@
   "about": "-1 if it didn't change since the last heartbeat; the maximum 
time in milliseconds that the coordinator will wait on the member to revoke its 
partitions otherwise." },
 { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", 
"nullableVersions": "0+", "default": "null", "entityType": "topicName",
   "about": "null if it didn't change since the last heartbeat; the 
subscribed topic names otherwise." },
+{ "name": "SubscribedTopicRegex", "type": "string", "versions": "0+", 
"nullableVersions": "0+", "default": "null",
+  "about": "null if it didn't change since the last heartbeat; the 
subscribed topic regex otherwise" },

Review Comment:
   This is incorrect. We cannot add a field to a released version. I would 
suggest to bump the version of the RPC and use the correct version here. We 
should also mark the last version as unstable with `latestVersionUnstable` set 
to `true`.



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-16 Thread via GitHub


dajac commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1948333168

   > > @cadonna @lianetm, since we're supporting for both subscribe method 
using java.util.regex.Pattern and SubscriptionPattern, I think we should throw 
a illegal heartbeat exeption when user try to use both method at the same time 
and inform the user to use once at a time, since the field SubscribedRegex is 
used for java.util.regex.Pattern as well as SubscriptionPattern. What do you 
guys think?
   > 
   > IMO, we must support the deprecated pattern subscriptions with 
`java.util.regex.Pattern` to ensure backwards compatibility, but we do not need 
to support mixed usage of `java.util.regex.Pattern` and Google regex patterns. 
I think this is a blind spot in the KIP. I propose to throw an 
`IllegalStateException` if `subscribe(java.util.regex.Pattern)` is called after 
`subscribe(SubscriptionPattern)` (and vice versa) without calling 
`unsubscribe()` in between. That is similar to the restrictions between 
pattern, topic, and partition subscriptions @lianetm linked above. I do not 
think it is worth to consider the edge case of mixed usage of the two pattern 
types. Does this make sense to you? \cc @dajac What do you as the original 
author of the KIP think? Should we update the KIP to make this clear?
   
   @cadonna I would rather follow what we already do with `subscribe` today. 
The last one called takes precedence.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-13 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1941295224

   > > @cadonna @lianetm, since we're supporting for both subscribe method 
using java.util.regex.Pattern and SubscriptionPattern, I think we should throw 
a illegal heartbeat exeption when user try to use both method at the same time 
and inform the user to use once at a time, since the field SubscribedRegex is 
used for java.util.regex.Pattern as well as SubscriptionPattern. What do you 
guys think?
   > 
   > 
   > 
   > IMO, we must support the deprecated pattern subscriptions with 
`java.util.regex.Pattern` to ensure backwards compatibility, but we do not need 
to support mixed usage of `java.util.regex.Pattern` and Google regex patterns. 
I think this is a blind spot in the KIP. I propose to throw an 
`IllegalStateException` if `subscribe(java.util.regex.Pattern)` is called after 
`subscribe(SubscriptionPattern)` (and vice versa) without calling 
`unsubscribe()` in between. That is similar to the restrictions between 
pattern, topic, and partition subscriptions @lianetm linked above. I do not 
think it is worth to consider the edge case of mixed usage of the two pattern 
types.
   > 
   > Does this make sense to you?
   > 
   > \cc @dajac What do you as the original author of the KIP think? Should we 
update the KIP to make this clear?
   
   @cadonna that makes sense 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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-13 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1941284769

   @cadonna, sorry for the delay. I'll push the changes tomorrow 


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-13 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1941283018

   @lianetm thanks for the reply. I was more wondering about the testing 
strategy of the new subscribe(SubscriptionPattern) method when it comes to 
receiving the assignment, since that part is not finished by the broker, but 
I'll follow the same testing logic as the subscribe(Pattern) method as you 
mentioned the receiving part work the same for both


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-13 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1941278766

   @Phuc-Hong-Tran Could you please implement the changes that I requested so 
that we can move on?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-13 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1941264974

   > I could be missing something, but I would say we don't need any changes 
for the part where the client receives the assignment from the broker after 
subscribing to a regex. It should be exactly the same logic as when a client 
receives an assignment from the broker after subscribing to a list of topics. 
After sending the HB with the new regex, the client will receive the list of 
partitions assigned to it, and will reconcile them, just as it reconciles all 
assignments received (no matter the subscription type that led to receiving 
that assignment).
   
   That is also my understanding.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-13 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1941258118

   > @cadonna @lianetm, since we're supporting for both subscribe method using 
java.util.regex.Pattern and SubscriptionPattern, I think we should throw a 
illegal heartbeat exeption when user try to use both method at the same time 
and inform the user to use once at a time, since the field SubscribedRegex is 
used for java.util.regex.Pattern as well as SubscriptionPattern. What do you 
guys think?
   
   IMO, we must support the deprecated pattern subscriptions with 
`java.util.regex.Pattern` to ensure backwards compatibility, but we do not need 
to support mixed usage of `java.util.regex.Pattern` and Google regex patterns. 
I think this is a blind spot in the KIP. I propose to throw an 
`IllegalStateException` if `subscribe(java.util.regex.Pattern)` is called after 
`subscribe(SubscriptionPattern)` (and vice versa) without calling 
`unsubscribe()` in between. That is similar to the restrictions between 
pattern, topic, and partition subscriptions @lianetm linked above. I do not 
think it is worth to consider the edge case of mixed usage of the two pattern 
types.
   Does this make sense to you?
   \cc @dajac What do you as the original author of the KIP think? Should we 
update the KIP to make this clear?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-09 Thread via GitHub


lianetm commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1936141808

   @Phuc-Hong-Tran regarding this:
   
   > Just for clarification, when we were talking about "implement and test 
everything up to the point where the field is populated", does that mean we're 
not gonna implement and test the part where the client receive the assignment 
from broker at this stage?
   
   We do need to:
   - `implement and test everything up to the point where the field is 
populated ` (from the point the user calls subscribe with the 
SubscriptionPattern, to the point where the supplied regex is available in the 
HB builder, to be included in the HB request). 
   - include the field in the HB request. This is where we do need the RPC to 
be updated to support the new field.
   
   I could be missing something, but I would say we don't need any changes for 
the part where the client receives the assignment from the broker after 
subscribing to a regex. It should be exactly the same logic as when a client 
receives an assignment from the broker after subscribing to a list of topics. 
After sending the HB with the new regex, the client will receive the list of 
partitions assigned to it, and will reconcile them, just as it reconciles all 
assignments received (no matter the subscription type that led to receiving 
that assignment).
   
   Just for the record, the legacy coordinator does have a bit of logic 
([here](https://github.com/apache/kafka/blob/ec4a8aaadbc95cfcf0de2f5e1385373f095298ca/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L367))
 for validating assignments received after subscribing to a regex, where it 
checks that the assignment received matches the regex. Our initial thought was 
not to include any assignment validation like that in the client, in an attempt 
to simplify it: the broker is the sole responsible for computing the regex and 
target assignment, the client takes and reconciles whatever the broker sends, 
and if the subscription changes from the client side, we have a common logic 
(not specific for regex), to make sure that the new subscription is sent to the 
broker (what the legacy achieved with the rejoin)


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-09 Thread via GitHub


lianetm commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1936091369

   Hey @Phuc-Hong-Tran , regarding the mixed usage of subscribe with `Pattern` 
and with `SubscriptionPattern`, my opinion is that it is something we should 
live with to provide a smooth transition, while the usage of `Pattern` is 
deprecated. So I would say that we shouldn't restrict it by throwing any new 
exception to the user (which btw, would introduce API level changes not 
included in the KIP, so it would require an updated/new KIP). We could just 
allow subsequent calls to both subscribe with Pattern or SubscriptionPattern, 
and just ensure that the latest prevails. This is the behaviour for subsequent 
calls to `subscribe(Pattern..)`, tested in 
[testSubsequentPatternSubscription](https://github.com/apache/kafka/blob/ec4a8aaadbc95cfcf0de2f5e1385373f095298ca/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala#L417).
  
   
   Just for the record, there is a restriction (see 
[here](https://github.com/apache/kafka/blob/ec4a8aaadbc95cfcf0de2f5e1385373f095298ca/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java#L162))
 for not allowing mixed usage of subscribe, but only when mixing different 
subscription types (topics, partitions, pattern). We continue to respect that, 
without introducing any new restriction for the calls that in the end represent 
the same pattern-based subscription type (AUTO_PATTERN).  


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-06 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1930102760

   @cadonna Just for clarification, when we were talking about "implement and 
test everything up to the point where the field is populated", does that mean 
we're not gonna implement and test the part where the client receive the 
assignment from broker at this stage? (I'm mostly blocked at this part)


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-06 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1929317343

   @cadonna @lianetm, since we're supporting for both subscribe method using 
java.util.regex.Pattern and SubscriptionPattern, I think we should throw a 
illegal heartbeat exeption when user try to use both method at the same time 
and inform the user to use once at a time, since the field SubscribedRegex is 
used for java.util.regex.Pattern as well as SubscriptionPattern. What do you 
guys think?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-06 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1479598776


##
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;
 
+/* we should rename this to something more specific */
+private SubscriptionPattern subscriptionPattern;

Review Comment:
   Will do 



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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-02-03 Thread via GitHub


Phuc-Hong-Tran closed pull request #15188: KAFKA-15561: Client support for new 
SubscriptionPattern based subscription
URL: https://github.com/apache/kafka/pull/15188


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918921920

   @dajac If you feel more comfortable, we could implement and test everything 
up to the point where the field is populated. We would then not populate the 
field so that you do not need to add the field to the RPC before the 
broker-side is done. 


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


dajac commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918915394

   @Phuc-Hong-Tran Yep, that's right.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


dajac commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918915086

   @cadonna Yes. We could for instance commit the RPC and then work 
independently on the client and the server. My only concern is that we usually 
discover issues while working on the server side. This is why I usually prefer 
to get the server code into a reasonable state first. In this case, the risk is 
low as we are talking about a single field.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918907960

   @dajac, when we're talking about the RPC, do we mean the field for the regex 
in ConsumerGroupHeartbeatRequest.json?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918901251

   @dajac OK, but we can implement and unit test everything up to the RPC, 
right?


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


dajac commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918899056

   In my opinion, we should merge the client side only after the server side is 
implemented. The reason is that we need to change the RPC (this is actually 
missing in this PR) and this should be driven by the server side work.


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918895644

   I understand, will get back to speed on this one


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918887578

   @Phuc-Hong-Tran 
   
   > though isn't it this one aiming for 3.8 release?
   
   Yes, but we have time-based releases in Apache Kafka. That means that the 
deadline for the release will be somewhere in the beginning of April. We need 
to keep enough time before this data  for testing.
   
   


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918876275

   @cadonna thanks for the comment, I can still finish one as the deadline 
required, though isn't it this one aiming for 3.8 release? Also the logic on 
the broker is not implemented yet, have a look at 
https://issues.apache.org/jira/browse/KAFKA-14517


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


cadonna commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1918874061

   > - This feature is not yet implemented on the broker, so 
t[here](https://github.com/apache/kafka/blob/055ff2b831193f5935f9efc2f7809f853f63de5f/clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java#L537)
 are a few areas that may need alignment. Would maybe make sense to wait for 
some progress in the broker and then give this another push.
   > - This PR attempts to ensure that the new regex is included in the 
subscription state and that's definitely needed. But then we're missing the 
other half of the story: we should make sure that the regex is passed-on to the 
broker on the next heartbeat request (see HeartbeatRequestManager here)
   
   I do not understand why we want to wait. We should make progress as much as 
possible even if the broker code is not there yet. In the worst case, we would 
get an error back from the broker until the broker-side is implemented. But 
that would be fine in my opinion. 


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



Re: [PR] KAFKA-15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-31 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -753,6 +753,10 @@ public void subscribe(Pattern pattern, 
ConsumerRebalanceListener listener) {
 public void subscribe(Pattern pattern) {
 delegate.subscribe(pattern);
 }
+@Override
+public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {}

Review Comment:
   This method needs javadocs describing what this method does. Please also add 
an empty line before the method.



##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##
@@ -1724,6 +1738,21 @@ private void subscribeInternal(Pattern pattern, 
Optional listener) {
+acquireAndEnsureOpen();
+try {
+maybeThrowInvalidGroupIdException();
+if (pattern == null || pattern.pattern().isEmpty())
+throw new IllegalArgumentException("Topic pattern to subscribe 
to cannot be " + (pattern == null ?
+"null" : "empty"));
+throwIfNoAssignorsConfigured();
+log.info("Subscribed to pattern: '{}'", pattern);

Review Comment:
   This will probably not work since you did not add a `toString()` method to 
`SubscriptionPattern`. You should either add `toString()` to 
`SubscriptionPattern` or change this line to
   ```suggestion
   log.info("Subscribed to pattern: '{}'", pattern.pattern());
   ```



##
clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java:
##
@@ -72,6 +72,10 @@ public interface Consumer extends Closeable {
 */
 void subscribe(Pattern pattern);
 
+void subscribe(SubscriptionPattern pattern, ConsumerRebalanceListener 
callback);

Review Comment:
   This method needs a link to the javadocs in `KafkaConsumer`:
   ```suggestion
   /**
* @see KafkaConsumer#subscribe(SubscriptionPattern, 
ConsumerRebalanceListener)
*/
   void subscribe(SubscriptionPattern pattern, ConsumerRebalanceListener 
callback);
   ```



##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -753,6 +753,10 @@ public void subscribe(Pattern pattern, 
ConsumerRebalanceListener listener) {
 public void subscribe(Pattern pattern) {
 delegate.subscribe(pattern);
 }
+@Override
+public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {}
+@Override
+public void subscribe(SubscriptionPattern pattern) {}

Review Comment:
   This method needs javadocs describing what this method does. Please also add 
an empty line before the method.



##
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;
 
+/* we should rename this to something more specific */
+private SubscriptionPattern subscriptionPattern;

Review Comment:
   Could you please also reset this field in `unsubscribe()` and add it to the 
return value of `toString()`?



##
clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java:
##
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+public class SubscriptionPattern {

Review Comment:
   > This class would need some doc explaining what it represents.
   
   I definitely agree with @lianetm on this.
   
   > Also not sure if this is the right place for it (given that this whole 
intention with the new regex is driven by the broker, but is not implemented 
yet). So at this point is not clear to me if we would prefer to define this on 
the broker side to be used there?
   
   The KIP says this class should be added to the public API of the consumer. I 
agree with @Phuc-Hong-Tran that this class is merely there to differentiate 
patterns that are Google RE2/J from `java.util.regex.Pattern` as stated in the 
KIP.  
   
   



##
clients/src/main/ja

Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1454760385


##
clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java:
##
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+public class SubscriptionPattern {

Review Comment:
   I think we should leave it on the consumer side, as I think its 
functionalities is more to differenciate between using Java.util.Pattern and 
using regex that compatible with RE2J. If we were to use a string instead of 
SubscribedPattern, the user may get confused about why they need to use Pattern 
for the other subsribe methods.



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1454761116


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java:
##
@@ -86,7 +86,6 @@
 import static org.mockito.Mockito.when;
 
 public class OffsetsRequestManagerTest {
-

Review Comment:
   My bad



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


Phuc-Hong-Tran commented on code in PR #15188:
URL: https://github.com/apache/kafka/pull/15188#discussion_r1454760385


##
clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java:
##
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+public class SubscriptionPattern {

Review Comment:
   I think we should leave it on the consumer side, as I think its 
functionalities is more to differenciate between using Java.util.Pattern and 
using regex that compatible with RE2J. If we were just to use a string instead 
of SubscribedPattern, the user may get confused about why they need to use 
Pattern for the other subsribe methods.



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1894635327

   Thanks @lianetm 


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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


lianetm commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1894569927

   This is the task to closely follow 
https://issues.apache.org/jira/browse/KAFKA-14517, where the broker will 
support the new regex.


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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1894475663

   @lianetm, thanks for the comments, I will make sure to address those points 
in my next PR.
   
   Regarding your point about passing the regex for HeartbeatRequestManager, I 
origninally included that in my code change, then I came across this PR 
https://github.com/apache/kafka/pull/14956 and decided that we need to wait for 
the broker to implement new regex logic first.


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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


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


##
clients/src/test/java/org/apache/kafka/clients/consumer/internals/OffsetsRequestManagerTest.java:
##
@@ -86,7 +86,6 @@
 import static org.mockito.Mockito.when;
 
 public class OffsetsRequestManagerTest {
-

Review Comment:
   Nit: I find it's better to avoid changes in unrelated files, even if minor



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/clients/consumer/SubscriptionPattern.java:
##
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+public class SubscriptionPattern {

Review Comment:
   This class would need some doc explaining what it represents. Also not sure 
if this is the right place for it (given that this whole intention with the new 
regex is driven by the broker, but is not implemented yet). So at this point is 
not clear to me if we would prefer to define this on the broker side to be used 
there? Could be. As suggested in the Jira, maybe we should wait for the broker 
implementation of the new regex, and then align on this class? 



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


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


##
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;
 
+/* we should rename this to something more specific */

Review Comment:
   Agree that it's confusing but can't think of a better naming. I would 
suggest though that we add a proper comment, stating that this represents the 
RE2J regex (vs the java regex represented by the `Pattern subscribedPattern` 



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


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


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

Review Comment:
   Not stated in the KIP I believe, but I would say that we should log at least 
a warn indicating that this is not supported with the legacy protocol (similar 
to what the KIP states for logging a warn for the `enforceRebalance` that is 
not supported with the new protocol, and that we already do 
[here](https://github.com/apache/kafka/blob/055ff2b831193f5935f9efc2f7809f853f63de5f/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L1187))



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

Review Comment:
   same as above comment



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-16 Thread via GitHub


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


##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -753,6 +753,10 @@ public void subscribe(Pattern pattern, 
ConsumerRebalanceListener listener) {
 public void subscribe(Pattern pattern) {
 delegate.subscribe(pattern);
 }
+@Override
+public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {}

Review Comment:
   I expect we should be calling the delegate.subscribe(SubscriptionPattern..) 
here, otherwise the actual implementation in the AsyncKafkaConsumer won't be 
called. Similar to the subscribe(Pattern..) 
   (This KafkaConsumer is the user-facing api, that ends up calling the Legacy 
or Async consumer via the delegate)



##
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##
@@ -753,6 +753,10 @@ public void subscribe(Pattern pattern, 
ConsumerRebalanceListener listener) {
 public void subscribe(Pattern pattern) {
 delegate.subscribe(pattern);
 }
+@Override
+public void subscribe(SubscriptionPattern pattern, 
ConsumerRebalanceListener callback) {}
+@Override
+public void subscribe(SubscriptionPattern pattern) {}

Review Comment:
   Same as above comment



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



Re: [PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-13 Thread via GitHub


Phuc-Hong-Tran commented on PR #15188:
URL: https://github.com/apache/kafka/pull/15188#issuecomment-1890427403

   @lianetm, PTAL, thanks in advance.


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



[PR] Kafka 15561: Client support for new SubscriptionPattern based subscription [kafka]

2024-01-13 Thread via GitHub


Phuc-Hong-Tran opened a new pull request, #15188:
URL: https://github.com/apache/kafka/pull/15188

   Change:
   1. Implement methods in AsyncKafkaConsumer that accept SubscriptionPattern 
to subscribe to topic(s).
   2. Pass on the subscriptionPattern to SubscriptionState to use once server 
support RE2/J.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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