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


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##########
@@ -790,6 +812,197 @@ public void 
testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup() {
         verify(membershipManager, never()).transitionToJoining();
     }
 
+    @Test
+    public void testListenerCallbacksBasic() {

Review Comment:
   @kirktrue the current implementation ignores the `onPartitionsRevoked` 
callback result 
[here](https://github.com/apache/kafka/blob/bc4e2aaf3381297fbe2e947acac3ed692c7e7da0/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java#L752)
 and continues to `assignPartitions` to invoke `onPartitionsAssigned`, so looks 
like a bug (`onPartitionsRevoked` completion failing differently from what was 
expected?)
   
   That being said, @dajac is bringing out a valid point. We were intentionally 
continuing in this case of failure, just to call both callbacks as the legacy 
logic does, but I don't see any clear value in it. Just failing the 
reconciliation when `onPartitionsRevoked` fails seems more consistent to me: 
the client failed to reconcile (revoke), so it does not carry on with the 
reconciliation, and does not ack to the broker. Stays RECONCILING, with the 
`assignmentReadyToReconcile`, and we should ensure that it is all processed 
again on the next reconciliation loop.
   
   If we agree on that, it means that we should handle the revocation result 
[here](https://github.com/apache/kafka/blob/bc4e2aaf3381297fbe2e947acac3ed692c7e7da0/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java#L752)
 where it is currently ignored, and just do what it is already done when the 
`onPartitionsAssgined` fails 
[here](https://github.com/apache/kafka/blob/bc4e2aaf3381297fbe2e947acac3ed692c7e7da0/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java#L777).
 Makes sense @kirktrue ?



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