yangy0000 commented on code in PR #14408: URL: https://github.com/apache/kafka/pull/14408#discussion_r1341918630
########## 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 -> { Review Comment: nit: the else braces can be simplify to ``` final TimelineHashMap<Integer, OffsetAndMetadata> offsetsByPartition = offsetsByTopic == null ? null : offsetsByTopic.get(topic.name()); if( offsetsByPartition !=null) continue; for(Integer paritionIndex : offsetsByPartition.keySet()) { responsePartitionCollection.add(new OffsetDeleteResponseData.OffsetDeleteResponsePartition().setPartitionIndex(paritionIndex) ); records.add(RecordHelpers.newOffsetCommitTombstoneRecord( request.groupId(), topic.name(), paritionIndex )); } ``` ########## 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 -> { Review Comment: nit: the else braces can be simplified to ``` final TimelineHashMap<Integer, OffsetAndMetadata> offsetsByPartition = offsetsByTopic == null ? null : offsetsByTopic.get(topic.name()); if( offsetsByPartition !=null) continue; for(Integer paritionIndex : offsetsByPartition.keySet()) { responsePartitionCollection.add(new OffsetDeleteResponseData.OffsetDeleteResponsePartition().setPartitionIndex(paritionIndex) ); records.add(RecordHelpers.newOffsetCommitTombstoneRecord( request.groupId(), topic.name(), paritionIndex )); } ``` -- 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