lianetm commented on code in PR #16461: URL: https://github.com/apache/kafka/pull/16461#discussion_r1666029689
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManager.java: ########## @@ -0,0 +1,1105 @@ +/* + * 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; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate.PollResult; +import org.apache.kafka.clients.consumer.internals.Utils.TopicIdPartitionComparator; +import org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator; +import org.apache.kafka.clients.consumer.internals.metrics.ShareRebalanceMetricsManager; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryProvider; +import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; + +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * Group manager for a single consumer that has a group id defined in the config + * {@link ConsumerConfig#GROUP_ID_CONFIG} and the share group protocol to get automatically + * assigned partitions when calling the subscribe API. + * <p/> + * + * While the subscribe API hasn't been called (or if the consumer called unsubscribe), this manager + * will only be responsible for keeping the member in the {@link MemberState#UNSUBSCRIBED} state, + * without joining the group. + * <p/> + * + * If the consumer subscribe API is called, this manager will use the {@link #groupId()} to join the + * share group, and based on the share group protocol heartbeats, will handle the full + * lifecycle of the member as it joins the group, reconciles assignments, handles fatal errors, + * and leaves the group. + * <p/> + * + * Reconciliation process:<p/> + * The member accepts all assignments received from the broker, resolves topic names from + * metadata, reconciles the resolved assignments, and keeps the unresolved to be reconciled when + * discovered with a metadata update. Reconciliations of resolved assignments are executed + * sequentially and acknowledged to the server as they complete. The reconciliation process + * involves multiple async operations, so the member will continue to heartbeat while these + * operations complete, to make sure that the member stays in the group while reconciling. + * <p/> + * + * Reconciliation steps: + * <ol> + * <li>Resolve topic names for all topic IDs received in the target assignment. Topic names + * found in metadata are then ready to be reconciled. Topic IDs not found are kept as + * unresolved, and the member request metadata updates until it resolves them (or the broker + * removes it from the target assignment).</li> + * <li>When the above steps complete, the member acknowledges the reconciled assignment, + * which is the subset of the target that was resolved from metadata and actually reconciled. + * The ack is performed by sending a heartbeat request back to the broker.</li> + * </ol> + * + */ +public class ShareMembershipManager implements RequestManager { Review Comment: High level thoughts. I expected very similar components (this and the membershipMgr for consumer groups), but after having a first look at this code, don't we have too much in common from the client state machine POV that we're not reusing? (states, transitions, assignment name resolution logic, reconciliation triggering/flow, stale logic). Particularly painful because I see all the recent/inflight tweaks we are right now working on for the membershipMgr, that I see we would end up having to do here again for sure. As I see it, the main conceptual differences between the managers for each group would be: 1. HB response processing, given that each has different RPC/data 2. revoke and assign flows, given that it's way simpler for share groups with no callbacks/commit (lucky you). Am I missing too much of the differences, high level? If not, an option to consider could be keeping a base manager with the core state machine logic that applies to both (I guess we could have a lot here), and have the specific managers overwriting only what they need different. Thoughts? Kind of half-way between having separate components, and having the same component handling both groups. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
