jolshan commented on code in PR #14067:
URL: https://github.com/apache/kafka/pull/14067#discussion_r1275334511


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java:
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
+import org.apache.kafka.common.errors.GroupIdNotFoundException;
+import org.apache.kafka.common.errors.IllegalGenerationException;
+import org.apache.kafka.common.errors.RebalanceInProgressException;
+import org.apache.kafka.common.errors.StaleMemberEpochException;
+import org.apache.kafka.common.errors.UnknownMemberIdException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.JoinGroupRequestData;
+import org.apache.kafka.common.message.OffsetCommitRequestData;
+import org.apache.kafka.common.message.OffsetCommitResponseData;
+import org.apache.kafka.common.network.ClientInformation;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.RequestContext;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
+import org.apache.kafka.coordinator.group.assignor.RangeAssignor;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
+import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
+import org.apache.kafka.coordinator.group.generic.GenericGroup;
+import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
+import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.server.common.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class OffsetMetadataManagerTest {
+    static class OffsetMetadataManagerTestContext {
+        public static class Builder {
+            final private MockTime time = new MockTime();
+            final private LogContext logContext = new LogContext();
+            final private SnapshotRegistry snapshotRegistry = new 
SnapshotRegistry(logContext);
+            private MetadataImage metadataImage = null;
+            private int offsetMetadataMaxSize = 4096;
+
+            Builder withOffsetMetadataMaxSize(int offsetMetadataMaxSize) {
+                this.offsetMetadataMaxSize = offsetMetadataMaxSize;
+                return this;
+            }
+
+            OffsetMetadataManagerTestContext build() {
+                if (metadataImage == null) metadataImage = MetadataImage.EMPTY;
+
+                GroupMetadataManager groupMetadataManager = new 
GroupMetadataManager.Builder()
+                    .withTime(time)
+                    .withTimer(new MockCoordinatorTimer<>(time))
+                    .withSnapshotRegistry(snapshotRegistry)
+                    .withLogContext(logContext)
+                    .withMetadataImage(metadataImage)
+                    .withTopicPartition(new 
TopicPartition("__consumer_offsets", 0))
+                    .withAssignors(Collections.singletonList(new 
RangeAssignor()))
+                    .build();
+
+                OffsetMetadataManager offsetMetadataManager = new 
OffsetMetadataManager.Builder()
+                    .withTime(time)
+                    .withLogContext(logContext)
+                    .withSnapshotRegistry(snapshotRegistry)
+                    .withMetadataImage(metadataImage)
+                    .withGroupMetadataManager(groupMetadataManager)
+                    .withOffsetMetadataMaxSize(offsetMetadataMaxSize)
+                    .build();
+
+                return new OffsetMetadataManagerTestContext(
+                    time,
+                    snapshotRegistry,
+                    groupMetadataManager,
+                    offsetMetadataManager
+                );
+            }
+        }
+
+        final MockTime time;
+        final SnapshotRegistry snapshotRegistry;
+        final GroupMetadataManager groupMetadataManager;
+        final OffsetMetadataManager offsetMetadataManager;
+
+        long lastCommittedOffset = 0L;
+        long lastWrittenOffset = 0L;
+
+        OffsetMetadataManagerTestContext(
+            MockTime time,
+            SnapshotRegistry snapshotRegistry,
+            GroupMetadataManager groupMetadataManager,
+            OffsetMetadataManager offsetMetadataManager
+        ) {
+            this.time = time;
+            this.snapshotRegistry = snapshotRegistry;
+            this.groupMetadataManager = groupMetadataManager;
+            this.offsetMetadataManager = offsetMetadataManager;
+        }
+
+        public CoordinatorResult<OffsetCommitResponseData, Record> 
commitOffset(
+            OffsetCommitRequestData request
+        ) {
+            return commitOffset(ApiKeys.OFFSET_COMMIT.latestVersion(), 
request);
+        }
+
+        public CoordinatorResult<OffsetCommitResponseData, Record> 
commitOffset(
+            short version,
+            OffsetCommitRequestData request
+        ) {
+            snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset);
+
+            RequestContext context = new RequestContext(
+                new RequestHeader(
+                    ApiKeys.OFFSET_COMMIT,
+                    version,
+                    "client",
+                    0
+                ),
+                "1",
+                InetAddress.getLoopbackAddress(),
+                KafkaPrincipal.ANONYMOUS,
+                ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                SecurityProtocol.PLAINTEXT,
+                ClientInformation.EMPTY,
+                false
+            );
+
+            CoordinatorResult<OffsetCommitResponseData, Record> result = 
offsetMetadataManager.commitOffset(
+                context,
+                request
+            );
+
+            result.records().forEach(this::replay);
+            return result;
+        }
+
+        private ApiMessage messageOrNull(ApiMessageAndVersion 
apiMessageAndVersion) {
+            if (apiMessageAndVersion == null) {
+                return null;
+            } else {
+                return apiMessageAndVersion.message();
+            }
+        }
+
+        private void replay(
+            Record record
+        ) {
+            ApiMessageAndVersion key = record.key();
+            ApiMessageAndVersion value = record.value();
+
+            if (key == null) {
+                throw new IllegalStateException("Received a null key in " + 
record);
+            }
+
+            switch (key.version()) {
+                case OffsetCommitKey.HIGHEST_SUPPORTED_VERSION:
+                    offsetMetadataManager.replay(
+                        (OffsetCommitKey) key.message(),
+                        (OffsetCommitValue) messageOrNull(value)
+                    );
+                    break;
+
+                default:
+                    throw new IllegalStateException("Received an unknown 
record type " + key.version()
+                        + " in " + record);
+            }
+
+            lastWrittenOffset++;
+        }
+    }
+
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT)
+    public void testOffsetCommitWithUnknownGroup(short version) {
+        OffsetMetadataManagerTestContext context = new 
OffsetMetadataManagerTestContext.Builder().build();
+
+        Class<? extends Throwable> expectedType;
+        if (version >= 9) {
+            expectedType = GroupIdNotFoundException.class;
+        } else {
+            expectedType = IllegalGenerationException.class;
+        }
+
+        // Verify that the request is rejected with the correct exception.
+        assertThrows(expectedType, () -> context.commitOffset(
+            version,
+            new OffsetCommitRequestData()
+                .setGroupId("foo")
+                .setMemberId("member")
+                .setGenerationIdOrMemberEpoch(10)
+                .setTopics(Collections.singletonList(
+                    new OffsetCommitRequestData.OffsetCommitRequestTopic()
+                        .setName("bar")
+                        .setPartitions(Collections.singletonList(
+                            new 
OffsetCommitRequestData.OffsetCommitRequestPartition()
+                                .setPartitionIndex(0)
+                                .setCommittedOffset(100L)
+                        ))
+                ))
+            )
+        );
+    }
+
+    @Test
+    public void testGenericGroupOffsetCommitWithDeadGroup() {
+        OffsetMetadataManagerTestContext context = new 
OffsetMetadataManagerTestContext.Builder().build();
+
+        // Create a dead group.
+        GenericGroup group = 
context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "foo",
+            true
+        );
+        group.transitionTo(GenericGroupState.DEAD);
+
+        // Verify that the request is rejected with the correct exception.
+        assertThrows(CoordinatorNotAvailableException.class, () -> 
context.commitOffset(
+            new OffsetCommitRequestData()
+                .setGroupId("foo")
+                .setMemberId("member")
+                .setGenerationIdOrMemberEpoch(10)
+                .setTopics(Collections.singletonList(
+                    new OffsetCommitRequestData.OffsetCommitRequestTopic()
+                        .setName("bar")
+                        .setPartitions(Collections.singletonList(
+                            new 
OffsetCommitRequestData.OffsetCommitRequestPartition()
+                                .setPartitionIndex(0)
+                                .setCommittedOffset(100L)
+                        ))
+                ))
+            )
+        );
+    }
+
+    @Test
+    public void testGenericGroupOffsetCommitWithUnknownMemberId() {
+        OffsetMetadataManagerTestContext context = new 
OffsetMetadataManagerTestContext.Builder().build();
+
+        // Create an empty group.
+        context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "foo",
+            true
+        );
+
+        // Verify that the request is rejected with the correct exception.
+        assertThrows(UnknownMemberIdException.class, () -> 
context.commitOffset(
+            new OffsetCommitRequestData()
+                .setGroupId("foo")
+                .setMemberId("member")
+                .setGenerationIdOrMemberEpoch(10)
+                .setTopics(Collections.singletonList(
+                    new OffsetCommitRequestData.OffsetCommitRequestTopic()
+                        .setName("bar")
+                        .setPartitions(Collections.singletonList(
+                            new 
OffsetCommitRequestData.OffsetCommitRequestPartition()
+                                .setPartitionIndex(0)
+                                .setCommittedOffset(100L)
+                        ))
+                ))
+            )
+        );
+    }
+
+    @Test
+    public void testGenericGroupOffsetCommitWithIllegalGeneration() {
+        OffsetMetadataManagerTestContext context = new 
OffsetMetadataManagerTestContext.Builder().build();
+
+        // Create an empty group.
+        GenericGroup group = 
context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "foo",
+            true
+        );
+
+        // Add member.
+        group.add(new GenericGroupMember(
+            "member",
+            Optional.of("new-instance-id"),
+            "client-id",
+            "host",
+            5000,
+            5000,
+            "consumer",
+            new JoinGroupRequestData.JoinGroupRequestProtocolCollection(
+                Collections.singletonList(new 
JoinGroupRequestData.JoinGroupRequestProtocol()
+                    .setName("range")
+                    .setMetadata(new byte[0])
+                ).iterator()
+            )
+        ));
+
+        // Transition to next generation.
+        group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+        group.initNextGeneration();
+        assertEquals(1, group.generationId());
+
+        // Verify that the request is rejected with the correct exception.
+        assertThrows(IllegalGenerationException.class, () -> 
context.commitOffset(
+            new OffsetCommitRequestData()
+                .setGroupId("foo")
+                .setMemberId("member")
+                .setGenerationIdOrMemberEpoch(10)
+                .setTopics(Collections.singletonList(
+                    new OffsetCommitRequestData.OffsetCommitRequestTopic()
+                        .setName("bar")
+                        .setPartitions(Collections.singletonList(
+                            new 
OffsetCommitRequestData.OffsetCommitRequestPartition()
+                                .setPartitionIndex(0)
+                                .setCommittedOffset(100L)
+                        ))
+                ))
+            )
+        );
+    }
+
+    @Test
+    public void testGenericGroupOffsetCommitWithUnknownInstanceId() {

Review Comment:
   I figured the error was the same, but just wanted to confirm we were testing 
the correct path in the code.



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