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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerRebalanceListenerCallbackNeededEvent.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.internals.events;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import 
org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerMethodName;
+import org.apache.kafka.common.TopicPartition;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.SortedSet;
+
+/**
+ * Event that signifies that the network I/O thread wants to invoke one of the 
callback methods on the
+ * {@link ConsumerRebalanceListener}. This event will be processed by the 
application thread when the next
+ * {@link Consumer#poll(Duration)} call is performed by the user. When 
processed, the application thread should
+ * invoke the appropriate callback method (based on {@link #methodName()}) 
with the given partitions.
+ */
+public class ConsumerRebalanceListenerCallbackNeededEvent extends 
BackgroundEvent {
+
+    private final ConsumerRebalanceListenerMethodName methodName;
+    private final SortedSet<TopicPartition> partitions;
+
+    public 
ConsumerRebalanceListenerCallbackNeededEvent(ConsumerRebalanceListenerMethodName
 methodName,

Review Comment:
   I approached this change with the idea of making the event hold a list of 
invocations such that the `ApplicationEventProcessor` could simply loop over 
them to invoke callbacks serially. But looking more closely at 
[`ConsumerCoordinator`](https://github.com/apache/kafka/blob/2b99d0e45027c88ae2347fa8f7d1ff4b2b919089/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L397-L420)
 shows that there are some steps that need to be executed between those two 
invocations:
   
   1. Invoke `ConsumerRebalanceListener.onPartitionsRevoked()`
   2. Invoke `SubscriptionState.subscribeFromPattern()` if there are patterns 
involved (which we're totally ignoring at present)
   3. Invoke `ConsumerPartitionAssignor.onAssignment()`
   4. Update the auto-commit timer (which isn't presently available in the 
application thread)
   5. Invoke `SubscriptionState.assignFromSubscribed()`
   6. Invoke `ConsumerRebalanceListener.onPartitionsAssigned()`
   
   If we were to perform both invocations in the application thread, we would 
need to execute steps 1-6 during the processing of the 
`ConsumerRebalanceListenerCallbackNeededEvent`. It's not necessarily that much 
code, but there are two main downsides:
   
   1. The clear separation of flow, logic, and data used by the reconciliation 
logic would no longer reside just in `MembershipManagerImpl`, but would spill 
over into `ApplicationEventProcessor`
   2. The current event is generalized to support execution of _any_ of the 
three `ConsumerRebalanceListener` methods. We would need to replace that with 
the more specific processing listed in the six steps above. We would then need 
to add at least one additional event/processor method to handle the 
`onPartitionsLost` case.
   
   In the end "it's just code," so we can change it however we want. Given the 
deadline, I would recommend mulling this over after we have everything 
functionally correct.



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