From a6dfde7ce64ee00d257bb6ff20ce923dacdc0edb Mon Sep 17 00:00:00 2001
From: Sushant Mahajan
Date: Fri, 11 Apr 2025 00:45:13 +0530
Subject: [PATCH] KAFKA-18629: Utilize share group partition metadata for
delete group. (#19363)
* Currently, the delete share group code flow uses
`group.subscribedTopicNames()` to fetch information about all the share
partitions to which a share group is subscribed to.
* However, this is incorrect since once the group is EMPTY, a
precondition for delete, the aforementioned method will return an empty
list.
* In this PR we have leveraged the `ShareGroupStatePartitionMetadata`
record to grab the `initialized` and `initializing` partitions to build
the delete candidates, which remedies the situation.
Reviewers: Andrew Schofield
---
.../group/GroupCoordinatorRecordHelpers.java | 15 +++
.../group/GroupCoordinatorService.java | 34 ++++--
.../group/GroupCoordinatorShard.java | 9 +-
.../group/GroupMetadataManager.java | 79 ++++++++-----
.../group/modern/share/ShareGroup.java | 1 +
.../kafka/coordinator/group/Assertions.java | 31 ++++-
.../group/GroupCoordinatorShardTest.java | 43 ++++---
.../group/GroupMetadataManagerTest.java | 108 ++++++++++++------
.../share/ShareCoordinatorShard.java | 10 ++
.../share/ShareCoordinatorShardTest.java | 23 +---
10 files changed, 234 insertions(+), 119 deletions(-)
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 d0f6901d444..42790515a2d 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
@@ -814,6 +814,21 @@ public class GroupCoordinatorRecordHelpers {
);
}
+ /**
+ * Creates a ShareGroupStatePartitionMetadata tombstone.
+ *
+ * @param groupId The share group id.
+ * @return The record.
+ */
+ public static CoordinatorRecord newShareGroupStatePartitionMetadataTombstoneRecord(
+ String groupId
+ ) {
+ return CoordinatorRecord.tombstone(
+ new ShareGroupStatePartitionMetadataKey()
+ .setGroupId(groupId)
+ );
+ }
+
/**
* Creates a ShareGroupStatePartitionMetadata record.
*
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
index 97bc7e69622..c00502af48f 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
@@ -1073,24 +1073,25 @@ public class GroupCoordinatorService implements GroupCoordinator {
groupsByTopicPartition.forEach((topicPartition, groupList) -> {
CompletableFuture future = deleteShareGroups(topicPartition, groupList).thenCompose(groupErrMap -> {
- DeleteGroupsResponseData.DeletableGroupResultCollection collection = new DeleteGroupsResponseData.DeletableGroupResultCollection();
- List retainedGroupIds = deleteCandidateGroupIds(groupErrMap, groupList, collection);
+ DeleteGroupsResponseData.DeletableGroupResultCollection deletableGroupResults = new DeleteGroupsResponseData.DeletableGroupResultCollection();
+ List retainedGroupIds = updateResponseAndGetNonErrorGroupList(groupErrMap, groupList, deletableGroupResults);
if (retainedGroupIds.isEmpty()) {
- return CompletableFuture.completedFuture(collection);
+ return CompletableFuture.completedFuture(deletableGroupResults);
}
return handleDeleteGroups(context, topicPartition, retainedGroupIds)
- .whenComplete((resp, __) -> resp.forEach(result -> collection.add(result.duplicate())))
- .thenApply(__ -> collection);
+ .whenComplete((resp, __) -> resp.forEach(result -> deletableGroupResults.add(result.duplicate())))
+ .thenApply(__ -> deletableGroupResults);
});
// deleteShareGroups has its own exceptionally block, so we don't need one here.
// This future object has the following stages:
// - First it invokes the share group delete flow where the shard sharePartitionDeleteRequests
// method is invoked, and it returns request objects for each valid share group passed to it.
+ // All initialized and initializing share partitions are moved to deleting.
// - Then the requests are passed to the persister.deleteState method one at a time. The results
// are collated as a Map of groupId -> persister errors
- // - The above map is then used to decide whether to invoke the group coordinator delete groups logic
+ // - The above map can be used to decide whether to invoke the group coordinator delete groups logic
// - Share groups with failed persister delete are NOT CONSIDERED for group coordinator delete.
// TLDR: DeleteShareGroups -> filter erroneous persister deletes -> general delete groups logic
futures.add(future);
@@ -1102,17 +1103,26 @@ public class GroupCoordinatorService implements GroupCoordinator {
(accumulator, newResults) -> newResults.forEach(result -> accumulator.add(result.duplicate())));
}
- private List deleteCandidateGroupIds(
- Map groupErrMap,
+ /**
+ * Processes input shareGroupErrMap by retaining only those which do not contain an error.
+ * Also updates the result collection input arg with share groups containing errors.
+ *
+ * @param shareGroupErrMap Map keyed on share groupId and value as the error (NONE for no error).
+ * @param groupList Entire list of groups (all types)
+ * @param deletableGroupResults Collection of responses for delete groups request.
+ * @return A list of all non-error groupIds
+ */
+ private List updateResponseAndGetNonErrorGroupList(
+ Map shareGroupErrMap,
List groupList,
- DeleteGroupsResponseData.DeletableGroupResultCollection collection
+ DeleteGroupsResponseData.DeletableGroupResultCollection deletableGroupResults
) {
List errGroupIds = new ArrayList<>();
- groupErrMap.forEach((groupId, error) -> {
+ shareGroupErrMap.forEach((groupId, error) -> {
if (error.code() != Errors.NONE.code()) {
log.error("Error deleting share group {} due to error {}", groupId, error);
errGroupIds.add(groupId);
- collection.add(
+ deletableGroupResults.add(
new DeleteGroupsResponseData.DeletableGroupResult()
.setGroupId(groupId)
.setErrorCode(error.code())
@@ -1153,7 +1163,7 @@ public class GroupCoordinatorService implements GroupCoordinator {
TopicPartition topicPartition,
List groupList
) {
- // topicPartition refers to internal topic __consumer_offsets
+ // topicPartition refers to internal topic __consumer_offsets.
return runtime.scheduleWriteOperation(
"delete-share-groups",
topicPartition,
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 ded14600dab..812641c9459 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
@@ -589,18 +589,19 @@ public class GroupCoordinatorShard implements CoordinatorShard
+ *
* The groupIds are first filtered by type to restrict the list to share groups.
* @param groupIds - A list of groupIds as string
- * @return {@link CoordinatorResult} object always containing empty records and Map keyed on groupId and value pair (req, error)
+ * @return A result object containing a map keyed on groupId and value pair (req, error) and related coordinator records.
*/
public CoordinatorResult