chia7712 commented on code in PR #16449:
URL: https://github.com/apache/kafka/pull/16449#discussion_r1665940798


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1490,6 +1490,8 @@ public void unsubscribe() {
                     log.error("Failed while waiting for the unsubscribe event 
to complete");
                 }
                 resetGroupMetadata();
+            } else {

Review Comment:
   inspired by @lianetm comments in jira. Maybe we can leverage 
`UnsubscribeEvent` to handle the `unsubscribe`. For example:
   
   ```java
       public void unsubscribe() {
           acquireAndEnsureOpen();
           try {
               fetchBuffer.retainAll(Collections.emptySet());
               Timer timer = time.timer(Long.MAX_VALUE);
               UnsubscribeEvent unsubscribeEvent = new 
UnsubscribeEvent(calculateDeadlineMs(timer));
               applicationEventHandler.add(unsubscribeEvent);
               log.info("Unsubscribing all topics or patterns and assigned 
partitions {}",
                       subscriptions.assignedPartitions());
               try {
                   processBackgroundEvents(unsubscribeEvent.future(), timer);
                   log.info("Unsubscribed all topics or patterns and assigned 
partitions");
               } catch (TimeoutException e) {
                   log.error("Failed while waiting for the unsubscribe event to 
complete");
               }
               if (groupMetadata.get().isPresent()) resetGroupMetadata();
           } catch (Exception e) {
               log.error("Unsubscribe failed", e);
               throw e;
           } finally {
               release();
           }
       }
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to