nizhikov commented on code in PR #14471:
URL: https://github.com/apache/kafka/pull/14471#discussion_r1530807872


##########
tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java:
##########
@@ -0,0 +1,1240 @@
+/*
+ * 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.tools.consumer.group;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import joptsimple.OptionException;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.AbstractOptions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AlterConsumerGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.ConsumerGroupDescription;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.clients.admin.DeleteConsumerGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.DeleteConsumerGroupOffsetsResult;
+import org.apache.kafka.clients.admin.DeleteConsumerGroupsOptions;
+import org.apache.kafka.clients.admin.DescribeConsumerGroupsOptions;
+import org.apache.kafka.clients.admin.DescribeTopicsOptions;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions;
+import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsSpec;
+import org.apache.kafka.clients.admin.ListConsumerGroupsOptions;
+import org.apache.kafka.clients.admin.ListConsumerGroupsResult;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.MemberDescription;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.common.GroupType;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.tools.ToolsUtils;
+import org.apache.kafka.tools.Tuple2;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.ToIntFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class ConsumerGroupCommand {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(ConsumerGroupCommand.class);
+
+    static final String MISSING_COLUMN_VALUE = "-";
+
+    public static void main(String[] args) {
+        ConsumerGroupCommandOptions opts = 
ConsumerGroupCommandOptions.fromArgs(args);
+        try {
+            CommandLineUtils.maybePrintHelpOrVersion(opts, "This tool helps to 
list all consumer groups, describe a consumer group, delete consumer group 
info, or reset consumer group offsets.");
+
+            // should have exactly one action
+            long actions = Stream.of(opts.listOpt, opts.describeOpt, 
opts.deleteOpt, opts.resetOffsetsOpt, 
opts.deleteOffsetsOpt).filter(opts.options::has).count();
+            if (actions != 1)
+                CommandLineUtils.printUsageAndExit(opts.parser, "Command must 
include exactly one action: --list, --describe, --delete, --reset-offsets, 
--delete-offsets");
+
+            run(opts);
+        } catch (OptionException e) {
+            CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage());
+        }
+    }
+
+    static void run(ConsumerGroupCommandOptions opts) {
+        try (ConsumerGroupService consumerGroupService = new 
ConsumerGroupService(opts, Collections.emptyMap())) {
+            if (opts.options.has(opts.listOpt))
+                consumerGroupService.listGroups();
+            else if (opts.options.has(opts.describeOpt))
+                consumerGroupService.describeGroups();
+            else if (opts.options.has(opts.deleteOpt))
+                consumerGroupService.deleteGroups();
+            else if (opts.options.has(opts.resetOffsetsOpt)) {
+                Map<String, Map<TopicPartition, OffsetAndMetadata>> 
offsetsToReset = consumerGroupService.resetOffsets();
+                if (opts.options.has(opts.exportOpt)) {
+                    String exported = 
consumerGroupService.exportOffsetsToCsv(offsetsToReset);
+                    System.out.println(exported);
+                } else
+                    printOffsetsToReset(offsetsToReset);
+            } else if (opts.options.has(opts.deleteOffsetsOpt)) {
+                consumerGroupService.deleteOffsets();
+            }
+        } catch (IllegalArgumentException e) {
+            CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage());
+        } catch (Throwable e) {
+            printError("Executing consumer group command failed due to " + 
e.getMessage(), Optional.of(e));
+        }
+    }
+
+    static Set<ConsumerGroupState> consumerGroupStatesFromString(String input) 
{
+        Set<ConsumerGroupState> parsedStates = 
Arrays.stream(input.split(",")).map(s -> 
ConsumerGroupState.parse(s.trim())).collect(Collectors.toSet());
+        if (parsedStates.contains(ConsumerGroupState.UNKNOWN)) {
+            Collection<ConsumerGroupState> validStates = 
Arrays.stream(ConsumerGroupState.values()).filter(s -> s != 
ConsumerGroupState.UNKNOWN).collect(Collectors.toList());
+            throw new IllegalArgumentException("Invalid state list '" + input 
+ "'. Valid states are: " + Utils.join(validStates, ", "));
+        }
+        return parsedStates;
+    }
+
+    @SuppressWarnings("Regexp")
+    static Set<GroupType> consumerGroupTypesFromString(String input) {
+        Set<GroupType> parsedTypes = 
Stream.of(input.toLowerCase().split(",")).map(s -> 
GroupType.parse(s.trim())).collect(Collectors.toSet());
+        if (parsedTypes.contains(GroupType.UNKNOWN)) {
+            List<String> validTypes = 
Arrays.stream(GroupType.values()).filter(t -> t != 
GroupType.UNKNOWN).map(Object::toString).collect(Collectors.toList());
+            throw new IllegalArgumentException("Invalid types list '" + input 
+ "'. Valid types are: " + String.join(", ", validTypes));
+        }
+        return parsedTypes;
+    }
+
+    static void printError(String msg, Optional<Throwable> e) {
+        System.out.println("\nError: " + msg);
+        e.ifPresent(Throwable::printStackTrace);
+    }
+
+    static void printOffsetsToReset(Map<String, Map<TopicPartition, 
OffsetAndMetadata>> groupAssignmentsToReset) {
+        String format = "%-30s %-30s %-10s %-15s";
+        if (!groupAssignmentsToReset.isEmpty())
+            System.out.printf("\n" + format, "GROUP", "TOPIC", "PARTITION", 
"NEW-OFFSET");
+
+        groupAssignmentsToReset.forEach((groupId, assignment) ->
+            assignment.forEach((consumerAssignment, offsetAndMetadata) ->
+                System.out.printf(format,
+                    groupId,
+                    consumerAssignment.topic(),
+                    consumerAssignment.partition(),
+                    offsetAndMetadata.offset())));
+    }
+
+    @SuppressWarnings("ClassFanOutComplexity")
+    static class ConsumerGroupService implements AutoCloseable {
+        final ConsumerGroupCommandOptions opts;
+        final Map<String, String> configOverrides;
+        private final Admin adminClient;
+
+        ConsumerGroupService(ConsumerGroupCommandOptions opts, Map<String, 
String> configOverrides) {
+            this.opts = opts;
+            this.configOverrides = configOverrides;
+            try {
+                this.adminClient = createAdminClient(configOverrides);
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        Optional<Map<String, Map<TopicPartition, OffsetAndMetadata>>> 
resetPlanFromFile() {
+            if (opts.options.has(opts.resetFromFileOpt)) {
+                try {
+                    String resetPlanPath = 
opts.options.valueOf(opts.resetFromFileOpt);
+                    String resetPlanCsv = 
Utils.readFileAsString(resetPlanPath);
+                    Map<String, Map<TopicPartition, OffsetAndMetadata>> 
resetPlan = parseResetPlan(resetPlanCsv);
+                    return Optional.of(resetPlan);
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            } else return Optional.empty();
+        }
+
+        void listGroups() throws ExecutionException, InterruptedException {
+            boolean includeType = opts.options.has(opts.typeOpt);
+            boolean includeState = opts.options.has(opts.stateOpt);
+
+            if (includeType || includeState) {
+                Set<GroupType> types = typeValues();
+                Set<ConsumerGroupState> states = stateValues();
+                List<ConsumerGroupListing> listings = 
listConsumerGroupsWithFilters(types, states);
+
+                printGroupInfo(listings, includeType, includeState);
+            } else {
+                listConsumerGroups().forEach(System.out::println);
+            }
+        }
+
+        private Set<ConsumerGroupState> stateValues() {
+            String stateValue = opts.options.valueOf(opts.stateOpt);
+            return (stateValue == null || stateValue.isEmpty())
+                ? Collections.emptySet()
+                : consumerGroupStatesFromString(stateValue);
+        }
+
+        private Set<GroupType> typeValues() {
+            String typeValue = opts.options.valueOf(opts.typeOpt);
+            return (typeValue == null || typeValue.isEmpty())
+                ? Collections.emptySet()
+                : consumerGroupTypesFromString(typeValue);
+        }
+
+        private void printGroupInfo(List<ConsumerGroupListing> groups, boolean 
includeType, boolean includeState) {
+            Function<ConsumerGroupListing, String> groupId = 
ConsumerGroupListing::groupId;
+            Function<ConsumerGroupListing, String> groupType = groupListing -> 
groupListing.type().orElse(GroupType.UNKNOWN).toString();
+            Function<ConsumerGroupListing, String> groupState = groupListing 
-> groupListing.state().orElse(ConsumerGroupState.UNKNOWN).toString();
+
+            OptionalInt maybeMax = groups.stream().mapToInt(groupListing -> 
Math.max(15, groupId.apply(groupListing).length())).max();
+            int maxGroupLen = maybeMax.orElse(15) + 10;
+            String format = "%-" + maxGroupLen + "s";
+            List<String> header = new ArrayList<>();
+            header.add("GROUP");
+            List<Function<ConsumerGroupListing, String>> extractors = new 
ArrayList<>();
+            extractors.add(groupId);
+
+            if (includeType) {
+                header.add("TYPE");
+                extractors.add(groupType);
+                format += " %-20s";
+            }
+
+            if (includeState) {
+                header.add("STATE");
+                extractors.add(groupState);
+                format += " %-20s";
+            }
+
+            System.out.printf(format + "%n", header.toArray(new Object[0]));
+
+            for (ConsumerGroupListing groupListing : groups) {
+                Object[] info = extractors.stream().map(extractor -> 
extractor.apply(groupListing)).toArray(Object[]::new);
+                System.out.printf(format + "%n", info);
+            }
+        }
+
+        List<String> listConsumerGroups() {
+            try {
+                ListConsumerGroupsResult result = 
adminClient.listConsumerGroups(withTimeoutMs(new ListConsumerGroupsOptions()));
+                Collection<ConsumerGroupListing> listings = result.all().get();
+                return 
listings.stream().map(ConsumerGroupListing::groupId).collect(Collectors.toList());
+            } catch (InterruptedException | ExecutionException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        List<ConsumerGroupListing> 
listConsumerGroupsWithFilters(Set<GroupType> types, Set<ConsumerGroupState> 
states) throws ExecutionException, InterruptedException {
+            ListConsumerGroupsOptions listConsumerGroupsOptions = 
withTimeoutMs(new ListConsumerGroupsOptions());
+            listConsumerGroupsOptions
+                .inStates(states)
+                .withTypes(types);
+            ListConsumerGroupsResult result = 
adminClient.listConsumerGroups(listConsumerGroupsOptions);
+            return new ArrayList<>(result.all().get());
+        }
+
+        private boolean shouldPrintMemberState(String group, Optional<String> 
state, Optional<Integer> numRows) {
+            // numRows contains the number of data rows, if any, compiled from 
the API call in the caller method.
+            // if it's undefined or 0, there is no relevant group information 
to display.
+            if (!numRows.isPresent()) {
+                printError("The consumer group '" + group + "' does not 
exist.", Optional.empty());
+                return false;
+            }
+
+            int num = numRows.get();
+
+            String state0 = state.orElse("NONE");
+            switch (state0) {
+                case "Dead":
+                    printError("Consumer group '" + group + "' does not 
exist.", Optional.empty());
+                    break;
+                case "Empty":
+                    System.err.println("\nConsumer group '" + group + "' has 
no active members.");
+                    break;
+                case "PreparingRebalance":
+                case "CompletingRebalance":
+                case "Assigning":
+                case "Reconciling":
+                    System.err.println("\nWarning: Consumer group '" + group + 
"' is rebalancing.");
+                    break;
+                case "Stable":
+                    break;
+                default:
+                    // the control should never reach here
+                    throw new KafkaException("Expected a valid consumer group 
state, but found '" + state0 + "'.");
+            }
+
+            return !state0.contains("Dead") && num > 0;
+        }
+
+        private Optional<Integer> size(Optional<? extends Collection<?>> 
colOpt) {
+            return colOpt.map(Collection::size);
+        }
+
+        private void printOffsets(Map<String, Tuple2<Optional<String>, 
Optional<Collection<PartitionAssignmentState>>>> offsets) {
+            offsets.forEach((groupId, tuple) -> {
+                Optional<String> state = tuple.v1;
+                Optional<Collection<PartitionAssignmentState>> assignments = 
tuple.v2;
+
+                if (shouldPrintMemberState(groupId, state, size(assignments))) 
{
+                    // find proper columns width
+                    int maxGroupLen = 15, maxTopicLen = 15, maxConsumerIdLen = 
15, maxHostLen = 15;

Review Comment:
   Done.



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