chia7712 commented on code in PR #20267:
URL: https://github.com/apache/kafka/pull/20267#discussion_r2318276501


##########
tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java:
##########
@@ -66,45 +69,42 @@ public static void main(String[] args) {
             if (!options.hideHeader())
                 printHeader(options.showDetailedStats());
 
-            KafkaConsumer<byte[], byte[]> consumer = new 
KafkaConsumer<>(options.props());
-            long bytesRead = 0L;
-            long messagesRead = 0L;
-            long lastBytesRead = 0L;
-            long 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)
-                );
-            }
+            try (Consumer<byte[], byte[]> consumer = 
consumerCreator.apply(options.props())) {
+                long bytesRead = 0L;
+                long messagesRead = 0L;
+                long lastBytesRead = 0L;
+                long 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();
+
+                // 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);
+                if (options.printMetrics()) {

Review Comment:
   Great find. It looks like `ShareConsumerPerformance` has a similar issue. 
@kirktrue do you have some time to fix it as well
   
   
https://github.com/apache/kafka/blob/4271fd8c8b17cc7f31be027a7c25fd31ea11a7d7/tools/src/main/java/org/apache/kafka/tools/ShareConsumerPerformance.java#L86



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