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

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

Reply via email to