You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "fvaleri (via GitHub)" <gi...@apache.org> on 2023/02/07 17:46:26 UTC

[GitHub] [kafka] fvaleri opened a new pull request, #13215: KAFKA-14578: Move ConsumerPerformance to tools

fvaleri opened a new pull request, #13215:
URL: https://github.com/apache/kafka/pull/13215

   ```sh
   === test data
   bin/kafka-producer-perf-test.sh --num-records 30000 --record-size 1000 --throughput 200 --topic my-topic --producer-props bootstrap.servers=:9092 linger.ms=0 batch.size=16384 ...
   30000 records sent, 199.973337 records/sec (0.19 MB/sec), 1.26 ms avg latency, 205.00 ms max latency, 1 ms 50th, 2 ms 95th, 2 ms 99th, 14 ms 99.9th.
   
   === test 1
   bin/kafka-consumer-perf-test.sh --bootstrap-server :9092 --messages 3000 --timeout 10000 --fetch-size 1048576 --topic my-topic --group my-group start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec, rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec 2023-02-07 17:53:25:127, 2023-02-07 17:53:28:418, 3.2864, 0.9986, 3446, 1047.0981, 3222, 69, 47.6284, 49942.0290 start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec, rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec 2023-02-07 17:54:23:268, 2023-02-07 17:54:26:579, 3.2806, 0.9908, 3440, 1038.9610, 96, 3215, 1.0204, 1069.9844
   
   === test 2
   bin/kafka-consumer-perf-test.sh --bootstrap-server :9092 --messages 3000 --timeout 10000 --fetch-size 100 --topic my-topic --group my-group --show-detailed-stats --reporting-interval 100 === old
   2023-02-07 17:58:09:633, 0, 0.0010, 0.0003, 1, 0.3089, 3212, 25, 0.0381, 40.0000 2023-02-07 17:58:09:733, 0, 0.2279, 2.2697, 239, 2380.0000, 0, 100, 2.2697, 2380.0000 2023-02-07 17:58:09:833, 0, 0.5789, 3.5095, 607, 3680.0000, 0, 100, 3.5095, 3680.0000 2023-02-07 17:58:09:933, 0, 1.0824, 5.0354, 1135, 5280.0000, 0, 100, 5.0354, 5280.0000 2023-02-07 17:58:10:033, 0, 1.8015, 7.1907, 1889, 7540.0000, 0, 100, 7.1907, 7540.0000 2023-02-07 17:58:10:133, 0, 2.8172, 10.1566, 2954, 10650.0000, 0, 100, 10.1566, 10650.0000 === new
   2023-02-07 17:58:39:549, 0, 0.0010, 0.0003, 1, 0.0000, 0, 3244, 0.0003, 0.3083 2023-02-07 17:58:39:649, 0, 0.2327, 2.3174, 244, 2000.0000, 0, 100, 2.3174, 2430.0000 2023-02-07 17:58:39:749, 0, 0.5293, 2.9659, 555, 3000.0000, 0, 100, 2.9659, 3110.0000 2023-02-07 17:58:39:849, 0, 0.8564, 3.2711, 898, 3000.0000, 0, 100, 3.2711, 3430.0000 2023-02-07 17:58:39:949, 0, 1.2941, 4.3774, 1357, 4000.0000, 0, 100, 4.3774, 4590.0000 2023-02-07 17:58:40:049, 0, 2.0247, 7.3051, 2123, 7000.0000, 0, 100, 7.3051, 7660.0000
   
   === test 3
   bin/kafka-consumer-perf-test.sh --bootstrap-server :9092 --messages 3000 --timeout 10000 --fetch-size 100 --topic my-topic --group my-group & bin/kafka-consumer-perf-test.sh --bootstrap-server :9092 --messages 3000 --timeout 10000 --fetch-size 100 --topic my-topic --group my-group & wait
   === old
   2023-02-07 18:10:21:826, 2023-02-07 18:10:25:780, 2.8610, 0.7236, 3000, 758.7253, 3238, 716, 3.9958, 4189.9441 2023-02-07 18:10:21:824, 2023-02-07 18:10:25:988, 2.8610, 0.6871, 3000, 720.4611, 3239, 925, 3.0930, 3243.2432 === new
   2023-02-07 18:11:10:173, 2023-02-07 18:11:14:111, 2.8610, 0.7265, 3000, 761.8080, 671, 3267, 0.8757, 918.2736 2023-02-07 18:11:10:238, 2023-02-07 18:11:14:294, 2.8610, 0.7054, 3000, 739.6450, 852, 3204, 0.8930, 936.3296
   
   === system test
   TC_PATHS="tests/kafkatest/benchmarks/core/benchmark_test.py::Benchmark.test_producer_and_consumer" bash tests/docker/run_tests.sh === old
   SESSION REPORT (ALL TESTS)
   ducktape version: 0.11.3
   session_id:       2023-02-07--006
   run time:         7 minutes 12.212 seconds
   tests run:        6
   passed:           6
   flaky:            0
   failed:           0
   ignored:          0
   === new
   SESSION REPORT (ALL TESTS)
   ducktape version: 0.11.3
   session_id:       2023-02-07--007
   run time:         7 minutes 21.450 seconds
   tests run:        6
   passed:           6
   flaky:            0
   failed:           0
   ignored:          0
   ```
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1115535441


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);

Review Comment:
   This is only related to the extended class, but you are right, we need to use AtomicLong passed by reference here. There is also another issue which I'm going to fix.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1102766672


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Integer> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .describedAs("interval_ms")
+                .ofType(Integer.class)
+                .defaultsTo(5000);
+            dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " +
+                    "See java.text.SimpleDateFormat for options.")
+                .withRequiredArg()
+                .describedAs("date format")
+                .ofType(String.class)
+                .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS");
+            hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats");
+            try {
+                options = parser.parse(args);
+            } catch (OptionException e) {
+                CommandLineUtils.printUsageAndExit(parser, e.getMessage());
+                return;
+            }
+            if (options != null) {
+                if (options.has(numThreadsOpt) || options.has(numFetchersOpt))
+                    System.out.println("WARNING: option [threads] and [num-fetch-threads] have been deprecated and will be ignored by the test");
+                CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance.");
+                CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt);
+            }
+        }
+
+        public boolean printMetrics() {
+            return options.has(printMetricsOpt);
+        }
+
+        public String brokerHostsAndPorts() {
+            return options.valueOf(options.has(bootstrapServerOpt) ? bootstrapServerOpt : brokerListOpt);
+        }
+
+        public Properties props() throws IOException {
+            Properties props = (options.has(consumerConfigOpt))
+                ? Utils.loadProps(options.valueOf(consumerConfigOpt))
+                : new Properties();
+            props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts());
+            props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt));
+            props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString());
+            props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString());
+            props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+                options.has(resetBeginningOffsetOpt) ? "latest" : "earliest");
+            props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false");
+            if (props.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null)
+                props.put(ConsumerConfig.CLIENT_ID_CONFIG, "perf-consumer-client");
+            return props;
+        }
+
+        public Set<String> topic() {
+            return Collections.singleton(options.valueOf(topicOpt));
+        }
+
+        public long numMessages() {
+            return options.valueOf(numMessagesOpt);
+        }
+
+        public int reportingIntervalMs() {
+            int value = options.valueOf(reportingIntervalOpt);
+            if (value <= 0)

Review Comment:
   You are absolutely right: same code, different behavior. Thanks for spotting this. I added validation to that option and switched to long.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison merged pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison merged PR #13215:
URL: https://github.com/apache/kafka/pull/13215


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1102766672


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Integer> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .describedAs("interval_ms")
+                .ofType(Integer.class)
+                .defaultsTo(5000);
+            dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " +
+                    "See java.text.SimpleDateFormat for options.")
+                .withRequiredArg()
+                .describedAs("date format")
+                .ofType(String.class)
+                .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS");
+            hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats");
+            try {
+                options = parser.parse(args);
+            } catch (OptionException e) {
+                CommandLineUtils.printUsageAndExit(parser, e.getMessage());
+                return;
+            }
+            if (options != null) {
+                if (options.has(numThreadsOpt) || options.has(numFetchersOpt))
+                    System.out.println("WARNING: option [threads] and [num-fetch-threads] have been deprecated and will be ignored by the test");
+                CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance.");
+                CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt);
+            }
+        }
+
+        public boolean printMetrics() {
+            return options.has(printMetricsOpt);
+        }
+
+        public String brokerHostsAndPorts() {
+            return options.valueOf(options.has(bootstrapServerOpt) ? bootstrapServerOpt : brokerListOpt);
+        }
+
+        public Properties props() throws IOException {
+            Properties props = (options.has(consumerConfigOpt))
+                ? Utils.loadProps(options.valueOf(consumerConfigOpt))
+                : new Properties();
+            props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts());
+            props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt));
+            props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString());
+            props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString());
+            props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+                options.has(resetBeginningOffsetOpt) ? "latest" : "earliest");
+            props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false");
+            if (props.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null)
+                props.put(ConsumerConfig.CLIENT_ID_CONFIG, "perf-consumer-client");
+            return props;
+        }
+
+        public Set<String> topic() {
+            return Collections.singleton(options.valueOf(topicOpt));
+        }
+
+        public long numMessages() {
+            return options.valueOf(numMessagesOpt);
+        }
+
+        public int reportingIntervalMs() {
+            int value = options.valueOf(reportingIntervalOpt);
+            if (value <= 0)

Review Comment:
   You are absolutely right: same code, different behavior. Thanks for spotting this. With my last commit I added validation to that option and default value in the description.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1115434679


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < numMessages)
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs);
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Long> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .withValuesConvertedBy(regex("^\\d+$"))

Review Comment:
   This is discussed in https://github.com/apache/kafka/pull/13215#discussion_r1102766672. Basically this validation is implicitly added in Scala, so the change is needed to have the same validation behavior.



##########
core/src/main/scala/kafka/tools/ConsumerPerformance.scala:
##########
@@ -1,306 +0,0 @@
-/**
- * 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 kafka.tools
-
-import java.text.SimpleDateFormat
-import java.time.Duration
-import java.util
-import java.util.concurrent.atomic.AtomicLong
-import java.util.{Properties, Random}
-import com.typesafe.scalalogging.LazyLogging
-import joptsimple.OptionException
-import kafka.utils.ToolsUtils
-import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer}
-import org.apache.kafka.common.serialization.ByteArrayDeserializer
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.common.{Metric, MetricName, TopicPartition}
-import org.apache.kafka.server.util.CommandLineUtils
-
-import scala.jdk.CollectionConverters._
-import scala.collection.mutable
-
-/**
- * Performance test for the full zookeeper consumer
- */
-object ConsumerPerformance extends LazyLogging {
-
-  def main(args: Array[String]): Unit = {
-
-    val config = new ConsumerPerfConfig(args)
-    logger.info("Starting consumer...")

Review Comment:
   This is the only log we have and it's pretty useless



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1115441558


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < numMessages)
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs);
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;

Review Comment:
   It is, look at `printConsumerProgress`.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1102626937


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());

Review Comment:
   Makes sense.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1102629200


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Integer> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .describedAs("interval_ms")
+                .ofType(Integer.class)
+                .defaultsTo(5000);
+            dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " +
+                    "See java.text.SimpleDateFormat for options.")
+                .withRequiredArg()
+                .describedAs("date format")
+                .ofType(String.class)
+                .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS");
+            hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats");
+            try {
+                options = parser.parse(args);
+            } catch (OptionException e) {
+                CommandLineUtils.printUsageAndExit(parser, e.getMessage());
+                return;
+            }
+            if (options != null) {
+                if (options.has(numThreadsOpt) || options.has(numFetchersOpt))
+                    System.out.println("WARNING: option [threads] and [num-fetch-threads] have been deprecated and will be ignored by the test");
+                CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance.");
+                CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt);
+            }
+        }
+
+        public boolean printMetrics() {
+            return options.has(printMetricsOpt);
+        }
+
+        public String brokerHostsAndPorts() {
+            return options.valueOf(options.has(bootstrapServerOpt) ? bootstrapServerOpt : brokerListOpt);
+        }
+
+        public Properties props() throws IOException {
+            Properties props = (options.has(consumerConfigOpt))
+                ? Utils.loadProps(options.valueOf(consumerConfigOpt))
+                : new Properties();
+            props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts());
+            props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt));
+            props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString());
+            props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString());
+            props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+                options.has(resetBeginningOffsetOpt) ? "latest" : "earliest");
+            props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false");
+            if (props.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null)
+                props.put(ConsumerConfig.CLIENT_ID_CONFIG, "perf-consumer-client");
+            return props;
+        }
+
+        public Set<String> topic() {
+            return Collections.singleton(options.valueOf(topicOpt));
+        }
+
+        public long numMessages() {
+            return options.valueOf(numMessagesOpt);
+        }
+
+        public int reportingIntervalMs() {
+            int value = options.valueOf(reportingIntervalOpt);
+            if (value <= 0)

Review Comment:
   I think the scala version performs the validation at initialisation, before starting the test. That is a minor non-impactful difference though.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1102766672


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Integer> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .describedAs("interval_ms")
+                .ofType(Integer.class)
+                .defaultsTo(5000);
+            dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " +
+                    "See java.text.SimpleDateFormat for options.")
+                .withRequiredArg()
+                .describedAs("date format")
+                .ofType(String.class)
+                .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS");
+            hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats");
+            try {
+                options = parser.parse(args);
+            } catch (OptionException e) {
+                CommandLineUtils.printUsageAndExit(parser, e.getMessage());
+                return;
+            }
+            if (options != null) {
+                if (options.has(numThreadsOpt) || options.has(numFetchersOpt))
+                    System.out.println("WARNING: option [threads] and [num-fetch-threads] have been deprecated and will be ignored by the test");
+                CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance.");
+                CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt);
+            }
+        }
+
+        public boolean printMetrics() {
+            return options.has(printMetricsOpt);
+        }
+
+        public String brokerHostsAndPorts() {
+            return options.valueOf(options.has(bootstrapServerOpt) ? bootstrapServerOpt : brokerListOpt);
+        }
+
+        public Properties props() throws IOException {
+            Properties props = (options.has(consumerConfigOpt))
+                ? Utils.loadProps(options.valueOf(consumerConfigOpt))
+                : new Properties();
+            props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts());
+            props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt));
+            props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString());
+            props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString());
+            props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+                options.has(resetBeginningOffsetOpt) ? "latest" : "earliest");
+            props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false");
+            if (props.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null)
+                props.put(ConsumerConfig.CLIENT_ID_CONFIG, "perf-consumer-client");
+            return props;
+        }
+
+        public Set<String> topic() {
+            return Collections.singleton(options.valueOf(topicOpt));
+        }
+
+        public long numMessages() {
+            return options.valueOf(numMessagesOpt);
+        }
+
+        public int reportingIntervalMs() {
+            int value = options.valueOf(reportingIntervalOpt);
+            if (value <= 0)

Review Comment:
   You are absolutely right: same code, different behavior. Thanks for spotting this. With my last commit I added validation to that option and switched to long.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1115441558


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < numMessages)
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs);
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;

Review Comment:
   It is, look at `onPartitionsRevoked`.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1099914935


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Integer> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .describedAs("interval_ms")
+                .ofType(Integer.class)
+                .defaultsTo(5000);
+            dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " +
+                    "See java.text.SimpleDateFormat for options.")
+                .withRequiredArg()
+                .describedAs("date format")
+                .ofType(String.class)
+                .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS");
+            hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats");
+            try {
+                options = parser.parse(args);
+            } catch (OptionException e) {
+                CommandLineUtils.printUsageAndExit(parser, e.getMessage());
+                return;
+            }
+            if (options != null) {
+                if (options.has(numThreadsOpt) || options.has(numFetchersOpt))
+                    System.out.println("WARNING: option [threads] and [num-fetch-threads] have been deprecated and will be ignored by the test");
+                CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance.");
+                CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt);
+            }
+        }
+
+        public boolean printMetrics() {
+            return options.has(printMetricsOpt);
+        }
+
+        public String brokerHostsAndPorts() {
+            return options.valueOf(options.has(bootstrapServerOpt) ? bootstrapServerOpt : brokerListOpt);
+        }
+
+        public Properties props() throws IOException {
+            Properties props = (options.has(consumerConfigOpt))
+                ? Utils.loadProps(options.valueOf(consumerConfigOpt))
+                : new Properties();
+            props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts());
+            props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt));
+            props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString());
+            props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString());
+            props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+                options.has(resetBeginningOffsetOpt) ? "latest" : "earliest");
+            props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false");
+            if (props.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null)
+                props.put(ConsumerConfig.CLIENT_ID_CONFIG, "perf-consumer-client");
+            return props;
+        }
+
+        public Set<String> topic() {
+            return Collections.singleton(options.valueOf(topicOpt));
+        }
+
+        public long numMessages() {
+            return options.valueOf(numMessagesOpt);
+        }
+
+        public int reportingIntervalMs() {
+            int value = options.valueOf(reportingIntervalOpt);
+            if (value <= 0)

Review Comment:
   Should this be checked at initialization time?



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());

Review Comment:
   Maybe create via instance method to add a unit test which exercises the consume on a mock?



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),

Review Comment:
   Maybe define as instance fields?



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();

Review Comment:
   nit - not needed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1123250142


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinTimeMs = new AtomicLong(0), joinTimeMsInSingleRound = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                AtomicLong joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound.get());
+                    joinTimeMsInSingleRound = new AtomicLong(0);
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < numMessages)
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs);
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long joinTimeMsInSingleRound) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, joinTimeMsInSingleRound);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long joinTimeMsInSingleRound) {
+        long fetchTimeMs = endMs - startMs - joinTimeMsInSingleRound;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", joinTimeMsInSingleRound,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinTimeMs, joinTimeMsInSingleRound;

Review Comment:
   Let's have a single field per line.



##########
core/src/main/scala/kafka/tools/ConsumerPerformance.scala:
##########
@@ -1,306 +0,0 @@
-/**
- * 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 kafka.tools
-
-import java.text.SimpleDateFormat
-import java.time.Duration
-import java.util
-import java.util.concurrent.atomic.AtomicLong
-import java.util.{Properties, Random}
-import com.typesafe.scalalogging.LazyLogging
-import joptsimple.OptionException
-import kafka.utils.ToolsUtils
-import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer}
-import org.apache.kafka.common.serialization.ByteArrayDeserializer
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.common.{Metric, MetricName, TopicPartition}
-import org.apache.kafka.server.util.CommandLineUtils
-
-import scala.jdk.CollectionConverters._
-import scala.collection.mutable
-
-/**
- * Performance test for the full zookeeper consumer
- */
-object ConsumerPerformance extends LazyLogging {
-
-  def main(args: Array[String]): Unit = {
-
-    val config = new ConsumerPerfConfig(args)
-    logger.info("Starting consumer...")

Review Comment:
   Still, not sure why we can't just keep it?



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),

Review Comment:
   Let's have a single variable per line



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1101637316


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());

Review Comment:
   Thought about this, but in the end I decided to migrate the test as is because it is similar to other tools. We can always do some refactoring later (e.g. command interface).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1101636896


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();

Review Comment:
   Right.



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+            // consumer counters
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < options.numMessages() && currentTimeMs - lastConsumedTimeMs <= options.recordFetchTimeoutMs()) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= options.reportingIntervalMs()) {
+                    if (options.showDetailedStats())
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, options.dateFormat(), joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < options.numMessages())
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", options.recordFetchTimeoutMs());
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            super();
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Integer> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .describedAs("interval_ms")
+                .ofType(Integer.class)
+                .defaultsTo(5000);
+            dateFormatOpt = parser.accepts("date-format", "The date format to use for formatting the time field. " +
+                    "See java.text.SimpleDateFormat for options.")
+                .withRequiredArg()
+                .describedAs("date format")
+                .ofType(String.class)
+                .defaultsTo("yyyy-MM-dd HH:mm:ss:SSS");
+            hideHeaderOpt = parser.accepts("hide-header", "If set, skips printing the header for the stats");
+            try {
+                options = parser.parse(args);
+            } catch (OptionException e) {
+                CommandLineUtils.printUsageAndExit(parser, e.getMessage());
+                return;
+            }
+            if (options != null) {
+                if (options.has(numThreadsOpt) || options.has(numFetchersOpt))
+                    System.out.println("WARNING: option [threads] and [num-fetch-threads] have been deprecated and will be ignored by the test");
+                CommandLineUtils.maybePrintHelpOrVersion(this, "This tool is used to verify the consumer performance.");
+                CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, numMessagesOpt);
+            }
+        }
+
+        public boolean printMetrics() {
+            return options.has(printMetricsOpt);
+        }
+
+        public String brokerHostsAndPorts() {
+            return options.valueOf(options.has(bootstrapServerOpt) ? bootstrapServerOpt : brokerListOpt);
+        }
+
+        public Properties props() throws IOException {
+            Properties props = (options.has(consumerConfigOpt))
+                ? Utils.loadProps(options.valueOf(consumerConfigOpt))
+                : new Properties();
+            props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerHostsAndPorts());
+            props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt));
+            props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString());
+            props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString());
+            props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+                options.has(resetBeginningOffsetOpt) ? "latest" : "earliest");
+            props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
+            props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false");
+            if (props.getProperty(ConsumerConfig.CLIENT_ID_CONFIG) == null)
+                props.put(ConsumerConfig.CLIENT_ID_CONFIG, "perf-consumer-client");
+            return props;
+        }
+
+        public Set<String> topic() {
+            return Collections.singleton(options.valueOf(topicOpt));
+        }
+
+        public long numMessages() {
+            return options.valueOf(numMessagesOpt);
+        }
+
+        public int reportingIntervalMs() {
+            int value = options.valueOf(reportingIntervalOpt);
+            if (value <= 0)

Review Comment:
   Looks like it's not.



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,394 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            // group counters
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),

Review Comment:
   This is a static method, so it won't make any difference. Better to stick with the original design.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on PR #13215:
URL: https://github.com/apache/kafka/pull/13215#issuecomment-1421199547

   @mimaison @OmniaGM @clolov 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on PR #13215:
URL: https://github.com/apache/kafka/pull/13215#issuecomment-1441594858

   @mimaison thanks for the review.
   
   I fixed the rebalance listener and re-ran all the test in the description. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on a diff in pull request #13215: KAFKA-14578: Move ConsumerPerformance to tools

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13215:
URL: https://github.com/apache/kafka/pull/13215#discussion_r1113147375


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < numMessages)
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs);
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;

Review Comment:
   It looks like `joinTimeMsInSingleRound` is not used.



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")

Review Comment:
   We don't need this `SuppressWarnings`



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);

Review Comment:
   This `joinTimeMsInSingleRound` will not be updated by the custom rebalance listener, is it still correct?



##########
core/src/main/scala/kafka/tools/ConsumerPerformance.scala:
##########
@@ -1,306 +0,0 @@
-/**
- * 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 kafka.tools
-
-import java.text.SimpleDateFormat
-import java.time.Duration
-import java.util
-import java.util.concurrent.atomic.AtomicLong
-import java.util.{Properties, Random}
-import com.typesafe.scalalogging.LazyLogging
-import joptsimple.OptionException
-import kafka.utils.ToolsUtils
-import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer}
-import org.apache.kafka.common.serialization.ByteArrayDeserializer
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.common.{Metric, MetricName, TopicPartition}
-import org.apache.kafka.server.util.CommandLineUtils
-
-import scala.jdk.CollectionConverters._
-import scala.collection.mutable
-
-/**
- * Performance test for the full zookeeper consumer
- */
-object ConsumerPerformance extends LazyLogging {
-
-  def main(args: Array[String]): Unit = {
-
-    val config = new ConsumerPerfConfig(args)
-    logger.info("Starting consumer...")

Review Comment:
   We seem to have lost this log line, is this expected?



##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -0,0 +1,399 @@
+/*
+ * 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.OptionException;
+import joptsimple.OptionSpec;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.utils.Exit;
+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.ToolsUtils;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static joptsimple.util.RegexMatcher.regex;
+
+public class ConsumerPerformance {
+    private static final Random RND = new Random();
+
+    public static void main(String[] args) {
+        try {
+            ConsumerPerfOptions options = new ConsumerPerfOptions(args);
+            AtomicLong totalMessagesRead = new AtomicLong(0), totalBytesRead = new AtomicLong(0),
+                joinGroupTimeMs = new AtomicLong(0);
+
+            if (!options.hideHeader())
+                printHeader(options.showDetailedStats());
+
+            KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(options.props());
+            long bytesRead = 0L, messagesRead = 0L, lastBytesRead = 0L, lastMessagesRead = 0L,
+                joinTimeMsInSingleRound = 0L;
+            long currentTimeMs = System.currentTimeMillis();
+            long joinStartMs = currentTimeMs;
+            long startMs = currentTimeMs;
+            consume(consumer, options, totalMessagesRead, totalBytesRead, joinGroupTimeMs,
+                bytesRead, messagesRead, lastBytesRead, lastMessagesRead,
+                joinStartMs, joinTimeMsInSingleRound);
+            long endMs = System.currentTimeMillis();
+
+            Map<MetricName, ? extends Metric> metrics = null;
+            if (options.printMetrics())
+                metrics = consumer.metrics();
+            consumer.close();
+
+            // print final stats
+            double elapsedSec = (endMs - startMs) / 1_000.0;
+            long fetchTimeInMs = (endMs - startMs) - joinGroupTimeMs.get();
+            if (!options.showDetailedStats()) {
+                double totalMbRead = (totalBytesRead.get() * 1.0) / (1024 * 1024);
+                System.out.printf("%s, %s, %.4f, %.4f, %d, %.4f, %d, %d, %.4f, %.4f%n",
+                    options.dateFormat().format(startMs),
+                    options.dateFormat().format(endMs),
+                    totalMbRead,
+                    totalMbRead / elapsedSec,
+                    totalMessagesRead.get(),
+                    totalMessagesRead.get() / elapsedSec,
+                    joinGroupTimeMs.get(),
+                    fetchTimeInMs,
+                    totalMbRead / (fetchTimeInMs / 1000.0),
+                    totalMessagesRead.get() / (fetchTimeInMs / 1000.0)
+                );
+            }
+
+            if (metrics != null)
+                ToolsUtils.printMetrics(metrics);
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            Exit.exit(1);
+        }
+    }
+
+    protected static void printHeader(boolean showDetailedStats) {
+        String newFieldsInHeader = ", rebalance.time.ms, fetch.time.ms, fetch.MB.sec, fetch.nMsg.sec";
+        if (!showDetailedStats)
+            System.out.printf("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+        else
+            System.out.printf("time, threadId, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec%s%n", newFieldsInHeader);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void consume(KafkaConsumer<byte[], byte[]> consumer,
+                                ConsumerPerfOptions options,
+                                AtomicLong totalMessagesRead,
+                                AtomicLong totalBytesRead,
+                                AtomicLong joinGroupTimeMs,
+                                long bytesRead,
+                                long messagesRead,
+                                long lastBytesRead,
+                                long lastMessagesRead,
+                                long joinStartMs,
+                                long joinTimeMsInSingleRound) {
+        long numMessages = options.numMessages();
+        long recordFetchTimeoutMs = options.recordFetchTimeoutMs();
+        long reportingIntervalMs = options.reportingIntervalMs();
+        boolean showDetailedStats = options.showDetailedStats();
+        SimpleDateFormat dateFormat = options.dateFormat();
+        consumer.subscribe(options.topic(),
+            new ConsumerPerfRebListener(joinGroupTimeMs, joinStartMs, joinTimeMsInSingleRound));
+
+        // now start the benchmark
+        long currentTimeMs = System.currentTimeMillis();
+        long lastReportTimeMs = currentTimeMs;
+        long lastConsumedTimeMs = currentTimeMs;
+
+        while (messagesRead < numMessages && currentTimeMs - lastConsumedTimeMs <= recordFetchTimeoutMs) {
+            ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100));
+            currentTimeMs = System.currentTimeMillis();
+            if (!records.isEmpty())
+                lastConsumedTimeMs = currentTimeMs;
+            for (ConsumerRecord<byte[], byte[]> record : records) {
+                messagesRead += 1;
+                if (record.key() != null)
+                    bytesRead += record.key().length;
+                if (record.value() != null)
+                    bytesRead += record.value().length;
+                if (currentTimeMs - lastReportTimeMs >= reportingIntervalMs) {
+                    if (showDetailedStats)
+                        printConsumerProgress(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead,
+                            lastReportTimeMs, currentTimeMs, dateFormat, joinTimeMsInSingleRound);
+                    joinTimeMsInSingleRound = 0L;
+                    lastReportTimeMs = currentTimeMs;
+                    lastMessagesRead = messagesRead;
+                    lastBytesRead = bytesRead;
+                }
+            }
+        }
+
+        if (messagesRead < numMessages)
+            System.out.printf("WARNING: Exiting before consuming the expected number of messages: timeout (%d ms) exceeded. " +
+                "You can use the --timeout option to increase the timeout.%n", recordFetchTimeoutMs);
+        totalMessagesRead.set(messagesRead);
+        totalBytesRead.set(bytesRead);
+    }
+
+    protected static void printConsumerProgress(int id,
+                                                long bytesRead,
+                                                long lastBytesRead,
+                                                long messagesRead,
+                                                long lastMessagesRead,
+                                                long startMs,
+                                                long endMs,
+                                                SimpleDateFormat dateFormat,
+                                                long periodicJoinTimeInMs) {
+        printBasicProgress(id, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, dateFormat);
+        printExtendedProgress(bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, endMs, periodicJoinTimeInMs);
+        System.out.println();
+    }
+
+    private static void printBasicProgress(int id,
+                                           long bytesRead,
+                                           long lastBytesRead,
+                                           long messagesRead,
+                                           long lastMessagesRead,
+                                           long startMs,
+                                           long endMs,
+                                           SimpleDateFormat dateFormat) {
+        double elapsedMs = endMs - startMs;
+        double totalMbRead = (bytesRead * 1.0) / (1024 * 1024);
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        double intervalMbPerSec = 1000.0 * intervalMbRead / elapsedMs;
+        double intervalMessagesPerSec = ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0;
+        System.out.printf("%s, %d, %.4f, %.4f, %d, %.4f", dateFormat.format(endMs), id,
+            totalMbRead, intervalMbPerSec, messagesRead, intervalMessagesPerSec);
+    }
+
+    private static void printExtendedProgress(long bytesRead,
+                                              long lastBytesRead,
+                                              long messagesRead,
+                                              long lastMessagesRead,
+                                              long startMs,
+                                              long endMs,
+                                              long periodicJoinTimeInMs) {
+        long fetchTimeMs = endMs - startMs - periodicJoinTimeInMs;
+        double intervalMbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024);
+        long intervalMessagesRead = messagesRead - lastMessagesRead;
+        double intervalMbPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMbRead / fetchTimeMs;
+        double intervalMessagesPerSec = (fetchTimeMs <= 0) ? 0.0 : 1000.0 * intervalMessagesRead / fetchTimeMs;
+        System.out.printf(", %d, %d, %.4f, %.4f", periodicJoinTimeInMs,
+            fetchTimeMs, intervalMbPerSec, intervalMessagesPerSec);
+    }
+
+    public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
+        private AtomicLong joinGroupTimeMs;
+        private long joinStartMs, joinTimeMsInSingleRound;
+
+        public ConsumerPerfRebListener(AtomicLong joinGroupTimeMs, long joinStartMs, long joinTimeMsInSingleRound) {
+            this.joinGroupTimeMs = joinGroupTimeMs;
+            this.joinStartMs = joinStartMs;
+            this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            long elapsedMs = System.currentTimeMillis() - joinStartMs;
+            joinGroupTimeMs.addAndGet(elapsedMs);
+            joinTimeMsInSingleRound += elapsedMs;
+        }
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            joinStartMs = System.currentTimeMillis();
+        }
+    }
+
+    protected static class ConsumerPerfOptions extends CommandDefaultOptions {
+        private final OptionSpec<String> brokerListOpt;
+        private final OptionSpec<String> bootstrapServerOpt;
+        private final OptionSpec<String> topicOpt;
+        private final OptionSpec<String> groupIdOpt;
+        private final OptionSpec<Integer> fetchSizeOpt;
+        private final OptionSpec<Void> resetBeginningOffsetOpt;
+        private final OptionSpec<Integer> socketBufferSizeOpt;
+        private final OptionSpec<Integer> numThreadsOpt;
+        private final OptionSpec<Integer> numFetchersOpt;
+        private final OptionSpec<String> consumerConfigOpt;
+        private final OptionSpec<Void> printMetricsOpt;
+        private final OptionSpec<Void> showDetailedStatsOpt;
+        private final OptionSpec<Long> recordFetchTimeoutOpt;
+        private final OptionSpec<Long> numMessagesOpt;
+        private final OptionSpec<Long> reportingIntervalOpt;
+        private final OptionSpec<String> dateFormatOpt;
+        private final OptionSpec<Void> hideHeaderOpt;
+
+        public ConsumerPerfOptions(String[] args) {
+            super(args);
+            brokerListOpt = parser.accepts("broker-list", "DEPRECATED, use --bootstrap-server instead; ignored if --bootstrap-server is specified. The broker list string in the form HOST1:PORT1,HOST2:PORT2.")
+                .withRequiredArg()
+                .describedAs("broker-list")
+                .ofType(String.class);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED unless --broker-list(deprecated) is specified. The server(s) to connect to.")
+                .requiredUnless("broker-list")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            groupIdOpt = parser.accepts("group", "The group id to consume on.")
+                .withRequiredArg()
+                .describedAs("gid")
+                .defaultsTo("perf-consumer-" + RND.nextInt(100_000))
+                .ofType(String.class);
+            fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(1024 * 1024);
+            resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " +
+                "offset to consume from, start with the latest message present in the log rather than the earliest message.");
+            socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.")
+                .withRequiredArg()
+                .describedAs("size")
+                .ofType(Integer.class)
+                .defaultsTo(2 * 1024 * 1024);
+            numThreadsOpt = parser.accepts("threads", "DEPRECATED AND IGNORED: Number of processing threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(10);
+            numFetchersOpt = parser.accepts("num-fetch-threads", "DEPRECATED AND IGNORED: Number of fetcher threads.")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Integer.class)
+                .defaultsTo(1);
+            consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file.")
+                .withRequiredArg()
+                .describedAs("config file")
+                .ofType(String.class);
+            printMetricsOpt = parser.accepts("print-metrics", "Print out the metrics.");
+            showDetailedStatsOpt = parser.accepts("show-detailed-stats", "If set, stats are reported for each reporting " +
+                "interval as configured by reporting-interval");
+            recordFetchTimeoutOpt = parser.accepts("timeout", "The maximum allowed time in milliseconds between returned records.")
+                .withOptionalArg()
+                .describedAs("milliseconds")
+                .ofType(Long.class)
+                .defaultsTo(10_000L);
+            numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
+                .withRequiredArg()
+                .describedAs("count")
+                .ofType(Long.class);
+            reportingIntervalOpt = parser.accepts("reporting-interval", "Interval in milliseconds at which to print progress info.")
+                .withRequiredArg()
+                .withValuesConvertedBy(regex("^\\d+$"))

Review Comment:
   Why do we need this?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org