AndrewJSchofield commented on code in PR #20168:
URL: https://github.com/apache/kafka/pull/20168#discussion_r2215468267


##########
tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java:
##########
@@ -1028,15 +1029,38 @@ private Map<TopicPartition, OffsetAndMetadata> 
prepareOffsetsToReset(String grou
             return null;
         }
 
-        private void 
checkAllTopicPartitionsHaveLeader(Collection<TopicPartition> partitionsToReset) 
{
+        private void checkAllTopicPartitionsValid(Collection<TopicPartition> 
partitionsToReset) {
+            // check the partitions exist
+            List<TopicPartition> partitionsNotExistList = 
filterNotExistPartitions(partitionsToReset);
+            if (!partitionsNotExistList.isEmpty()) {
+                String partitionStr = 
partitionsNotExistList.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
+                throw new UnknownTopicOrPartitionException("The partitions \"" 
+ partitionStr + "\" does not exist");
+            }
+
+            // check the partitions have leader
             List<TopicPartition> partitionsWithoutLeader = 
filterNoneLeaderPartitions(partitionsToReset);
             if (!partitionsWithoutLeader.isEmpty()) {
                 String partitionStr = 
partitionsWithoutLeader.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
-                // throw exception
                 throw new LeaderNotAvailableException("The partitions \"" + 
partitionStr + "\" have no leader");
             }
         }
 
+        private List<TopicPartition> 
filterNotExistPartitions(Collection<TopicPartition> topicPartitions) {
+            // collect all topics
+            Set<String> topics = 
topicPartitions.stream().map(TopicPartition::topic).collect(Collectors.toSet());
+            try {
+                List<TopicPartition> existPartitions = 
adminClient.describeTopics(topics).allTopicNames().get().entrySet()
+                        .stream()
+                        .flatMap(entry -> 
entry.getValue().partitions().stream()
+                                .map(partitionInfo -> new 
TopicPartition(entry.getKey(), partitionInfo.partition())))
+                        .toList();
+
+                return topicPartitions.stream().filter(element -> 
!existPartitions.contains(element)).toList();
+            } catch (Exception e) {

Review Comment:
   For consistency, we tend to ` catch (InterruptedException | 
ExecutionException e)` in this source file.



##########
tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java:
##########
@@ -1028,15 +1029,38 @@ private Map<TopicPartition, OffsetAndMetadata> 
prepareOffsetsToReset(String grou
             return null;
         }
 
-        private void 
checkAllTopicPartitionsHaveLeader(Collection<TopicPartition> partitionsToReset) 
{
+        private void checkAllTopicPartitionsValid(Collection<TopicPartition> 
partitionsToReset) {
+            // check the partitions exist
+            List<TopicPartition> partitionsNotExistList = 
filterNotExistPartitions(partitionsToReset);
+            if (!partitionsNotExistList.isEmpty()) {
+                String partitionStr = 
partitionsNotExistList.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
+                throw new UnknownTopicOrPartitionException("The partitions \"" 
+ partitionStr + "\" does not exist");

Review Comment:
   nit: Grammar "do not exist" I think.



##########
tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java:
##########
@@ -1028,15 +1029,38 @@ private Map<TopicPartition, OffsetAndMetadata> 
prepareOffsetsToReset(String grou
             return null;
         }
 
-        private void 
checkAllTopicPartitionsHaveLeader(Collection<TopicPartition> partitionsToReset) 
{
+        private void checkAllTopicPartitionsValid(Collection<TopicPartition> 
partitionsToReset) {
+            // check the partitions exist
+            List<TopicPartition> partitionsNotExistList = 
filterNotExistPartitions(partitionsToReset);
+            if (!partitionsNotExistList.isEmpty()) {
+                String partitionStr = 
partitionsNotExistList.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
+                throw new UnknownTopicOrPartitionException("The partitions \"" 
+ partitionStr + "\" does not exist");
+            }
+
+            // check the partitions have leader
             List<TopicPartition> partitionsWithoutLeader = 
filterNoneLeaderPartitions(partitionsToReset);
             if (!partitionsWithoutLeader.isEmpty()) {
                 String partitionStr = 
partitionsWithoutLeader.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
-                // throw exception
                 throw new LeaderNotAvailableException("The partitions \"" + 
partitionStr + "\" have no leader");
             }
         }
 
+        private List<TopicPartition> 
filterNotExistPartitions(Collection<TopicPartition> topicPartitions) {

Review Comment:
   nit: `filterNonExistentPartitions` is probably a bit better.



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