[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-17 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1265943974


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##
@@ -106,4 +114,17 @@ private boolean process(final OffsetFetchApplicationEvent 
event) {
 manager.addOffsetFetchRequest(event.partitions);
 return true;
 }
+
+private boolean process(final NewTopicsMetadataUpdateRequestEvent event) {
+metadata.requestUpdateForNewTopics();
+return true;
+}
+
+private boolean process(final AssignmentChangeApplicationEvent event) {
+Optional commitRequestManger = 
registry.get(RequestManager.Type.COMMIT);
+CommitRequestManager manager = (CommitRequestManager) 
commitRequestManger.get();

Review Comment:
   I think an isPresent check is missing here. Updated. To your question: when 
groupId is null, we don't build the coordinator request manager and thus the 
commit request manager, so isPresent check is needed.



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



[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-17 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1265949795


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##
@@ -106,4 +114,17 @@ private boolean process(final OffsetFetchApplicationEvent 
event) {
 manager.addOffsetFetchRequest(event.partitions);
 return true;
 }
+
+private boolean process(final NewTopicsMetadataUpdateRequestEvent event) {
+metadata.requestUpdateForNewTopics();
+return true;
+}
+
+private boolean process(final AssignmentChangeApplicationEvent event) {
+Optional commitRequestManger = 
registry.get(RequestManager.Type.COMMIT);
+CommitRequestManager manager = (CommitRequestManager) 
commitRequestManger.get();

Review Comment:
   FWIW: I think currently, group.id can be null as well as an empty string.  
Only when the group id is null, the coordinator object won't be created.  We 
are planning to drop the support of empty string soon.



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



[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-17 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1265943974


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##
@@ -106,4 +114,17 @@ private boolean process(final OffsetFetchApplicationEvent 
event) {
 manager.addOffsetFetchRequest(event.partitions);
 return true;
 }
+
+private boolean process(final NewTopicsMetadataUpdateRequestEvent event) {
+metadata.requestUpdateForNewTopics();
+return true;
+}
+
+private boolean process(final AssignmentChangeApplicationEvent event) {
+Optional commitRequestManger = 
registry.get(RequestManager.Type.COMMIT);
+CommitRequestManager manager = (CommitRequestManager) 
commitRequestManger.get();

Review Comment:
   I think an isPresent check is missing here. Updated.



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



[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-17 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1265845671


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -522,7 +525,35 @@ public void subscribe(Collection topics, 
ConsumerRebalanceListener callb
 
 @Override
 public void assign(Collection partitions) {
-throw new KafkaException("method not implemented");
+if (partitions == null) {
+throw new IllegalArgumentException("Topic partitions collection to 
assign to cannot be null");
+}
+
+if (partitions.isEmpty()) {
+this.unsubscribe();
+return;
+}
+
+for (TopicPartition tp : partitions) {
+String topic = (tp != null) ? tp.topic() : null;
+if (Utils.isBlank(topic))
+throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
+}
+// TODO: implement fetcher
+// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+commit(subscriptions.allConsumed());

Review Comment:
   Thanks, @lianetm - As autocommit is handled by the commitRequestManager, the 
application thread is only responsible for sending an assignment change event 
to the background thread.  This is tested in the 
`PrototypeAsyncConsumer.testAssign()`.  The autocommit was also tested in the 
request manager.  I would add a test to the DefaultBackgroundThreadTest to test 
if the AssignmentChange event triggers the autocommit.



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



[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-17 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1265809110


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -522,7 +525,35 @@ public void subscribe(Collection topics, 
ConsumerRebalanceListener callb
 
 @Override
 public void assign(Collection partitions) {
-throw new KafkaException("method not implemented");
+if (partitions == null) {
+throw new IllegalArgumentException("Topic partitions collection to 
assign to cannot be null");
+}
+
+if (partitions.isEmpty()) {
+this.unsubscribe();
+return;
+}
+
+for (TopicPartition tp : partitions) {
+String topic = (tp != null) ? tp.topic() : null;
+if (Utils.isBlank(topic))
+throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
+}
+// TODO: implement fetcher
+// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+commit(subscriptions.allConsumed());

Review Comment:
   Hey @junrao - Thanks for catching this.  I made a minor change to the 
implementation that the application thread would send an event (with the topic 
partitions to commit) to the background thread to try to perform an autocommit. 
 I think now it has the same logic as the KafkaConsumer that it would check - 
1. if the commit manager is there (per the explanation above), 2. autocommit is 
enabled or not, and 3. update the auto-commit timer and check if the timer is 
expired. If the timer has expired, send an auto-commit and reset the timer.



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



[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-12 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1261798307


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -522,7 +525,35 @@ public void subscribe(Collection topics, 
ConsumerRebalanceListener callb
 
 @Override
 public void assign(Collection partitions) {
-throw new KafkaException("method not implemented");
+if (partitions == null) {
+throw new IllegalArgumentException("Topic partitions collection to 
assign to cannot be null");
+}
+
+if (partitions.isEmpty()) {
+this.unsubscribe();
+return;
+}
+
+for (TopicPartition tp : partitions) {
+String topic = (tp != null) ? tp.topic() : null;
+if (Utils.isBlank(topic))
+throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
+}
+// TODO: implement fetcher
+// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+commit(subscriptions.allConsumed());

Review Comment:
   Hey @junrao - Sorry I misunderstood your concern.  To your question: If the 
coordinator is not available, the commit request manager won't be built and 
this commit() will be skipped.  In the `RequestManagers.java` we've got
   ```
   if (groupRebalanceConfig != null && groupRebalanceConfig.groupId != null) {
   final GroupState groupState = new 
GroupState(groupRebalanceConfig);
   coordinator = new CoordinatorRequestManager(time,
   logContext,
   retryBackoffMs,
   errorEventHandler,
   groupState.groupId);
   commit = new CommitRequestManager(time, logContext, 
subscriptions, config, coordinator, groupState);
   }
   ```
   
   to your second question: autoCommit is still guarded by the config per this 
line in `CommitRequestManager.java`
   ```
   private void maybeAutoCommit() {
  if (!autoCommitState.isPresent()) {
   return;
   }
  // autocommit otherwise
   }
   ```
   
   On the Consumer API level, we treat sync/async commit to be the same, except 
sync commit waits for the completion of the future.



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



[GitHub] [kafka] philipnee commented on a diff in pull request #13797: KAFKA-14950: implement assign() and assignment()

2023-07-11 Thread via GitHub


philipnee commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1260427182


##
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##
@@ -522,7 +525,35 @@ public void subscribe(Collection topics, 
ConsumerRebalanceListener callb
 
 @Override
 public void assign(Collection partitions) {
-throw new KafkaException("method not implemented");
+if (partitions == null) {
+throw new IllegalArgumentException("Topic partitions collection to 
assign to cannot be null");
+}
+
+if (partitions.isEmpty()) {
+this.unsubscribe();
+return;
+}
+
+for (TopicPartition tp : partitions) {
+String topic = (tp != null) ? tp.topic() : null;
+if (Utils.isBlank(topic))
+throw new IllegalArgumentException("Topic partitions to assign 
to cannot have null or empty topic");
+}
+// TODO: implement fetcher
+// fetcher.clearBufferedDataForUnassignedPartitions(partitions);
+
+// make sure the offsets of topic partitions the consumer is 
unsubscribing from
+// are committed since there will be no following rebalance
+commit(subscriptions.allConsumed());

Review Comment:
   Hey @junrao - I believe we still want to commit the offset before there is a 
partition change. I think if we don't commit offset here, it is possible to 
lose some progress. 



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