yangy0000 commented on code in PR #14408:
URL: https://github.com/apache/kafka/pull/14408#discussion_r1342851207


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java:
##########
@@ -333,6 +348,87 @@ public CoordinatorResult<OffsetCommitResponseData, Record> 
commitOffset(
         return new CoordinatorResult<>(records, response);
     }
 
+    /**
+     * Handles an OffsetDelete request.
+     *
+     * @param request The OffsetDelete request.
+     *
+     * @return A Result containing the OffsetDeleteResponseData response and
+     *         a list of records to update the state machine.
+     */
+    public CoordinatorResult<OffsetDeleteResponseData, Record> deleteOffsets(
+        OffsetDeleteRequestData request
+    ) throws ApiException {
+        final Group group = validateOffsetDelete(request);
+        final List<Record> records = new ArrayList<>();
+        final OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection 
responseTopicCollection =
+            new OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection();
+        final OffsetDeleteResponseData response = new 
OffsetDeleteResponseData();
+        final TimelineHashMap<String, TimelineHashMap<Integer, 
OffsetAndMetadata>> offsetsByTopic =
+            offsetsByGroup.get(request.groupId());
+
+        request.topics().forEach(topic -> {
+            final 
OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection 
responsePartitionCollection =
+                new 
OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection();
+            final TimelineHashMap<Integer, OffsetAndMetadata> 
offsetsByPartition = offsetsByTopic == null ?
+                null : offsetsByTopic.get(topic.name());
+
+            if (group.isSubscribedToTopic(topic.name())) {
+                topic.partitions().forEach(partition ->
+                    responsePartitionCollection.add(new 
OffsetDeleteResponseData.OffsetDeleteResponsePartition()
+                        .setPartitionIndex(partition.partitionIndex())
+                        .setErrorCode(Errors.GROUP_SUBSCRIBED_TO_TOPIC.code())
+                    )
+                );
+            } else {
+                topic.partitions().forEach(partition -> {
+                    if (offsetsByPartition != null && 
offsetsByPartition.containsKey(partition.partitionIndex())) {
+                        responsePartitionCollection.add(new 
OffsetDeleteResponseData.OffsetDeleteResponsePartition()
+                            .setPartitionIndex(partition.partitionIndex())
+                        );
+                        
records.add(RecordHelpers.newOffsetCommitTombstoneRecord(
+                            request.groupId(),
+                            topic.name(),
+                            partition.partitionIndex()
+                        ));
+                    }
+                });
+            }
+
+            final OffsetDeleteResponseData.OffsetDeleteResponseTopic 
responseTopic =
+                new OffsetDeleteResponseData.OffsetDeleteResponseTopic()
+                    .setName(topic.name())
+                    .setPartitions(responsePartitionCollection);
+            responseTopicCollection.add(responseTopic);
+        });
+        response.setTopics(responseTopicCollection);
+
+        return new CoordinatorResult<>(records, response);
+    }
+
+    /**
+     * Deletes offsets as part of a DeleteGroups request.
+     * Populates the record list passed in with records to update the state 
machine.
+     * Validations are done in {@link 
GroupCoordinatorShard#deleteGroups(RequestContext, List)}
+     *
+     * @param groupId The ID of the given group.
+     * @param records The record list to populate.
+     */
+    public void deleteAllOffsets(
+        String groupId,
+        List<Record> records
+    ) {
+        TimelineHashMap<String, TimelineHashMap<Integer, OffsetAndMetadata>> 
offsetsByTopic = offsetsByGroup.get(groupId);
+
+        if (offsetsByTopic != null) {

Review Comment:
   Any chance deleteAllOffsets will get invoked before the group is completely 
removed?



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