This is an automated email from the ASF dual-hosted git repository.

dajac pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 1792b19a057 KAFKA-17593; [4/N] Introduce 
ConsumerGroupRegularExpression record & related bookkeeping in ConsumerGroup 
(#17694)
1792b19a057 is described below

commit 1792b19a057662263b01a7e6c6546de48934389a
Author: David Jacot <[email protected]>
AuthorDate: Fri Nov 8 08:16:51 2024 +0100

    KAFKA-17593; [4/N] Introduce ConsumerGroupRegularExpression record & 
related bookkeeping in ConsumerGroup (#17694)
    
    This patch introduces the ConsumerGroupRegularExpression record (key + 
value) and updates the `GroupMatadataManager` and the `ConsumerGroup` to 
bookkeep it appropriately. Note that with this change, regular expressions are 
counted as subscribers in the `subscribedTopicNames` data structure. This is 
important because the topic metadata of the group is computed based on it.
    
    Reviewers: Jeff Kim <[email protected]>, Lianet Magrans 
<[email protected]>
---
 .../group/GroupCoordinatorRecordHelpers.java       |  58 +++++++++++
 .../coordinator/group/GroupCoordinatorShard.java   |  10 ++
 .../coordinator/group/GroupMetadataManager.java    |  37 +++++++
 .../coordinator/group/modern/ModernGroup.java      |   2 +-
 .../group/modern/consumer/ConsumerGroup.java       |  46 +++++++++
 .../modern/consumer/ResolvedRegularExpression.java |  80 +++++++++++++++
 .../message/ConsumerGroupRegularExpressionKey.json |  28 ++++++
 .../ConsumerGroupRegularExpressionValue.json       |  27 +++++
 .../group/GroupCoordinatorRecordHelpersTest.java   |  54 ++++++++++
 .../group/GroupCoordinatorShardTest.java           |  66 ++++++++++++
 .../group/GroupMetadataManagerTest.java            |  58 +++++++++++
 .../group/GroupMetadataManagerTestContext.java     |   9 ++
 .../group/modern/consumer/ConsumerGroupTest.java   | 111 +++++++++++++++++++++
 .../consumer/ResolvedRegularExpressionTest.java    |  68 +++++++++++++
 14 files changed, 653 insertions(+), 1 deletion(-)

diff --git 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java
 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java
index 67364470b0f..be912107504 100644
--- 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java
+++ 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java
@@ -29,6 +29,8 @@ import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey;
 import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionKey;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey;
@@ -51,6 +53,7 @@ import 
org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMe
 import 
org.apache.kafka.coordinator.group.generated.ShareGroupTargetAssignmentMetadataValue;
 import org.apache.kafka.coordinator.group.modern.TopicMetadata;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
+import 
org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
 import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
@@ -408,6 +411,61 @@ public class GroupCoordinatorRecordHelpers {
         );
     }
 
+    /**
+     * Creates a ConsumerGroupRegularExpression record.
+     *
+     * @param groupId                       The consumer group id.
+     * @param regex                         The regular expression.
+     * @param resolvedRegularExpression     The metadata associated with the 
regular expression.
+     * @return The record.
+     */
+    public static CoordinatorRecord newConsumerGroupRegularExpressionRecord(
+        String groupId,
+        String regex,
+        ResolvedRegularExpression resolvedRegularExpression
+    ) {
+        List<String> topics = new 
ArrayList<>(resolvedRegularExpression.topics);
+        Collections.sort(topics);
+
+        return new CoordinatorRecord(
+            new ApiMessageAndVersion(
+                new ConsumerGroupRegularExpressionKey()
+                    .setGroupId(groupId)
+                    .setRegularExpression(regex),
+                (short) 16
+            ),
+            new ApiMessageAndVersion(
+                new ConsumerGroupRegularExpressionValue()
+                    .setTopics(topics)
+                    .setVersion(resolvedRegularExpression.version)
+                    .setTimestamp(resolvedRegularExpression.timestamp),
+                (short) 0
+            )
+        );
+    }
+
+    /**
+     * Creates a ConsumerGroupRegularExpression tombstone.
+     *
+     * @param groupId   The consumer group id.
+     * @param regex     The regular expression.
+     * @return The record.
+     */
+    public static CoordinatorRecord newConsumerGroupRegularExpressionTombstone(
+        String groupId,
+        String regex
+    ) {
+        return new CoordinatorRecord(
+            new ApiMessageAndVersion(
+                new ConsumerGroupRegularExpressionKey()
+                    .setGroupId(groupId)
+                    .setRegularExpression(regex),
+                (short) 16
+            ),
+            null // Tombstone
+        );
+    }
+
     /**
      * Creates a GroupMetadata record.
      *
diff --git 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
index 70b8d88fa04..173ea366c7f 100644
--- 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
+++ 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
@@ -63,6 +63,8 @@ import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey;
 import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionKey;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey;
@@ -760,6 +762,7 @@ public class GroupCoordinatorShard implements 
CoordinatorShard<CoordinatorRecord
      * @param record        The record to apply to the state machine.
      * @throws RuntimeException
      */
+    @SuppressWarnings({"CyclomaticComplexity"})
     @Override
     public void replay(
         long offset,
@@ -872,6 +875,13 @@ public class GroupCoordinatorShard implements 
CoordinatorShard<CoordinatorRecord
                 );
                 break;
 
+            case 16:
+                groupMetadataManager.replay(
+                    (ConsumerGroupRegularExpressionKey) key.message(),
+                    (ConsumerGroupRegularExpressionValue) 
Utils.messageOrNull(value)
+                );
+                break;
+
             default:
                 throw new IllegalStateException("Received an unknown record 
type " + key.version()
                     + " in " + record);
diff --git 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
index 3435f3b3d2f..f2169740e3b 100644
--- 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
+++ 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
@@ -82,6 +82,8 @@ import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey;
 import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionKey;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey;
@@ -109,6 +111,7 @@ import 
org.apache.kafka.coordinator.group.modern.TopicMetadata;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
 import 
org.apache.kafka.coordinator.group.modern.consumer.CurrentAssignmentBuilder;
+import 
org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
 import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
 import 
org.apache.kafka.coordinator.group.modern.share.ShareGroupAssignmentBuilder;
 import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
@@ -3608,6 +3611,40 @@ public class GroupMetadataManager {
         }
     }
 
+    /**
+     * Replays ConsumerGroupRegularExpressionKey/Value to update the hard 
state of
+     * the consumer group.
+     *
+     * @param key   A ConsumerGroupRegularExpressionKey key.
+     * @param value A ConsumerGroupRegularExpressionValue record.
+     */
+    public void replay(
+        ConsumerGroupRegularExpressionKey key,
+        ConsumerGroupRegularExpressionValue value
+    ) {
+        String groupId = key.groupId();
+        String regex = key.regularExpression();
+
+        if (value != null) {
+            ConsumerGroup group = 
getOrMaybeCreatePersistedConsumerGroup(groupId, true);
+            group.updateResolvedRegularExpression(
+                regex,
+                new ResolvedRegularExpression(
+                    new HashSet<>(value.topics()),
+                    value.version(),
+                    value.timestamp()
+                )
+            );
+        } else {
+            try {
+                ConsumerGroup group = 
getOrMaybeCreatePersistedConsumerGroup(groupId, false);
+                group.removeResolvedRegularExpression(regex);
+            } catch (GroupIdNotFoundException ex) {
+                // If the group does not exist, we can ignore the tombstone.
+            }
+        }
+    }
+
     /**
      * Replays ShareGroupMemberMetadataKey/Value to update the hard state of
      * the share group. It updates the subscription part of the member or
diff --git 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
index 823792c198e..8b920a7e051 100644
--- 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
+++ 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
@@ -80,7 +80,7 @@ public abstract class ModernGroup<T extends 
ModernGroupMember> implements Group
     protected final TimelineHashMap<String, T> members;
 
     /**
-     * The number of subscribers per topic.
+     * The number of subscribers or regular expressions per topic.
      */
     protected final TimelineHashMap<String, Integer> subscribedTopicNames;
 
diff --git 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
index 6dc91a10fad..3be2124c058 100644
--- 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
+++ 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
@@ -135,6 +135,11 @@ public class ConsumerGroup extends 
ModernGroup<ConsumerGroupMember> {
      */
     private final TimelineHashMap<String, Integer> 
subscribedRegularExpressions;
 
+    /**
+     * The resolved regular expressions.
+     */
+    private final TimelineHashMap<String, ResolvedRegularExpression> 
resolvedRegularExpressions;
+
     public ConsumerGroup(
         SnapshotRegistry snapshotRegistry,
         String groupId,
@@ -149,6 +154,7 @@ public class ConsumerGroup extends 
ModernGroup<ConsumerGroupMember> {
         this.classicProtocolMembersSupportedProtocols = new 
TimelineHashMap<>(snapshotRegistry, 0);
         this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 
0);
         this.subscribedRegularExpressions = new 
TimelineHashMap<>(snapshotRegistry, 0);
+        this.resolvedRegularExpressions = new 
TimelineHashMap<>(snapshotRegistry, 0);
     }
 
     /**
@@ -342,6 +348,46 @@ public class ConsumerGroup extends 
ModernGroup<ConsumerGroupMember> {
         }
     }
 
+    /**
+     * Update the resolved regular expression.
+     *
+     * @param regex                         The regular expression.
+     * @param newResolvedRegularExpression  The regular expression's metadata.
+     */
+    public void updateResolvedRegularExpression(
+        String regex,
+        ResolvedRegularExpression newResolvedRegularExpression
+    ) {
+        removeResolvedRegularExpression(regex);
+        if (newResolvedRegularExpression != null) {
+            resolvedRegularExpressions.put(regex, 
newResolvedRegularExpression);
+            newResolvedRegularExpression.topics.forEach(topicName -> 
subscribedTopicNames.compute(topicName, Utils::incValue));
+        }
+    }
+
+    /**
+     * Remove the resolved regular expression.
+     *
+     * @param regex The regular expression.
+     */
+    public void removeResolvedRegularExpression(String regex) {
+        ResolvedRegularExpression oldResolvedRegularExpression = 
resolvedRegularExpressions.remove(regex);
+        if (oldResolvedRegularExpression != null) {
+            oldResolvedRegularExpression.topics.forEach(topicName -> 
subscribedTopicNames.compute(topicName, Utils::decValue));
+        }
+    }
+
+    /**
+     * Return an optional containing the resolved regular expression 
corresponding to the provided regex
+     * or an empty optional.
+     *
+     * @param regex The regular expression.
+     * @return The optional containing the resolved regular expression or an 
empty optional.
+     */
+    public Optional<ResolvedRegularExpression> regularExpression(String regex) 
{
+        return Optional.ofNullable(resolvedRegularExpressions.get(regex));
+    }
+
     /**
      * @return The number of members subscribed to the provided regex.
      */
diff --git 
a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java
 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java
new file mode 100644
index 00000000000..d13fb23da2f
--- /dev/null
+++ 
b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpression.java
@@ -0,0 +1,80 @@
+/*
+ * 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.modern.consumer;
+
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * The metadata associated with a regular expression in a Consumer Group.
+ */
+public class ResolvedRegularExpression {
+    /**
+     * The set of resolved topics.
+     */
+    public final Set<String> topics;
+
+    /**
+     * The version of the metadata image used to resolve the topics.
+     */
+    public final long version;
+
+    /**
+     * The timestamp at the time of the resolution.
+     */
+    public final long timestamp;
+
+    public ResolvedRegularExpression(
+        Set<String> topics,
+        long version,
+        long timestamp
+    ) {
+        this.topics = 
Collections.unmodifiableSet(Objects.requireNonNull(topics));
+        this.version = version;
+        this.timestamp = timestamp;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ResolvedRegularExpression that = (ResolvedRegularExpression) o;
+
+        if (version != that.version) return false;
+        if (timestamp != that.timestamp) return false;
+        return topics.equals(that.topics);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = topics.hashCode();
+        result = 31 * result + (int) (version ^ (version >>> 32));
+        result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "RegularExpressionMetadata(" +
+            "topics=" + topics +
+            ", version=" + version +
+            ", timestamp=" + timestamp +
+            ')';
+    }
+}
diff --git 
a/group-coordinator/src/main/resources/common/message/ConsumerGroupRegularExpressionKey.json
 
b/group-coordinator/src/main/resources/common/message/ConsumerGroupRegularExpressionKey.json
new file mode 100644
index 00000000000..3f761b694e4
--- /dev/null
+++ 
b/group-coordinator/src/main/resources/common/message/ConsumerGroupRegularExpressionKey.json
@@ -0,0 +1,28 @@
+// 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.
+
+// KIP-848 is in development. This schema is subject to 
non-backwards-compatible changes.
+{
+  "type": "data",
+  "name": "ConsumerGroupRegularExpressionKey",
+  "validVersions": "16",
+  "flexibleVersions": "none",
+  "fields": [
+    { "name": "GroupId", "type": "string", "versions": "16",
+      "about": "The group id." },
+    { "name": "RegularExpression", "type": "string", "versions": "16",
+      "about": "The regular expression." }
+  ]
+}
diff --git 
a/group-coordinator/src/main/resources/common/message/ConsumerGroupRegularExpressionValue.json
 
b/group-coordinator/src/main/resources/common/message/ConsumerGroupRegularExpressionValue.json
new file mode 100644
index 00000000000..ff0d1d624a3
--- /dev/null
+++ 
b/group-coordinator/src/main/resources/common/message/ConsumerGroupRegularExpressionValue.json
@@ -0,0 +1,27 @@
+// 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.
+
+// KIP-848 is in development. This schema is subject to 
non-backwards-compatible changes.
+{
+  "type": "data",
+  "name": "ConsumerGroupRegularExpressionValue",
+  "validVersions": "0",
+  "flexibleVersions": "0+",
+  "fields": [
+    { "name": "Topics", "versions": "0+", "type": "[]string" },
+    { "name": "Version", "versions": "0+", "type": "int64" },
+    { "name": "Timestamp", "versions": "0+", "type": "int64" }
+  ]
+}
diff --git 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java
 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java
index e0a9446ce6f..812ee093c2c 100644
--- 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java
+++ 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java
@@ -34,6 +34,8 @@ import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey;
 import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionKey;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey;
@@ -46,6 +48,7 @@ import 
org.apache.kafka.coordinator.group.generated.ShareGroupMetadataKey;
 import org.apache.kafka.coordinator.group.modern.MemberState;
 import org.apache.kafka.coordinator.group.modern.TopicMetadata;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
+import 
org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
 
@@ -807,6 +810,57 @@ public class GroupCoordinatorRecordHelpersTest {
         assertEquals(expectedRecord, record);
     }
 
+    @Test
+    public void testNewConsumerGroupRegularExpressionRecord() {
+        CoordinatorRecord expectedRecord = new CoordinatorRecord(
+            new ApiMessageAndVersion(
+                new ConsumerGroupRegularExpressionKey()
+                    .setGroupId("group-id")
+                    .setRegularExpression("ab*"),
+                (short) 16
+            ),
+            new ApiMessageAndVersion(
+                new ConsumerGroupRegularExpressionValue()
+                    .setTopics(Arrays.asList("abc", "abcd"))
+                    .setVersion(10L)
+                    .setTimestamp(12345L),
+                (short) 0
+            )
+        );
+
+        CoordinatorRecord record = 
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord(
+            "group-id",
+            "ab*",
+            new ResolvedRegularExpression(
+                Set.of("abc", "abcd"),
+                10L,
+                12345L
+            )
+        );
+
+        assertEquals(expectedRecord, record);
+    }
+
+    @Test
+    public void testNewConsumerGroupRegularExpressionTombstone() {
+        CoordinatorRecord expectedRecord = new CoordinatorRecord(
+            new ApiMessageAndVersion(
+                new ConsumerGroupRegularExpressionKey()
+                    .setGroupId("group-id")
+                    .setRegularExpression("ab*"),
+                (short) 16
+            ),
+            null
+        );
+
+        CoordinatorRecord record = 
GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(
+            "group-id",
+            "ab*"
+        );
+
+        assertEquals(expectedRecord, record);
+    }
+
     /**
      * Creates a map of partitions to racks for testing.
      *
diff --git 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
index 5a3db95c06a..cb68771c8a5 100644
--- 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
+++ 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
@@ -47,6 +47,8 @@ import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey;
 import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionKey;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey;
@@ -94,6 +96,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@SuppressWarnings({"ClassFanOutComplexity"})
 public class GroupCoordinatorShardTest {
 
     @Test
@@ -1272,4 +1275,67 @@ public class GroupCoordinatorShardTest {
 
         verify(groupMetadataManager, times(1)).replay(key, null);
     }
+
+    @Test
+    public void testReplayConsumerGroupRegularExpression() {
+        GroupMetadataManager groupMetadataManager = 
mock(GroupMetadataManager.class);
+        OffsetMetadataManager offsetMetadataManager = 
mock(OffsetMetadataManager.class);
+        CoordinatorMetrics coordinatorMetrics = mock(CoordinatorMetrics.class);
+        CoordinatorMetricsShard metricsShard = 
mock(CoordinatorMetricsShard.class);
+        GroupCoordinatorShard coordinator = new GroupCoordinatorShard(
+            new LogContext(),
+            groupMetadataManager,
+            offsetMetadataManager,
+            Time.SYSTEM,
+            new MockCoordinatorTimer<>(Time.SYSTEM),
+            mock(GroupCoordinatorConfig.class),
+            coordinatorMetrics,
+            metricsShard
+        );
+
+        ConsumerGroupRegularExpressionKey key = new 
ConsumerGroupRegularExpressionKey()
+            .setGroupId("group")
+            .setRegularExpression("ab*");
+
+        ConsumerGroupRegularExpressionValue value = new 
ConsumerGroupRegularExpressionValue()
+            .setTopics(Arrays.asList("abc", "abcd"))
+            .setVersion(10L)
+            .setTimestamp(12345L);
+
+        coordinator.replay(0L, RecordBatch.NO_PRODUCER_ID, 
RecordBatch.NO_PRODUCER_EPOCH, new CoordinatorRecord(
+            new ApiMessageAndVersion(key, (short) 16),
+            new ApiMessageAndVersion(value, (short) 0)
+        ));
+
+        verify(groupMetadataManager, times(1)).replay(key, value);
+    }
+
+    @Test
+    public void testReplayConsumerGroupRegularExpressionTombstone() {
+        GroupMetadataManager groupMetadataManager = 
mock(GroupMetadataManager.class);
+        OffsetMetadataManager offsetMetadataManager = 
mock(OffsetMetadataManager.class);
+        CoordinatorMetrics coordinatorMetrics = mock(CoordinatorMetrics.class);
+        CoordinatorMetricsShard metricsShard = 
mock(CoordinatorMetricsShard.class);
+        GroupCoordinatorShard coordinator = new GroupCoordinatorShard(
+            new LogContext(),
+            groupMetadataManager,
+            offsetMetadataManager,
+            Time.SYSTEM,
+            new MockCoordinatorTimer<>(Time.SYSTEM),
+            mock(GroupCoordinatorConfig.class),
+            coordinatorMetrics,
+            metricsShard
+        );
+
+        ConsumerGroupRegularExpressionKey key = new 
ConsumerGroupRegularExpressionKey()
+            .setGroupId("group")
+            .setRegularExpression("ab*");
+
+        coordinator.replay(0L, RecordBatch.NO_PRODUCER_ID, 
RecordBatch.NO_PRODUCER_EPOCH, new CoordinatorRecord(
+            new ApiMessageAndVersion(key, (short) 16),
+            null
+        ));
+
+        verify(groupMetadataManager, times(1)).replay(key, null);
+    }
 }
diff --git 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
index b912b4f18de..35fc8111658 100644
--- 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
+++ 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
@@ -84,6 +84,7 @@ import 
org.apache.kafka.coordinator.group.modern.TopicMetadata;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupBuilder;
 import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
+import 
org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
 import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
 import org.apache.kafka.coordinator.group.modern.share.ShareGroupBuilder;
 import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
@@ -15086,6 +15087,63 @@ public class GroupMetadataManagerTest {
         assertEquals(expected, actual);
     }
 
+    @Test
+    public void testReplayConsumerGroupRegularExpression() {
+        GroupMetadataManagerTestContext context = new 
GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        ResolvedRegularExpression resolvedRegularExpression = new 
ResolvedRegularExpression(
+            Set.of("abc", "abcd"),
+            10L,
+            12345L
+        );
+
+        
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord(
+            "foo",
+            "abc*",
+            resolvedRegularExpression
+        ));
+
+        assertEquals(
+            Optional.of(resolvedRegularExpression),
+            
context.groupMetadataManager.consumerGroup("foo").regularExpression("abc*")
+        );
+    }
+
+    @Test
+    public void testReplayConsumerGroupRegularExpressionTombstone() {
+        GroupMetadataManagerTestContext context = new 
GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // The group may not exist at all. Replaying the 
ConsumerGroupRegularExpression tombstone
+        // should be a no-op.
+        
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone("foo",
 "abc*"));
+        assertThrows(GroupIdNotFoundException.class, () -> 
context.groupMetadataManager.consumerGroup("foo"));
+
+        // Otherwise, it should remove the regular expression.
+        ResolvedRegularExpression resolvedRegularExpression = new 
ResolvedRegularExpression(
+            Set.of("abc", "abcd"),
+            10L,
+            12345L
+        );
+
+        
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionRecord(
+            "foo",
+            "abc*",
+            resolvedRegularExpression
+        ));
+
+        
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(
+            "foo",
+            "abc*"
+        ));
+
+        assertEquals(
+            Optional.empty(),
+            
context.groupMetadataManager.consumerGroup("foo").regularExpression("abc*")
+        );
+    }
+
     private static void checkJoinGroupResponse(
         JoinGroupResponseData expectedResponse,
         JoinGroupResponseData actualResponse,
diff --git 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
index 43329ab3e8f..f752fa82544 100644
--- 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
+++ 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
@@ -63,6 +63,8 @@ import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey;
 import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionKey;
+import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupRegularExpressionValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue;
 import 
org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey;
@@ -1557,6 +1559,13 @@ public class GroupMetadataManagerTestContext {
                 );
                 break;
 
+            case ConsumerGroupRegularExpressionKey.HIGHEST_SUPPORTED_VERSION:
+                groupMetadataManager.replay(
+                    (ConsumerGroupRegularExpressionKey) key.message(),
+                    (ConsumerGroupRegularExpressionValue) messageOrNull(value)
+                );
+                break;
+
             default:
                 throw new IllegalStateException("Received an unknown record 
type " + key.version()
                     + " in " + record);
diff --git 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
index 028cd23cfda..39140289ea4 100644
--- 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
+++ 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
@@ -58,9 +58,11 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
 import java.util.OptionalInt;
 import java.util.OptionalLong;
+import java.util.Set;
 
 import static org.apache.kafka.common.utils.Utils.mkEntry;
 import static org.apache.kafka.common.utils.Utils.mkMap;
@@ -1664,4 +1666,113 @@ public class ConsumerGroupTest {
         assertEquals(0, consumerGroup.numSubscribedMembers("regex2"));
         assertEquals(0, consumerGroup.numSubscribedMembers("regex3"));
     }
+
+    @Test
+    public void testUpdateAndRemoveRegularExpression() {
+        ConsumerGroup consumerGroup = createConsumerGroup("foo");
+
+        ConsumerGroupMember member1 = new 
ConsumerGroupMember.Builder("member1")
+            .setSubscribedTopicNames(Arrays.asList("foo", "bar", "zar"))
+            .build();
+        consumerGroup.updateMember(member1);
+
+        ConsumerGroupMember member2 = new 
ConsumerGroupMember.Builder("member2")
+            .setSubscribedTopicNames(Arrays.asList("foo", "bar"))
+            .build();
+        consumerGroup.updateMember(member2);
+
+        // Verify initial state.
+        assertEquals(
+            Map.of(
+                "foo", 2,
+                "bar", 2,
+                "zar", 1
+            ),
+            consumerGroup.subscribedTopicNames()
+        );
+
+        // Add a regex.
+        consumerGroup.updateResolvedRegularExpression(
+            "foo|bar",
+            new ResolvedRegularExpression(
+                Set.of("foo", "bar"),
+                10L,
+                12345L
+            )
+        );
+
+        assertEquals(
+            Map.of(
+                "foo", 3,
+                "bar", 3,
+                "zar", 1
+            ),
+            consumerGroup.subscribedTopicNames()
+        );
+
+        // Add a regex.
+        consumerGroup.updateResolvedRegularExpression(
+            "foobar",
+            new ResolvedRegularExpression(
+                Set.of("foobar"),
+                10L,
+                12345L
+            )
+        );
+
+        assertEquals(
+            Map.of(
+                "foo", 3,
+                "bar", 3,
+                "zar", 1,
+                "foobar", 1
+            ),
+            consumerGroup.subscribedTopicNames()
+        );
+
+        // Update a regex.
+        consumerGroup.updateResolvedRegularExpression(
+            "foo|bar",
+            new ResolvedRegularExpression(
+                Set.of("foo"),
+                10L,
+                12345L
+            )
+        );
+
+        assertEquals(
+            Map.of(
+                "foo", 3,
+                "bar", 2,
+                "zar", 1,
+                "foobar", 1
+            ),
+            consumerGroup.subscribedTopicNames()
+        );
+
+        // Remove a regex.
+        consumerGroup.removeResolvedRegularExpression("foo|bar");
+
+        assertEquals(
+            Map.of(
+                "foo", 2,
+                "bar", 2,
+                "zar", 1,
+                "foobar", 1
+            ),
+            consumerGroup.subscribedTopicNames()
+        );
+
+        // Remove another regex.
+        consumerGroup.removeResolvedRegularExpression("foobar");
+
+        assertEquals(
+            Map.of(
+                "foo", 2,
+                "bar", 2,
+                "zar", 1
+            ),
+            consumerGroup.subscribedTopicNames()
+        );
+    }
 }
diff --git 
a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpressionTest.java
 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpressionTest.java
new file mode 100644
index 00000000000..11b635af7d7
--- /dev/null
+++ 
b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ResolvedRegularExpressionTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.modern.consumer;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+public class ResolvedRegularExpressionTest {
+    @Test
+    public void testConstructor() {
+        ResolvedRegularExpression resolvedRegularExpression = new 
ResolvedRegularExpression(
+            Set.of("foo", "bar"),
+            10L,
+            12345L
+        );
+
+        assertEquals(Set.of("foo", "bar"), resolvedRegularExpression.topics);
+        assertEquals(10L, resolvedRegularExpression.version);
+        assertEquals(12345L, resolvedRegularExpression.timestamp);
+    }
+
+    @Test
+    public void testEquals() {
+        assertEquals(
+            new ResolvedRegularExpression(
+                Set.of("foo", "bar"),
+                10L,
+                12345L
+            ),
+            new ResolvedRegularExpression(
+                Set.of("foo", "bar"),
+                10L,
+                12345L
+            )
+        );
+
+        assertNotEquals(
+            new ResolvedRegularExpression(
+                Set.of("foo", "bar"),
+                10L,
+                12345L
+            ),
+            new ResolvedRegularExpression(
+                Set.of("foo"),
+                10L,
+                12345L
+            )
+        );
+    }
+}


Reply via email to