fvaleri commented on code in PR #14059:
URL: https://github.com/apache/kafka/pull/14059#discussion_r1270643655


##########
tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java:
##########
@@ -0,0 +1,740 @@
+/*
+ * 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;
+
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientRequest;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.ClientUtils;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.ManualMetadataUpdater;
+import org.apache.kafka.clients.NetworkClient;
+import org.apache.kafka.clients.NetworkClientUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.FetchResponseData;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.network.ChannelBuilder;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.Selectable;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.FetchRequest;
+import org.apache.kafka.common.requests.FetchResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.apache.kafka.server.util.ToolsUtils;
+import org.apache.kafka.server.util.TopicFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.SocketTimeoutException;
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.Collectors;
+
+import static java.lang.String.format;
+
+/**
+ * For verifying the consistency among replicas.
+ * <p>
+ *  1. start a fetcher on every broker
+ *  2. each fetcher does the following
+ *    2.1 issues fetch request
+ *    2.2 puts the fetched result in a shared buffer
+ *    2.3 waits for all other fetchers to finish step 2.2
+ *    2.4 one of the fetchers verifies the consistency of fetched results 
among replicas
+ * <p>
+ * The consistency verification is up to the high watermark. The tool reports 
the
+ * max lag between the verified offset and the high watermark among all 
partitions.
+ * <p>
+ * If a broker goes down, the verification of the partitions on that broker is 
delayed
+ * until the broker is up again.
+ * <p>
+ * Caveats:
+ * 1. The tools needs all brokers to be up at startup time.
+ * 2. The tool doesn't handle out of range offsets.
+ */
+public class ReplicaVerificationTool {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ReplicaVerificationTool.class);
+    private static final SimpleDateFormat DATE_FORMAT = new 
SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+
+    public static void main(String[] args) {
+        try {
+            ReplicaVerificationToolOptions options = new 
ReplicaVerificationToolOptions(args);
+            // getting topic metadata
+            LOG.info("Getting topic metadata...");
+            String bootstrapServer = options.brokerHostsAndPorts();
+
+            try (Admin adminClient = createAdminClient(bootstrapServer)) {
+                Collection<TopicDescription> topicsMetadata = 
listTopicsMetadata(adminClient);
+                Map<Integer, Node> brokerInfo = brokerDetails(adminClient);
+
+                Map<String, Uuid> topicIds = 
topicsMetadata.stream().collect(Collectors.toMap(TopicDescription::name, 
TopicDescription::topicId));
+
+                List<TopicDescription> filteredTopicMetadata = 
topicsMetadata.stream().filter(
+                    topicMetadata -> 
options.topicsIncludeFilter().isTopicAllowed(topicMetadata.name(), false)
+                ).collect(Collectors.toList());
+
+                if (filteredTopicMetadata.isEmpty()) {
+                    LOG.error("No topics found. {} if specified, is either 
filtering out all topics or there is no topic.", options.topicsIncludeOpt);
+                    Exit.exit(1);
+                }
+
+                List<TopicPartitionReplica> topicPartitionReplicas = 
filteredTopicMetadata.stream().flatMap(
+                    topicMetadata -> 
topicMetadata.partitions().stream().flatMap(
+                        partitionMetadata -> 
partitionMetadata.replicas().stream().map(
+                            node -> new 
TopicPartitionReplica(topicMetadata.name(), partitionMetadata.partition(), 
node.id())
+                        )
+                    )
+                ).collect(Collectors.toList());
+                LOG.debug("Selected topic partitions: {}", 
topicPartitionReplicas);
+
+                Map<Integer, List<TopicPartition>> brokerToTopicPartitions = 
topicPartitionReplicas.stream()
+                    .collect(Collectors.groupingBy(
+                        TopicPartitionReplica::brokerId,
+                        Collectors.mapping(
+                            replica -> new TopicPartition(replica.topic(), 
replica.partition()),
+                            Collectors.toList()
+                        )
+                    ));
+                LOG.debug("Topic partitions per broker: {}", 
brokerToTopicPartitions);
+
+                Map<TopicPartition, Integer> expectedReplicasPerTopicPartition 
= topicPartitionReplicas.stream()
+                    .collect(Collectors.groupingBy(
+                        replica -> new TopicPartition(replica.topic(), 
replica.partition()),
+                        Collectors.collectingAndThen(
+                            Collectors.toList(),
+                            List::size
+                        )
+                    ));
+                LOG.debug("Expected replicas per topic partition: {}", 
expectedReplicasPerTopicPartition);
+
+                List<TopicPartition> topicPartitions = 
filteredTopicMetadata.stream()
+                    .flatMap(topicMetadata -> 
topicMetadata.partitions().stream()
+                        .map(partitionMetadata -> new 
TopicPartition(topicMetadata.name(), partitionMetadata.partition()))
+                    )
+                    .collect(Collectors.toList());
+
+                Properties consumerProps = consumerConfig(bootstrapServer);
+
+                ReplicaBuffer replicaBuffer = new 
ReplicaBuffer(expectedReplicasPerTopicPartition,
+                    initialOffsets(topicPartitions, consumerProps, 
options.initialOffsetTime()),
+                    brokerToTopicPartitions.size(), options.reportInterval());
+
+                // create all replica fetcher threads
+                int verificationBrokerId = 
brokerToTopicPartitions.entrySet().iterator().next().getKey();
+                AtomicInteger counter = new AtomicInteger(0);
+                List<ReplicaFetcher> fetcherThreads = 
brokerToTopicPartitions.entrySet().stream()
+                    .map(entry -> {
+                        int brokerId = entry.getKey();
+                        Iterable<TopicPartition> partitions = entry.getValue();
+                        return new ReplicaFetcher(
+                            "ReplicaFetcher-" + brokerId,
+                            brokerInfo.get(brokerId),
+                            partitions,
+                            topicIds,
+                            replicaBuffer,
+                            options.fetchSize(),
+                            options.maxWaitMs(),
+                            1,
+                            brokerId == verificationBrokerId,
+                            consumerProps,
+                            counter.incrementAndGet()
+                        );
+                    })
+                    .collect(Collectors.toList());
+
+                Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+                    LOG.info("Stopping all fetchers");
+                    fetcherThreads.forEach(replicaFetcher -> {
+                        try {
+                            replicaFetcher.shutdown();
+                        } catch (InterruptedException ignored) {
+                        }
+                    });
+                }));
+
+                fetcherThreads.forEach(Thread::start);
+                System.out.printf("%s: verification process is started%n",
+                    DATE_FORMAT.format(new Date(Time.SYSTEM.milliseconds())));
+            }
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    private static Map<TopicPartition, Long> 
initialOffsets(List<TopicPartition> topicPartitions, Properties consumerConfig, 
long initialOffsetTime) {
+        try (KafkaConsumer<String, String> consumer = new 
KafkaConsumer<>(consumerConfig)) {
+            if (ListOffsetsRequest.LATEST_TIMESTAMP == initialOffsetTime) {
+                return consumer.endOffsets(topicPartitions).entrySet().stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, 
Map.Entry::getValue));
+            } else if (ListOffsetsRequest.EARLIEST_TIMESTAMP == 
initialOffsetTime) {
+                return 
consumer.beginningOffsets(topicPartitions).entrySet().stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, 
Map.Entry::getValue));
+            } else {
+                Map<TopicPartition, Long> timestampsToSearch = 
topicPartitions.stream()
+                    .collect(Collectors.toMap(Function.identity(), tp -> 
initialOffsetTime));
+                return 
consumer.offsetsForTimes(timestampsToSearch).entrySet().stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, e -> 
e.getValue().offset()));
+            }
+        }
+    }
+
+    private static Properties consumerConfig(String brokerUrl) {
+        Properties properties = new Properties();
+        properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl);
+        properties.put(ConsumerConfig.GROUP_ID_CONFIG, "ReplicaVerification");
+        properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        return properties;
+    }
+
+    private static Map<Integer, Node> brokerDetails(Admin adminClient) throws 
ExecutionException, InterruptedException {
+        return 
adminClient.describeCluster().nodes().get().stream().collect(Collectors.toMap(Node::id,
 Function.identity()));
+    }
+
+    private static Collection<TopicDescription> listTopicsMetadata(Admin 
adminClient) throws ExecutionException, InterruptedException {
+        Set<String> topics = adminClient.listTopics(new 
ListTopicsOptions().listInternal(true)).names().get();
+        return 
adminClient.describeTopics(topics).allTopicNames().get().values();
+    }
+
+    private static Admin createAdminClient(String bootstrapServer) {
+        Properties props = new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, 
bootstrapServer);
+        return Admin.create(props);
+    }
+
+    private static class ReplicaVerificationToolOptions extends 
CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Integer> maxWaitMsOpt;
+        private final OptionSpec<String> topicWhiteListOpt;
+        private final OptionSpec<String> topicsIncludeOpt;
+        private final OptionSpec<Long> initialOffsetTimeOpt;
+        private final OptionSpec<Long> reportIntervalOpt;
+
+        ReplicaVerificationToolOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server",

Review Comment:
   Removing extra changes.



##########
tools/src/main/java/org/apache/kafka/tools/ReplicaVerificationTool.java:
##########
@@ -0,0 +1,740 @@
+/*
+ * 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;
+
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.ApiVersions;
+import org.apache.kafka.clients.ClientRequest;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.ClientUtils;
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.ManualMetadataUpdater;
+import org.apache.kafka.clients.NetworkClient;
+import org.apache.kafka.clients.NetworkClientUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.FetchResponseData;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.network.ChannelBuilder;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.Selectable;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.RecordBatch;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.FetchRequest;
+import org.apache.kafka.common.requests.FetchResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.ShutdownableThread;
+import org.apache.kafka.server.util.ToolsUtils;
+import org.apache.kafka.server.util.TopicFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.SocketTimeoutException;
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.Collectors;
+
+import static java.lang.String.format;
+
+/**
+ * For verifying the consistency among replicas.
+ * <p>
+ *  1. start a fetcher on every broker
+ *  2. each fetcher does the following
+ *    2.1 issues fetch request
+ *    2.2 puts the fetched result in a shared buffer
+ *    2.3 waits for all other fetchers to finish step 2.2
+ *    2.4 one of the fetchers verifies the consistency of fetched results 
among replicas
+ * <p>
+ * The consistency verification is up to the high watermark. The tool reports 
the
+ * max lag between the verified offset and the high watermark among all 
partitions.
+ * <p>
+ * If a broker goes down, the verification of the partitions on that broker is 
delayed
+ * until the broker is up again.
+ * <p>
+ * Caveats:
+ * 1. The tools needs all brokers to be up at startup time.
+ * 2. The tool doesn't handle out of range offsets.
+ */
+public class ReplicaVerificationTool {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ReplicaVerificationTool.class);
+    private static final SimpleDateFormat DATE_FORMAT = new 
SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+
+    public static void main(String[] args) {
+        try {
+            ReplicaVerificationToolOptions options = new 
ReplicaVerificationToolOptions(args);
+            // getting topic metadata
+            LOG.info("Getting topic metadata...");
+            String bootstrapServer = options.brokerHostsAndPorts();
+
+            try (Admin adminClient = createAdminClient(bootstrapServer)) {
+                Collection<TopicDescription> topicsMetadata = 
listTopicsMetadata(adminClient);
+                Map<Integer, Node> brokerInfo = brokerDetails(adminClient);
+
+                Map<String, Uuid> topicIds = 
topicsMetadata.stream().collect(Collectors.toMap(TopicDescription::name, 
TopicDescription::topicId));
+
+                List<TopicDescription> filteredTopicMetadata = 
topicsMetadata.stream().filter(
+                    topicMetadata -> 
options.topicsIncludeFilter().isTopicAllowed(topicMetadata.name(), false)
+                ).collect(Collectors.toList());
+
+                if (filteredTopicMetadata.isEmpty()) {
+                    LOG.error("No topics found. {} if specified, is either 
filtering out all topics or there is no topic.", options.topicsIncludeOpt);
+                    Exit.exit(1);
+                }
+
+                List<TopicPartitionReplica> topicPartitionReplicas = 
filteredTopicMetadata.stream().flatMap(
+                    topicMetadata -> 
topicMetadata.partitions().stream().flatMap(
+                        partitionMetadata -> 
partitionMetadata.replicas().stream().map(
+                            node -> new 
TopicPartitionReplica(topicMetadata.name(), partitionMetadata.partition(), 
node.id())
+                        )
+                    )
+                ).collect(Collectors.toList());
+                LOG.debug("Selected topic partitions: {}", 
topicPartitionReplicas);
+
+                Map<Integer, List<TopicPartition>> brokerToTopicPartitions = 
topicPartitionReplicas.stream()
+                    .collect(Collectors.groupingBy(
+                        TopicPartitionReplica::brokerId,
+                        Collectors.mapping(
+                            replica -> new TopicPartition(replica.topic(), 
replica.partition()),
+                            Collectors.toList()
+                        )
+                    ));
+                LOG.debug("Topic partitions per broker: {}", 
brokerToTopicPartitions);
+
+                Map<TopicPartition, Integer> expectedReplicasPerTopicPartition 
= topicPartitionReplicas.stream()
+                    .collect(Collectors.groupingBy(
+                        replica -> new TopicPartition(replica.topic(), 
replica.partition()),
+                        Collectors.collectingAndThen(
+                            Collectors.toList(),
+                            List::size
+                        )
+                    ));
+                LOG.debug("Expected replicas per topic partition: {}", 
expectedReplicasPerTopicPartition);
+
+                List<TopicPartition> topicPartitions = 
filteredTopicMetadata.stream()
+                    .flatMap(topicMetadata -> 
topicMetadata.partitions().stream()
+                        .map(partitionMetadata -> new 
TopicPartition(topicMetadata.name(), partitionMetadata.partition()))
+                    )
+                    .collect(Collectors.toList());
+
+                Properties consumerProps = consumerConfig(bootstrapServer);
+
+                ReplicaBuffer replicaBuffer = new 
ReplicaBuffer(expectedReplicasPerTopicPartition,
+                    initialOffsets(topicPartitions, consumerProps, 
options.initialOffsetTime()),
+                    brokerToTopicPartitions.size(), options.reportInterval());
+
+                // create all replica fetcher threads
+                int verificationBrokerId = 
brokerToTopicPartitions.entrySet().iterator().next().getKey();
+                AtomicInteger counter = new AtomicInteger(0);
+                List<ReplicaFetcher> fetcherThreads = 
brokerToTopicPartitions.entrySet().stream()
+                    .map(entry -> {
+                        int brokerId = entry.getKey();
+                        Iterable<TopicPartition> partitions = entry.getValue();
+                        return new ReplicaFetcher(
+                            "ReplicaFetcher-" + brokerId,
+                            brokerInfo.get(brokerId),
+                            partitions,
+                            topicIds,
+                            replicaBuffer,
+                            options.fetchSize(),
+                            options.maxWaitMs(),
+                            1,
+                            brokerId == verificationBrokerId,
+                            consumerProps,
+                            counter.incrementAndGet()
+                        );
+                    })
+                    .collect(Collectors.toList());
+
+                Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+                    LOG.info("Stopping all fetchers");
+                    fetcherThreads.forEach(replicaFetcher -> {
+                        try {
+                            replicaFetcher.shutdown();
+                        } catch (InterruptedException ignored) {
+                        }
+                    });
+                }));
+
+                fetcherThreads.forEach(Thread::start);
+                System.out.printf("%s: verification process is started%n",
+                    DATE_FORMAT.format(new Date(Time.SYSTEM.milliseconds())));
+            }
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    private static Map<TopicPartition, Long> 
initialOffsets(List<TopicPartition> topicPartitions, Properties consumerConfig, 
long initialOffsetTime) {
+        try (KafkaConsumer<String, String> consumer = new 
KafkaConsumer<>(consumerConfig)) {
+            if (ListOffsetsRequest.LATEST_TIMESTAMP == initialOffsetTime) {
+                return consumer.endOffsets(topicPartitions).entrySet().stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, 
Map.Entry::getValue));
+            } else if (ListOffsetsRequest.EARLIEST_TIMESTAMP == 
initialOffsetTime) {
+                return 
consumer.beginningOffsets(topicPartitions).entrySet().stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, 
Map.Entry::getValue));
+            } else {
+                Map<TopicPartition, Long> timestampsToSearch = 
topicPartitions.stream()
+                    .collect(Collectors.toMap(Function.identity(), tp -> 
initialOffsetTime));
+                return 
consumer.offsetsForTimes(timestampsToSearch).entrySet().stream()
+                    .collect(Collectors.toMap(Map.Entry::getKey, e -> 
e.getValue().offset()));
+            }
+        }
+    }
+
+    private static Properties consumerConfig(String brokerUrl) {
+        Properties properties = new Properties();
+        properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl);
+        properties.put(ConsumerConfig.GROUP_ID_CONFIG, "ReplicaVerification");
+        properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        return properties;
+    }
+
+    private static Map<Integer, Node> brokerDetails(Admin adminClient) throws 
ExecutionException, InterruptedException {
+        return 
adminClient.describeCluster().nodes().get().stream().collect(Collectors.toMap(Node::id,
 Function.identity()));
+    }
+
+    private static Collection<TopicDescription> listTopicsMetadata(Admin 
adminClient) throws ExecutionException, InterruptedException {
+        Set<String> topics = adminClient.listTopics(new 
ListTopicsOptions().listInternal(true)).names().get();
+        return 
adminClient.describeTopics(topics).allTopicNames().get().values();
+    }
+
+    private static Admin createAdminClient(String bootstrapServer) {
+        Properties props = new Properties();
+        props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, 
bootstrapServer);
+        return Admin.create(props);
+    }
+
+    private static class ReplicaVerificationToolOptions extends 
CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Integer> maxWaitMsOpt;
+        private final OptionSpec<String> topicWhiteListOpt;
+        private final OptionSpec<String> topicsIncludeOpt;
+        private final OptionSpec<Long> initialOffsetTimeOpt;
+        private final OptionSpec<Long> reportIntervalOpt;
+
+        ReplicaVerificationToolOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server",
+                    "REQUIRED. The list of hostname and port of the server to 
connect to.")
+                .withRequiredArg()
+                .describedAs("hostname:port,...,hostname:port")
+                .ofType(String.class);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED. Use 
--bootstrap-server. " +
+                    "The list of hostname and port of the server to connect 
to.")
+                .withOptionalArg()
+                .describedAs("hostname:port,...,hostname:port")
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of 
each request.")
+                .withRequiredArg()
+                .describedAs("bytes")
+                .ofType(Integer.class)
+                .defaultsTo(ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES);
+            maxWaitMsOpt = parser.accepts("max-wait-ms", "The max amount of 
time each fetch request waits.")
+                .withRequiredArg()
+                .describedAs("ms")
+                .ofType(Integer.class)
+                .defaultsTo(1_000);
+            topicWhiteListOpt = parser.accepts("topic-white-list", 
"DEPRECATED. Use --topics-include. " +
+                    "List of topics to verify replica consistency.")
+                .withRequiredArg()
+                .describedAs("Java regex (String)")
+                .ofType(String.class)
+                .defaultsTo(".*");
+            topicsIncludeOpt = parser.accepts("topics-include", "List of 
topics to verify replica consistency.")
+                .withRequiredArg()
+                .describedAs("Java regex (String)")
+                .ofType(String.class)
+                .defaultsTo(".*");
+            initialOffsetTimeOpt = parser.accepts("time", "Timestamp for 
getting the initial offsets.")
+                .withRequiredArg()
+                .describedAs("timestamp/-1(latest)/-2(earliest)")
+                .ofType(Long.class)
+                .defaultsTo(-1L);
+            reportIntervalOpt = parser.accepts("report-interval-ms", "The 
reporting interval.")
+                .withRequiredArg()
+                .describedAs("ms")
+                .ofType(Long.class)
+                .defaultsTo(30_000L);
+            options = parser.parse(args);
+            if (args.length == 0 || options.has(helpOpt)) {
+                CommandLineUtils.printUsageAndExit(parser, "Validate that all 
replicas for a set of topics have the same data.");
+            }
+            if (options.has(versionOpt)) {
+                CommandLineUtils.printVersionAndExit();
+            }
+            CommandLineUtils.checkInvalidArgs(parser, options, 
bootstrapServerOpt, brokerListOpt);
+            CommandLineUtils.checkInvalidArgs(parser, options, 
topicsIncludeOpt, topicWhiteListOpt);
+            if (!options.has(bootstrapServerOpt) && 
!options.has(brokerListOpt)) {
+                CommandLineUtils.printUsageAndExit(parser, format("The %s 
option is required", bootstrapServerOpt));
+            }
+            if (options.has(brokerListOpt)) {
+                System.out.printf("WARNING: The %s option is deprecated and 
will be removed. " +
+                    "Use the %s option with the same syntax.%n", 
brokerListOpt, bootstrapServerOpt);
+            }
+            if (options.has(topicWhiteListOpt)) {

Review Comment:
   Removing extra changes.



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