dajac commented on code in PR #13639:
URL: https://github.com/apache/kafka/pull/13639#discussion_r1205178313


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java:
##########
@@ -0,0 +1,500 @@
+/*
+ * 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.errors.UnknownMemberIdException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.coordinator.group.GroupMetadataManagerTest;
+import org.apache.kafka.image.TopicsImage;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Optional;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ConsumerGroupTest {
+
+    private ConsumerGroup createConsumerGroup(String groupId) {
+        SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new 
LogContext());
+        return new ConsumerGroup(snapshotRegistry, groupId);
+    }
+
+    @Test
+    public void testGetOrCreateMember() {
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+        ConsumerGroupMember member;
+
+        // Create a group.
+        member = consumerGroup.getOrMaybeCreateMember("member-id", true);
+        assertEquals("member-id", member.memberId());
+
+        // Get that group back.
+        member = consumerGroup.getOrMaybeCreateMember("member-id", false);
+        assertEquals("member-id", member.memberId());
+
+        assertThrows(UnknownMemberIdException.class, () ->
+            consumerGroup.getOrMaybeCreateMember("does-not-exist", false));
+    }
+
+    @Test
+    public void testUpdateMember() {
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+        ConsumerGroupMember member;
+
+        member = consumerGroup.getOrMaybeCreateMember("member", true);
+
+        member = new ConsumerGroupMember.Builder(member)
+            .setSubscribedTopicNames(Arrays.asList("foo", "bar"))
+            .build();
+
+        consumerGroup.updateMember(member);
+
+        assertEquals(member, consumerGroup.getOrMaybeCreateMember("member", 
false));
+    }
+
+    @Test
+    public void testRemoveMember() {
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+
+        consumerGroup.getOrMaybeCreateMember("member", true);
+        assertTrue(consumerGroup.hasMember("member"));
+
+        consumerGroup.removeMember("member");
+        assertFalse(consumerGroup.hasMember("member"));
+
+    }
+
+    @Test
+    public void testUpdatingMemberUpdatesPartitionEpoch() {
+        Uuid fooTopicId = Uuid.randomUuid();
+        Uuid barTopicId = Uuid.randomUuid();
+        Uuid zarTopicId = Uuid.randomUuid();
+
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+        ConsumerGroupMember member;
+
+        member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(fooTopicId, 1, 2, 3)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(barTopicId, 4, 5, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(zarTopicId, 7, 8, 9)))
+            .build();
+
+        consumerGroup.updateMember(member);
+
+        assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 1));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 2));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 3));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 4));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 5));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 6));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 7));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 8));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 9));
+
+        member = new ConsumerGroupMember.Builder(member)
+            .setMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(barTopicId, 1, 2, 3)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(zarTopicId, 4, 5, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(fooTopicId, 7, 8, 9)))
+            .build();
+
+        consumerGroup.updateMember(member);
+
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 1));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 2));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 3));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 4));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 5));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 6));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 7));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 8));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9));
+    }
+
+    @Test
+    public void testDeletingMemberRemovesPartitionEpoch() {
+        Uuid fooTopicId = Uuid.randomUuid();
+        Uuid barTopicId = Uuid.randomUuid();
+        Uuid zarTopicId = Uuid.randomUuid();
+
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+        ConsumerGroupMember member;
+
+        member = new ConsumerGroupMember.Builder("member")
+            .setMemberEpoch(10)
+            .setAssignedPartitions(mkAssignment(
+                mkTopicAssignment(fooTopicId, 1, 2, 3)))
+            .setPartitionsPendingRevocation(mkAssignment(
+                mkTopicAssignment(barTopicId, 4, 5, 6)))
+            .setPartitionsPendingAssignment(mkAssignment(
+                mkTopicAssignment(zarTopicId, 7, 8, 9)))
+            .build();
+
+        consumerGroup.updateMember(member);
+
+        assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 1));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 2));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 3));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 4));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 5));
+        assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 6));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 7));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 8));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 9));
+
+        consumerGroup.removeMember(member.memberId());
+
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(barTopicId, 1));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(barTopicId, 2));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(barTopicId, 3));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 4));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 5));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 6));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 7));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 8));
+        assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9));
+    }
+
+    @Test
+    public void testGroupState() {
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+        assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, 
consumerGroup.state());
+
+        ConsumerGroupMember member1 = new 
ConsumerGroupMember.Builder("member1")
+            .setMemberEpoch(1)
+            .setPreviousMemberEpoch(0)
+            .setNextMemberEpoch(1)

Review Comment:
   That's right. There is only one case where it is not. When a member must 
revoke partitions, it stays in its current epoch so member epoch is different 
from the next epoch in this case.
   
   The rational of keeping track of the next epoch here is to basically prevent 
recomputing the state while the member is in revoking state. Without it, we 
would have to recompute it on every heartbeat.



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

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

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

Reply via email to