cadonna commented on code in PR #12465:
URL: https://github.com/apache/kafka/pull/12465#discussion_r964580803


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java:
##########
@@ -252,10 +252,15 @@ State setState(final State newState) {
 
     public boolean isRunning() {
         synchronized (stateLock) {
-            return state.isAlive();
+            return isStateAlive();
         }
     }
 
+    // Ensure Mockito can stub method for KafkaStreamTest.
+    public boolean isStateAlive() {
+        return state.isAlive();
+    }
+

Review Comment:
   Why is this needed?



##########
streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java:
##########
@@ -990,7 +990,12 @@ private StreamThread createAndAddStreamThread(final long 
cacheSizePerThread, fin
         return streamThread;
     }
 
-    static Metrics getMetrics(final StreamsConfig config, final Time time, 
final String clientId) {
+    // Ensure Mockito stub construct with capture argument for 
KafkaStreamsTest.
+    public static Metrics createMetrics(final MetricConfig metricConfig, final 
List<MetricsReporter> reporters, final Time time, final MetricsContext 
metricsContext) {
+        return new Metrics(metricConfig, reporters, time, metricsContext);
+    }

Review Comment:
   You cannot add a public method call to a public class without a KIP. IMO, it 
is also not good to add methods just for tests to a class. It often points to 
code-smell.



##########
streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java:
##########
@@ -192,154 +209,141 @@ public void before() throws Exception {
         prepareStreams();
     }
 
+    @After
+    public void tearDown() {
+        kafkaStreamsMockedStatic.close();
+        clientMetricsMockedStatic.close();
+        streamThreadMockedStatic.close();
+        globalStreamThreadMockedConstruction.close();
+        if (stateDirectoryMockedConstruction != null)
+            stateDirectoryMockedConstruction.close();
+        streamsConfigUtils.close();
+    }
+
     private void prepareStreams() throws Exception {
         // setup metrics
-        PowerMock.expectNew(Metrics.class,
-            anyObject(MetricConfig.class),
-            capture(metricsReportersCapture),
-            anyObject(Time.class),
-            anyObject(MetricsContext.class)
-        ).andAnswer(() -> {
+        kafkaStreamsMockedStatic = mockStatic(KafkaStreams.class, 
withSettings()
+                .defaultAnswer(InvocationOnMock::callRealMethod));
+        kafkaStreamsMockedStatic.when(() -> KafkaStreams.createMetrics(
+                any(MetricConfig.class),
+                metricsReportersCapture.capture(),
+                any(Time.class),
+                any(MetricsContext.class)
+        )).thenAnswer(invocation -> {

Review Comment:
   You might be able to mock the `Metrics.class` with `mockConstruction()` (see 
https://javadoc.io/doc/org.mockito/mockito-core/latest/org/mockito/Mockito.html#mocked_construction).
 Some instructions can also be found at 
https://www.davidvlijmincx.com/posts/mockito_mock_constructor/.
   In such a way you would avoid the change of the public API in `KafkaStreams`.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to