You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "dajac (via GitHub)" <gi...@apache.org> on 2023/04/25 12:32:33 UTC

[GitHub] [kafka] dajac opened a new pull request, #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

dajac opened a new pull request, #13638:
URL: https://github.com/apache/kafka/pull/13638

   Built on top of https://github.com/apache/kafka/pull/13544. This patch adds CurrentAssignmentBuilder.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183887136


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   >   * Checks whether the owned topic partitions passed by the member to the state
        * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
        * set.
        
   This comment confuses me. Are the owned topic partitions passed by the member simply (original set - the partitions we want to remove)? 
   
   I guess I'm confused where we decide which to assign. I guess we have assigned and assigning sets. The comment about "assigned set" confused me. 
   
   > The set of partitions currently assigned to the member. This represents what the member should have
   
   Is it correct that this set is actually just the set we should have after revoking? (But not before we add the assigning set?)



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183933777


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.

Review Comment:
   Will do.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac merged pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac merged PR #13638:
URL: https://github.com/apache/kafka/pull/13638


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1179738264


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch
+ *                    when it has revoked the partitions that it does not owned or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it could transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions

Review Comment:
   The name is a bit confusing imo, we might want to change it to pending or awaiting to show that we're waiting on these partitions to finish the assignment? Just an idea lmk what you think



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184368057


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   Yes intersect not union. 🤦‍♀️ Let me fix my comment
   



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183933309


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   The owned partitions are provided in the ConsumerGroupHeartbeat request. They correspond to what the member currently has assigned to it (from his point of view). If this corresponds to the assigned partitions, it means that the member is done with revoking the partitions.
   
   The assigned partition set is the set of partitions currently assigned to the member. Those are the partitions being processed/fetched by the member.
   
   I renamed `assigning set` to `partitions pending assignment` to make thing clearer. I hope it helps.
   
   The partitions pending assignment are the partitions that the member will eventually receive when it has confirmed that it has revoked the partitions that it no longer owns. The pending partitions will be assigned to the member if they are not owned by any other members or when they are freed up.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184321022


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   Ack. I think in my mind, the distinction was when we reached the final target state. But really stage 1 is getting to the state where we remove all unnecessary partitions and then bump the epoch. Then we proceed to stage 2 where we assign the ones that are missing. I think I was viewing "assigned" as the final state, but really this is just the final state of stage 1. (and target will be the final state of stage 2)



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184669415


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(Collections.emptyList()) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromAssigningToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(11)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> {
+                if (topicId.equals(topicId1))
+                    return -1;
+                else
+                    return 10;
+            })
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromAssigningToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(11)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromStableToStable() {

Review Comment:
   Indeed, it does not seem right. Let me fix it.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184401478


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(Collections.emptyList()) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromAssigningToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(11)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> {
+                if (topicId.equals(topicId1))
+                    return -1;
+                else
+                    return 10;
+            })
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromAssigningToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(11)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromStableToStable() {

Review Comment:
   Is this the right test? Seems like a repeat of the above. 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185475014


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    private static Stream<Arguments> ownedTopicPartitionsArguments() {
+        return Stream.of(
+            // Field not set in the heartbeat request.
+            null,
+            // Owned partitions does not match the assigned partitions.
+            Collections.emptyList()
+        ).map(Arguments::of);
+    }
+
+    @ParameterizedTest
+    @MethodSource("ownedTopicPartitionsArguments")
+    public void testTransitionFromRevokeToRevoke(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(ownedTopicPartitions)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());

Review Comment:
   can you point me to where the original pending assignments 
   ```
                   mkTopicAssignment(topicId1, 4, 5)
                   mkTopicAssignment(topicId2, 7, 8)
   ```
   were actually assigned? i guess i'm confused because L305-307 only contains the owned partitions



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185172881


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we

Review Comment:
   i think it would be good to mention this in one of the comment sections (here or in the class javadoc) and link the MemberState.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184651164


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   Let me use the correct symbol: `∩`.
   
   > So assigned is what we already had intersect with the target,
   revoking is what we had minus what we now have assigned
   and assigning is target - what we now have assigned?
   
   Correct.
   
   > Also, this is really minor, but why is the set not called a set and the map is called set? 😅
   
   I have removed all those Set/Map suffixes, I think.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184383745


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {

Review Comment:
   Could we parameterize this and the above test?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184677397


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(Collections.emptyList()) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {

Review Comment:
   Nope. Let me add it.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1177444040


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch

Review Comment:
   nit: transitions*



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183280551


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());

Review Comment:
   Which part are your referring to?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183277863


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch
+ *                    when it has revoked the partitions that it does not owned or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it could transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions

Review Comment:
   Renamed it.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183163284


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   Also would we expect this to match assigned set after we assign? We call this before we assign.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183161313


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   So these next lines are taking the previous state and generating the new state based on it? 
   Basically joining the two updates



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185749017


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)

Review Comment:
   you're right. good catch!



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1187582387


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The epoch of the member when the state was last updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:

Review Comment:
   I see. I guess it is a little confusing we have this as an all caps state, but don't list in in MemberState. Just wondering if it would be better to include a comment about the transient state when we transition or leave it out altogether



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1187640544


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The epoch of the member when the state was last updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:

Review Comment:
   makes sense thanks!
   



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184317038


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we

Review Comment:
   Is this transition done in the response or somewhere else?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184405678


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(Collections.emptyList()) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {

Review Comment:
   Do we have a test where the original state was to revoke some partitions and the target becomes those same revoked ones (so we should go to stable I think)?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184644685


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.

Review Comment:
   Done.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183160178


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.

Review Comment:
   Ah I see in line 163 is one point that addresses this.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183278628


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still

Review Comment:
   The coordinator is actually assigning partitions to the member in this state.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183282261


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.

Review Comment:
   That's correct. Previous and next epoch could have bigger deltas.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184654172


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we

Review Comment:
   It is done by the code right after. The member automatically transitions to Revoking when `newPartitionsPendingRevocation` is not empty.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185188619


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());

Review Comment:
   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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1186823970


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The epoch of the member when the state was last updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:

Review Comment:
   We do but it is a transient state so it is not part of the member's states. We transition to it [here](https://github.com/apache/kafka/pull/13638/files/0395a1780acd73e735bfee21da733075db85504d#diff-6c5e23803064f4b7a122eee29736d036b9cfe244e69f48751ba163d62e2bf35fR176).



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183279847


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {

Review Comment:
   Correct. This is to guarantee that offsets of revoked partitions are committed with the current member epoch. I have added a comment in the top level javadoc about this.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183282744


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   Right. It is a bit more than joining though :).



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184300900


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   I'm still struggling a bit -- I think /\ means union here? That would make things make sense.
   So assigned is what we already had union with the target,
   revoking is what we had minus what we now have assigned
   and assigning is target - what we now have assigned?
   
   Also, this is really minor, but why is the set not called a set and the map is called set? 😅 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184300900


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   I'm still struggling a bit -- I think /\ means union here? 
   So assigned is what we already had union with the target,
   revoking is what we had minus what we now have assigned
   and assigning is target - what we now have assigned?
   
   Also, this is really minor, but why is the set not called a set and the map is called set? 😅 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13638:
URL: https://github.com/apache/kafka/pull/13638#issuecomment-1532521629

   @jolshan @CalvinConfluent @rreddy-22 Thanks for your comments. I have addressed them.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184666684


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {

Review Comment:
   Seems possible.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185168753


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:
+ *   This is the initial state of the state machine. The state machine starts here when the next epoch
+ *   does not match the target epoch. It means that a new target assignment has been installed so the
+ *   reconciliation process must restart. In this state, the Assigned, Revoking and Assigning sets are
+ *   computed. If Revoking is not empty, the state machine transitions to REVOKING; if Assigning is not
+ *   empty, it transitions to ASSIGNING; otherwise it transitions to STABLE.
+ *
+ * - REVOKING:
+ *   This state means that the member must revoke partitions before it can transition to the next epoch
+ *   and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
+ *   are committed with the current epoch. The member transitions to the next state only when it has
+ *   acknowledged the revocation.
+ *
+ * - ASSIGNING:
+ *   This state means that the member waits on partitions which are still owned by other members in the
+ *   group. It remains in this state until they are all freed up.
+ *
+ * - STABLE:
+ *   This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {

Review Comment:
   initializing implies we're clearing out everything and going back to some base form. However, i don't think that makes sense in this case as we're moving towards one of the states. i would prefer `recomputeState` 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1177447512


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch
+ *                    when it has revoked the partitions that it does not owned or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it could transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions

Review Comment:
   Does this contain all the target partitions or only the ones which were owned previously? 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1177444507


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch
+ *                    when it has revoked the partitions that it does not owned or if it

Review Comment:
   nit: own*



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183162775


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   Yeah -- if we are checking the correct state after revoking, this name is a bit confusing



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184358012


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:

Review Comment:
   maybe we can link the ConsumerGroupMember.MemberState docs to this?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:
+ *   This is the initial state of the state machine. The state machine starts here when the next epoch
+ *   does not match the target epoch. It means that a new target assignment has been installed so the
+ *   reconciliation process must restart. In this state, the Assigned, Revoking and Assigning sets are
+ *   computed. If Revoking is not empty, the state machine transitions to REVOKING; if Assigning is not
+ *   empty, it transitions to ASSIGNING; otherwise it transitions to STABLE.
+ *
+ * - REVOKING:
+ *   This state means that the member must revoke partitions before it can transition to the next epoch
+ *   and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
+ *   are committed with the current epoch. The member transitions to the next state only when it has
+ *   acknowledged the revocation.
+ *
+ * - ASSIGNING:
+ *   This state means that the member waits on partitions which are still owned by other members in the
+ *   group. It remains in this state until they are all freed up.
+ *
+ * - STABLE:
+ *   This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {

Review Comment:
   i think the "initial state" is misleading. this method, after computing the sets will transition to revoking/assigning/stable right?
   
   how's `computePartitionSetsAndTransitionState()`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   i was confused on the `/\`. maybe saying intersect might be clearer.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13638:
URL: https://github.com/apache/kafka/pull/13638#issuecomment-1534267271

   @jeffkbkim @jolshan Thanks for your comments. I have addressed them.


-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183278201


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what

Review Comment:
   This is the set of partitions currently assigned to or owned by the member.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183155377


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.

Review Comment:
   I'm assuming this isn't current epoch -1 but instead the epoch for when the member last updated? 
   Likewise, next epoch is not current epoch + 1?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185460028


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)

Review Comment:
   should this be with epoch 11 so that we recompute from initial state?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1177445627


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch
+ *                    when it has revoked the partitions that it does not owned or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it could transition

Review Comment:
   nit: can*



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1179733530


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what

Review Comment:
   Is this the target set or the previously assigned set?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183282063


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   @CalvinConfluent Your understanding is correct.
   @jolshan No. This is the condition to transition to Assigning state. We basically want to consumer to ack'ed that it has revoked partitions before we assign new partitions.
   
   Let me see if I can rename it. I called it `hasRevokedAllPartitions` in the first place because this is actually what we want from the member.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183887136


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   >   * Checks whether the owned topic partitions passed by the member to the state
        * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
        * set.
        
   This comment confuses me. Are the owned topic partitions passed by the member simply (current set - the partitions we want to remove)? 
   
   I guess I'm confused where we decide which to assign. I guess we have assigned and assigning sets. The comment about "assigned set" confused me. 
   
   > The set of partitions currently assigned to the member. This represents what the member should have
   
   Is it correct that this set is actually just the set we should have after revoking? (But not before we add the assigning set?)



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184655502


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:
+ *   This is the initial state of the state machine. The state machine starts here when the next epoch
+ *   does not match the target epoch. It means that a new target assignment has been installed so the
+ *   reconciliation process must restart. In this state, the Assigned, Revoking and Assigning sets are
+ *   computed. If Revoking is not empty, the state machine transitions to REVOKING; if Assigning is not
+ *   empty, it transitions to ASSIGNING; otherwise it transitions to STABLE.
+ *
+ * - REVOKING:
+ *   This state means that the member must revoke partitions before it can transition to the next epoch
+ *   and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
+ *   are committed with the current epoch. The member transitions to the next state only when it has
+ *   acknowledged the revocation.
+ *
+ * - ASSIGNING:
+ *   This state means that the member waits on partitions which are still owned by other members in the
+ *   group. It remains in this state until they are all freed up.
+ *
+ * - STABLE:
+ *   This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {

Review Comment:
   How about `reinitializeState`?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184323192


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());

Review Comment:
   I think maybe he was referring to lines 276 and 277?
   
   Just to confirm, do we need to do a deep copy because the map can change? 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185204551


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we

Review Comment:
   Updated the comments. However, I feel like the link to MemberState is not really needed as we already explain all the states in the main javadoc.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1186307624


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The epoch of the member when the state was last updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:

Review Comment:
   Do we ever use NEW_TARGET_ASSIGNMENT state? I don't see it in the code anywhere else.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185195017


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:
+ *   This is the initial state of the state machine. The state machine starts here when the next epoch
+ *   does not match the target epoch. It means that a new target assignment has been installed so the
+ *   reconciliation process must restart. In this state, the Assigned, Revoking and Assigning sets are
+ *   computed. If Revoking is not empty, the state machine transitions to REVOKING; if Assigning is not
+ *   empty, it transitions to ASSIGNING; otherwise it transitions to STABLE.
+ *
+ * - REVOKING:
+ *   This state means that the member must revoke partitions before it can transition to the next epoch
+ *   and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
+ *   are committed with the current epoch. The member transitions to the next state only when it has
+ *   acknowledged the revocation.
+ *
+ * - ASSIGNING:
+ *   This state means that the member waits on partitions which are still owned by other members in the
+ *   group. It remains in this state until they are all freed up.
+ *
+ * - STABLE:
+ *   This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {

Review Comment:
   I will use `transitionToNewTargetAssignmentState`.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185752497


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    private static Stream<Arguments> ownedTopicPartitionsArguments() {
+        return Stream.of(
+            // Field not set in the heartbeat request.
+            null,
+            // Owned partitions does not match the assigned partitions.
+            Collections.emptyList()
+        ).map(Arguments::of);
+    }
+
+    @ParameterizedTest
+    @MethodSource("ownedTopicPartitionsArguments")
+    public void testTransitionFromRevokeToRevoke(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(ownedTopicPartitions)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());

Review Comment:
   you have to decompose it in two steps:
   1) The partitions pending revocation are acknowledged by the owned partitions at L305. This allows a transition to assigning state.
   2) At the partitions pending assignment are free (L304), they are directly assigned to the member and the member transitions to stable. This is this [method](https://github.com/apache/kafka/pull/13638/files#diff-6c5e23803064f4b7a122eee29736d036b9cfe244e69f48751ba163d62e2bf35fR293).



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183889134


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.

Review Comment:
   Could we change the comment to something like "The epoch of the member when the state was last 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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184401478


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java:
##########
@@ -0,0 +1,548 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment;
+import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class CurrentAssignmentBuilderTest {
+
+    @Test
+    public void testTransitionFromNewTargetToRevoke() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
+            mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromNewTargetToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2, 3),
+                mkTopicAssignment(topicId2, 4, 5, 6)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(10, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(10, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2, 3),
+            mkTopicAssignment(topicId2, 4, 5, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithNull() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(null) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToRevokeWithEmptyList() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(Collections.emptyList()) // The client has not revoked yet.
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(10, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 1, 2),
+            mkTopicAssignment(topicId2, 4, 5)
+        ), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> 10)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 4, 5),
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromRevokeToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(10)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(topicId1, 1, 2),
+                mkTopicAssignment(topicId2, 4, 5)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6))))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromAssigningToAssigning() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(11)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> {
+                if (topicId.equals(topicId1))
+                    return -1;
+                else
+                    return 10;
+            })
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId2, 7, 8)
+        ), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromAssigningToStable() {
+        Uuid topicId1 = Uuid.randomUuid();
+        Uuid topicId2 = Uuid.randomUuid();
+
+        ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setPreviousMemberEpoch(11)
+            .setNextMemberEpoch(11)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(topicId1, 3),
+                mkTopicAssignment(topicId2, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(topicId1, 4, 5),
+                mkTopicAssignment(topicId2, 7, 8)))
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
+
+        Assignment targetAssignment = new Assignment(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ));
+
+        ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
+            .withTargetAssignment(11, targetAssignment)
+            .withCurrentPartitionEpoch((topicId, partitionId) -> -1)
+            .build();
+
+        assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
+        assertEquals(10, updatedMember.previousMemberEpoch());
+        assertEquals(11, updatedMember.memberEpoch());
+        assertEquals(11, updatedMember.nextMemberEpoch());
+        assertEquals(mkAssignment(
+            mkTopicAssignment(topicId1, 3, 4, 5),
+            mkTopicAssignment(topicId2, 6, 7, 8)
+        ), updatedMember.assignedPartitions());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
+        assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
+    }
+
+    @Test
+    public void testTransitionFromStableToStable() {

Review Comment:
   Is this the right test? Seems like a repeat of the above.  I suppose 452 is different, but I would guess from the title that 463 should be STABLE right?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jolshan commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1184300900


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target

Review Comment:
   I'm still struggling a bit -- I think /\ means intersect here? That would make things make sense.
   So assigned is what we already had intersect with the target,
   revoking is what we had minus what we now have assigned
   and assigning is target - what we now have assigned?
   
   Also, this is really minor, but why is the set not called a set and the map is called set? 😅 



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1185193708


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:
+ *   This is the initial state of the state machine. The state machine starts here when the next epoch
+ *   does not match the target epoch. It means that a new target assignment has been installed so the
+ *   reconciliation process must restart. In this state, the Assigned, Revoking and Assigning sets are
+ *   computed. If Revoking is not empty, the state machine transitions to REVOKING; if Assigning is not
+ *   empty, it transitions to ASSIGNING; otherwise it transitions to STABLE.
+ *
+ * - REVOKING:
+ *   This state means that the member must revoke partitions before it can transition to the next epoch
+ *   and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
+ *   are committed with the current epoch. The member transitions to the next state only when it has
+ *   acknowledged the revocation.
+ *
+ * - ASSIGNING:
+ *   This state means that the member waits on partitions which are still owned by other members in the
+ *   group. It remains in this state until they are all freed up.
+ *
+ * - STABLE:
+ *   This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {

Review Comment:
   Thinking a little more about it, I think that `transitionToInitialState` was not too bad after all. In the main javadoc, we clearly define the initial state and explain what it does. I agree that it is a transient state but keeping the name aligned with the doc makes sense.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,436 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The previous epoch of the member when the state was updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:
+ *   This is the initial state of the state machine. The state machine starts here when the next epoch
+ *   does not match the target epoch. It means that a new target assignment has been installed so the
+ *   reconciliation process must restart. In this state, the Assigned, Revoking and Assigning sets are
+ *   computed. If Revoking is not empty, the state machine transitions to REVOKING; if Assigning is not
+ *   empty, it transitions to ASSIGNING; otherwise it transitions to STABLE.
+ *
+ * - REVOKING:
+ *   This state means that the member must revoke partitions before it can transition to the next epoch
+ *   and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
+ *   are committed with the current epoch. The member transitions to the next state only when it has
+ *   acknowledged the revocation.
+ *
+ * - ASSIGNING:
+ *   This state means that the member waits on partitions which are still owned by other members in the
+ *   group. It remains in this state until they are all freed up.
+ *
+ * - STABLE:
+ *   This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {

Review Comment:
   Thinking a little more about it, I think that `transitionToInitialState` was not too bad after all. In the main javadoc, we clearly define the initial state and explain what it does. I agree that it is a transient state but keeping the name aligned with the doc makes sense.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] CalvinConfluent commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "CalvinConfluent (via GitHub)" <gi...@apache.org>.
CalvinConfluent commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1183136736


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {

Review Comment:
   So before all the revoked partitions have been removed, we will not move the assigning partitions to assigned?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+        Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+        // If any partition can transition from assigning to assigned, we update
+        // the member. Otherwise, we return the current one.
+        if (maybeAssignPendingPartitions(newAssignedSet, newAssigningSet)) {
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Makes a deep copy of an assignment map.
+     *
+     * @param map The Map to copy.
+     * @return The copy.
+     */
+    private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
+        Map<Uuid, Set<Integer>> copy = new HashMap<>();
+        map.forEach((topicId, partitions) -> {
+            copy.put(topicId, new HashSet<>(partitions));
+        });
+        return copy;
+    }
+
+    /**
+     * Tries to move partitions from the Assigning set to the Assigned set
+     * if they are no longer owned.
+     *
+     * @param newAssignedSet  The Assigned set.
+     * @param newAssigningSet The Assigning set.
+     * @return A boolean indicating if any partitions were moved.
+     */
+    private boolean maybeAssignPendingPartitions(
+        Map<Uuid, Set<Integer>> newAssignedSet,
+        Map<Uuid, Set<Integer>> newAssigningSet
+    ) {
+        boolean changed = false;
+
+        Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator = newAssigningSet.entrySet().iterator();
+        while (assigningSetIterator.hasNext()) {
+            Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
+            Uuid topicId = pair.getKey();
+            Set<Integer> assigning = pair.getValue();
+            Iterator<Integer> assigningIterator = assigning.iterator();
+            while (assigningIterator.hasNext()) {
+                Integer partitionId = assigningIterator.next();
+                Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
+                if (partitionEpoch == -1) {
+                    assigningIterator.remove();
+                    put(newAssignedSet, topicId, partitionId);
+                    changed = true;
+                }
+            }
+            if (assigning.isEmpty()) {
+                assigningSetIterator.remove();
+            }
+        }
+
+        return changed;
+    }
+
+    /**
+     * Checks whether the owned topic partitions passed by the member to the state
+     * machine via the ConsumerGroupHeartbeat request corresponds to the Assigned
+     * set.
+     *
+     * @param ownedTopicPartitions The topic partitions owned by the remove client.
+     * @return A boolean indicating if the owned partitions matches the Assigned set.
+     */
+    private boolean hasRevokedAllPartitions(

Review Comment:
   From the name of hasRevokedAllPartitions seems it checks whether a client removed all the partitions but it actually checks whether a client owns the correct set of assigned partitions. Do I misunderstand too much?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still
+ *                           owned by other members in the group. It remains in this state until
+ *                           they are all freed up.
+ * - STABLE                - This state means that the member has received all its assigned partitions.
+ */
+public class CurrentAssignmentBuilder {
+    /**
+     * The consumer group member which is reconciled.
+     */
+    private final ConsumerGroupMember member;
+
+    /**
+     * The target assignment epoch.
+     */
+    private int targetAssignmentEpoch;
+
+    /**
+     * The target assignment.
+     */
+    private Assignment targetAssignment;
+
+    /**
+     * A function which returns the current epoch of a topic-partition or -1 if the
+     * topic-partition is not assigned. The current epoch is the epoch of the current owner.
+     */
+    private BiFunction<Uuid, Integer, Integer> currentPartitionEpoch;
+
+    /**
+     * The partitions owned by the consumer. This is directly provided by the member in the
+     * ConsumerGroupHeartbeat request.
+     */
+    private List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions;
+
+    /**
+     * Constructs the CurrentAssignmentBuilder based on the current state of the
+     * provided consumer group member.
+     *
+     * @param member The consumer group member that must be reconciled.
+     */
+    public CurrentAssignmentBuilder(ConsumerGroupMember member) {
+        this.member = Objects.requireNonNull(member);
+    }
+
+    /**
+     * Sets the target assignment epoch and the target assignment that the
+     * consumer group member must be reconciled to.
+     *
+     * @param targetAssignmentEpoch The target assignment epoch.
+     * @param targetAssignment      The target assignment.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withTargetAssignment(
+        int targetAssignmentEpoch,
+        Assignment targetAssignment
+    ) {
+        this.targetAssignmentEpoch = targetAssignmentEpoch;
+        this.targetAssignment = Objects.requireNonNull(targetAssignment);
+        return this;
+    }
+
+    /**
+     * Sets a BiFunction which allows to retrieve the current epoch of a
+     * partition. This is used by the state machine to determine if a
+     * partition is free or still used by another member.
+     *
+     * @param currentPartitionEpoch A BiFunction which gets the epoch of a
+     *                              topic id / partitions id pair.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withCurrentPartitionEpoch(
+        BiFunction<Uuid, Integer, Integer> currentPartitionEpoch
+    ) {
+        this.currentPartitionEpoch = Objects.requireNonNull(currentPartitionEpoch);
+        return this;
+    }
+
+    /**
+     * Sets the partitions currently owned by the member. This comes directly
+     * from the last ConsumerGroupHeartbeat request. This is used to determine
+     * if the member has revoked the necessary partitions.
+     *
+     * @param ownedTopicPartitions A list of topic-partitions.
+     * @return This object.
+     */
+    public CurrentAssignmentBuilder withOwnedTopicPartitions(
+        List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
+    ) {
+        this.ownedTopicPartitions = ownedTopicPartitions;
+        return this;
+    }
+
+    /**
+     * Builds the next state for the member or keep the current one if it
+     * is not possible to move forward with the current state.
+     *
+     * @return A new ConsumerGroupMember or the current one.
+     */
+    public ConsumerGroupMember build() {
+        // A new target assignment has been installed, we need to restart
+        // the reconciliation loop from the beginning.
+        if (targetAssignmentEpoch != member.nextMemberEpoch()) {
+            return transitionToInitialState();
+        }
+
+        switch (member.state()) {
+            // Check if the partitions have been revoked by the member.
+            case REVOKING:
+                return maybeTransitionFromRevokingToAssigningOrStable();
+
+            // Check if pending partitions have been freed up.
+            case ASSIGNING:
+                return maybeTransitionFromAssigningToAssigningOrStable();
+
+            // Nothing to do.
+            case STABLE:
+                return member;
+        }
+
+        return member;
+    }
+
+    /**
+     * Transitions to the initial state. Here we compute the Assigned,
+     * Revoking and Assigning sets.
+     *
+     * @return A new ConsumerGroupMember.
+     */
+    private ConsumerGroupMember transitionToInitialState() {
+        Map<Uuid, Set<Integer>> newAssignedSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newRevokingSet = new HashMap<>();
+        Map<Uuid, Set<Integer>> newAssigningSet = new HashMap<>();
+
+        // Compute the combined set of topics.
+        Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
+        allTopicIds.addAll(member.assignedPartitions().keySet());
+        allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
+        allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
+
+        for (Uuid topicId : allTopicIds) {
+            Set<Integer> target = targetAssignment.partitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentAssignedPartitions = member.assignedPartitions()
+                .getOrDefault(topicId, Collections.emptySet());
+            Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
+                .getOrDefault(topicId, Collections.emptySet());
+
+            // Assigned_1 = (Assigned_0 + Revoking_0) /\ Target
+            Set<Integer> newAssignedPartitions = new HashSet<>(currentAssignedPartitions);
+            newAssignedPartitions.addAll(currentRevokingPartitions);
+            newAssignedPartitions.retainAll(target);
+
+            // Revoking_1 = (Assigned_0 + Revoking_0) - Assigned_1
+            Set<Integer> newRevokingPartitions = new HashSet<>(currentAssignedPartitions);
+            newRevokingPartitions.addAll(currentRevokingPartitions);
+            newRevokingPartitions.removeAll(newAssignedPartitions);
+
+            // Assigning_1 = Target - Assigned_1
+            Set<Integer> newAssigningPartitions = new HashSet<>(target);
+            newAssigningPartitions.removeAll(newAssignedPartitions);
+
+            if (!newAssignedPartitions.isEmpty()) {
+                newAssignedSet.put(topicId, newAssignedPartitions);
+            }
+
+            if (!newRevokingPartitions.isEmpty()) {
+                newRevokingSet.put(topicId, newRevokingPartitions);
+            }
+
+            if (!newAssigningPartitions.isEmpty()) {
+                newAssigningSet.put(topicId, newAssigningPartitions);
+            }
+        }
+
+        if (!newRevokingSet.isEmpty()) {
+            // If the revoking set is not empty, we transition to Revoking and we
+            // stay in the current epoch.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(newRevokingSet)
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        }
+    }
+
+    /**
+     * Tries to transition from Revoke to Assigning or Stable. This is only
+     * possible when the member acknowledges that it only owns the partition
+     * in the Assigned set.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
+        if (member.partitionsPendingRevocation().isEmpty() || hasRevokedAllPartitions(ownedTopicPartitions)) {
+            Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());
+            Map<Uuid, Set<Integer>> newAssigningSet = deepCopy(member.partitionsPendingAssignment());
+
+            if (!newAssigningSet.isEmpty()) {
+                // If the assigning set is not empty, we check if some or all
+                // partitions are free to use. If they are, we move them to
+                // the assigned set.
+                maybeAssignPendingPartitions(newAssignedSet, newAssigningSet);
+            }
+
+            // We transition to the target epoch. If the assigning set is empty,
+            // the member transition to stable, otherwise to assigning.
+            return new ConsumerGroupMember.Builder(member)
+                .setAssignedPartitions(newAssignedSet)
+                .setPartitionsPendingRevocation(Collections.emptyMap())
+                .setPartitionsPendingAssignment(newAssigningSet)
+                .setPreviousMemberEpoch(member.memberEpoch())
+                .setMemberEpoch(targetAssignmentEpoch)
+                .setNextMemberEpoch(targetAssignmentEpoch)
+                .build();
+        } else {
+            return member;
+        }
+    }
+
+    /**
+     * Tries to transition from Assigning to Assigning or Stable. This is only
+     * possible when one or more partitions in the Assigning set have been freed
+     * up by other members in the group.
+     *
+     * @return A new ConsumerGroupMember with the new state or the current one
+     *         if the member stays in the current state.
+     */
+    private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
+        Map<Uuid, Set<Integer>> newAssignedSet = deepCopy(member.assignedPartitions());

Review Comment:
   This part can be reused by the maybeTransitionFromRevokingToAssigningOrStable, did we consider that?



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1187624902


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch:
+ *   The current epoch of the member.
+ *
+ * - Next Epoch:
+ *   The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
+ *   The member transitions to this epoch when it has revoked the partitions that it does not own
+ *   or if it does not have to revoke any.
+ *
+ * - Previous Epoch:
+ *   The epoch of the member when the state was last updated.
+ *
+ * - Assigned Partitions:
+ *   The set of partitions currently assigned to the member. This represents what the member should have.
+ *
+ * - Partitions Pending Revocation:
+ *   The set of partitions that the member should revoke before it can transition to the next state.
+ *
+ * - Partitions Pending Assignment:
+ *   The set of partitions that the member will eventually receive. The partitions in this set are
+ *   still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT:

Review Comment:
   Updated the comment. Let me know what you think.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dajac commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1179091537


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transition to this epoch
+ *                    when it has revoked the partitions that it does not owned or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it could transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions

Review Comment:
   It contains only the partitions that are pending -- still owned by other members.



-- 
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-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13638: KAFKA-14462; [11/N] Add CurrentAssignmentBuilder

Posted by "rreddy-22 (via GitHub)" <gi...@apache.org>.
rreddy-22 commented on code in PR #13638:
URL: https://github.com/apache/kafka/pull/13638#discussion_r1179739941


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.coordinator.group.consumer;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
+ * consumer group protocol. Given the current state of a member and a desired or target
+ * assignment state, the state machine takes the necessary steps to converge them.
+ *
+ * The member state has the following properties:
+ * - Current Epoch  - The current epoch of the member.
+ * - Next Epoch     - The desired epoch of the member. It corresponds to the epoch of
+ *                    the target/desired assignment. The member transitions to this epoch
+ *                    when it has revoked the partitions that it does not own or if it
+ *                    does not have to revoke any.
+ * - Previous Epoch - The previous epoch of the member when the state was updated.
+ * - Assigned Set   - The set of partitions currently assigned to the member. This represents what
+ *                    the member should have.
+ * - Revoking Set   - The set of partitions that the member should revoke before it can transition
+ *                    to the next state.
+ * - Assigning Set  - The set of partitions that the member will eventually receive. The partitions
+ *                    in this set are still owned by other members in the group.
+ *
+ * The state machine has four states:
+ * - NEW_TARGET_ASSIGNMENT - This is the initial state of the state machine. The state machine starts
+ *                           here when the next epoch does not match the target epoch. It means that
+ *                           a new target assignment has been installed so the reconciliation process
+ *                           must restart. In this state, the Assigned, Revoking and Assigning sets
+ *                           are computed. If Revoking is not empty, the state machine transitions
+ *                           to REVOKE; if Assigning is not empty, it transitions to ASSIGNING;
+ *                           otherwise it transitions to STABLE.
+ * - REVOKE                - This state means that the member must revoke partitions before it can
+ *                           transition to the next epoch and thus start receiving new partitions.
+ *                           The member transitions to the next state only when it has acknowledged
+ *                           the revocation.
+ * - ASSIGNING             - This state means that the member waits on partitions which are still

Review Comment:
   same here, assigning gives the impression that the member is assigning something? 



-- 
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-unsubscribe@kafka.apache.org

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