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


##########
tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java:
##########
@@ -301,24 +301,30 @@ public void 
testTransactionLogMessageFormatter(ClusterInstance cluster) throws E
             };
 
             ConsoleConsumerOptions options = new 
ConsoleConsumerOptions(transactionLogMessageFormatter);
-            ByteArrayOutputStream out = new ByteArrayOutputStream();
-            PrintStream output = new PrintStream(out);
-            ConsoleConsumer.process(1, options.formatter(),
-                    new ConsoleConsumer.ConsumerWrapper(options, 
createConsumer(cluster)), output, true);
-            JsonNode jsonNode = 
objectMapper.reader().readTree(out.toByteArray());
-            JsonNode keyNode = jsonNode.get("key");
             
-            TransactionLogKey logKey =
-                    TransactionLogKeyJsonConverter.read(keyNode.get("data"), 
TransactionLogKey.HIGHEST_SUPPORTED_VERSION);
-            assertNotNull(logKey);
-            assertEquals(transactionId, logKey.transactionalId());
-            
-            JsonNode valueNode = jsonNode.get("value");
-            TransactionLogValue logValue =
-                    
TransactionLogValueJsonConverter.read(valueNode.get("data"), 
TransactionLogValue.HIGHEST_SUPPORTED_VERSION);
-            assertNotNull(logValue);
-            assertEquals(0, logValue.producerId());
-            assertEquals(0, logValue.transactionStatus());
+            try (ByteArrayOutputStream out = new ByteArrayOutputStream();
+                 PrintStream output = new PrintStream(out)) {
+
+                ConsoleConsumer.ConsumerWrapper consumerWrapper = new 
ConsoleConsumer.ConsumerWrapper(options, createConsumer(cluster));
+                ConsoleConsumer.process(1, options.formatter(), 
consumerWrapper, output, true);
+                
+                JsonNode jsonNode = 
objectMapper.reader().readTree(out.toByteArray());
+                JsonNode keyNode = jsonNode.get("key");
+
+                TransactionLogKey logKey =
+                        
TransactionLogKeyJsonConverter.read(keyNode.get("data"), 
TransactionLogKey.HIGHEST_SUPPORTED_VERSION);
+                assertNotNull(logKey);
+                assertEquals(transactionId, logKey.transactionalId());
+
+                JsonNode valueNode = jsonNode.get("value");
+                TransactionLogValue logValue =
+                        
TransactionLogValueJsonConverter.read(valueNode.get("data"), 
TransactionLogValue.HIGHEST_SUPPORTED_VERSION);
+                assertNotNull(logValue);
+                assertEquals(0, logValue.producerId());
+                assertEquals(0, logValue.transactionStatus());
+                
+                consumerWrapper.cleanup();

Review Comment:
   Please make sure `cleanup` gets called.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java:
##########
@@ -49,8 +88,14 @@
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@ExtendWith(value = ClusterTestExtensions.class)

Review Comment:
   `value` is redundant.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java:
##########
@@ -49,8 +88,14 @@
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+@ExtendWith(value = ClusterTestExtensions.class)
+@Tag("integration")

Review Comment:
   we don't need this now, since the `tag` is added automatically. 



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