You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/06/09 09:42:26 UTC

[GitHub] [kafka] wycccccc opened a new pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

wycccccc opened a new pull request #10850:
URL: https://github.com/apache/kafka/pull/10850


   Development of EasyMock and PowerMock has stagnated while Mockito continues to be actively developed. With the new Java cadence, it's a problem to depend on libraries that do bytecode generation and are not actively maintained. In addition, Mockito is also easier to [use.KAFKA-7438](https://issues.apache.org/jira/browse/KAFKA-7438)
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

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



[GitHub] [kafka] vvcephei commented on pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#issuecomment-859023848


   Oh, right, I meant to say that the core integration tests are broken right now. I've just run the Streams tests on my laptop, and we also have passing tests for the ARM build.


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

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



[GitHub] [kafka] vvcephei commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r649499312



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -135,45 +124,38 @@ public void shouldGetFailedStreamThreadsSensor() {
             false,
             description
         );
-        replay(StreamsMetricsImpl.class, streamsMetrics);
 
         final Sensor sensor = ClientMetrics.failedStreamThreadSensor(streamsMetrics);
-
-        verify(StreamsMetricsImpl.class, streamsMetrics);

Review comment:
       I see. I thought it also verified that the mocked methods actually got used.
   
   Looking at the test closer, though, I think that additional verification isn't really needed here.




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

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



[GitHub] [kafka] wycccccc commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
wycccccc commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648532511



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetricsTest.java
##########
@@ -18,51 +18,38 @@
 
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version;
-import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
 import java.util.Collections;
 import java.util.Map;
 
 import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.TASK_LEVEL_GROUP;
-import static org.easymock.EasyMock.expect;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replay;
-import static org.powermock.api.easymock.PowerMock.verify;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamsMetricsImpl.class, Sensor.class, ThreadMetrics.class, StateStoreMetrics.class, ProcessorNodeMetrics.class})
+@RunWith(MockitoJUnitRunner.class)
 public class TaskMetricsTest {
 
     private final static String THREAD_ID = "test-thread";
     private final static String TASK_ID = "test-task";
 
-    private final StreamsMetricsImpl streamsMetrics = createMock(StreamsMetricsImpl.class);
-    private final Sensor expectedSensor = createMock(Sensor.class);
+    private final StreamsMetricsImpl streamsMetrics = Mockito.mock(StreamsMetricsImpl.class);
+    private final Sensor expectedSensor = Mockito.mock(Sensor.class);
     private final Map<String, String> tagMap = Collections.singletonMap("hello", "world");
 
-    @Before
-    public void setUp() {
-        expect(streamsMetrics.version()).andStubReturn(Version.LATEST);
-        mockStatic(StreamsMetricsImpl.class);

Review comment:
       In Mockito we can direct definition static method,so these are no longer 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.

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



[GitHub] [kafka] vvcephei commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648363737



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ProcessorNodeMetricsTest.java
##########
@@ -18,87 +18,73 @@
 
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
+
 
 import java.util.Collections;
 import java.util.Map;
 import java.util.function.Supplier;
 
 import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_LEVEL_GROUP;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replay;
-import static org.powermock.api.easymock.PowerMock.verify;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamsMetricsImpl.class, Sensor.class})
+@RunWith(MockitoJUnitRunner.class)
 public class ProcessorNodeMetricsTest {
 
     private static final String THREAD_ID = "test-thread";
     private static final String TASK_ID = "test-task";
     private static final String PROCESSOR_NODE_ID = "test-processor";
 
-    private final Sensor expectedSensor = mock(Sensor.class);
-    private final Sensor expectedParentSensor = mock(Sensor.class);
-    private final StreamsMetricsImpl streamsMetrics = createMock(StreamsMetricsImpl.class);
     private final Map<String, String> tagMap = Collections.singletonMap("hello", "world");
     private final Map<String, String> parentTagMap = Collections.singletonMap("hi", "universe");
 
-    @Before
-    public void setUp() {
-        expect(streamsMetrics.version()).andStubReturn(Version.LATEST);
-        mockStatic(StreamsMetricsImpl.class);
-    }
+    private final Sensor expectedSensor = Mockito.mock(Sensor.class);

Review comment:
       In general, it's good to try and avoid shuffling lines around and reformatting the code in functional PRs. It just makes it harder to ensure a good review.

##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -135,45 +124,38 @@ public void shouldGetFailedStreamThreadsSensor() {
             false,
             description
         );
-        replay(StreamsMetricsImpl.class, streamsMetrics);
 
         final Sensor sensor = ClientMetrics.failedStreamThreadSensor(streamsMetrics);
-
-        verify(StreamsMetricsImpl.class, streamsMetrics);

Review comment:
       Did we lose a verification here?

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ProcessorNodeMetricsTest.java
##########
@@ -208,11 +194,7 @@ private void setUpThroughputSensor(final String metricNamePrefix,
     }
 
     private void verifySensor(final Supplier<Sensor> sensorSupplier) {
-        replay(StreamsMetricsImpl.class, streamsMetrics);
-
         final Sensor sensor = sensorSupplier.get();
-
-        verify(StreamsMetricsImpl.class, streamsMetrics);

Review comment:
       Should we continue to verify the interactions?

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/TaskMetricsTest.java
##########
@@ -18,51 +18,38 @@
 
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version;
-import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
 import java.util.Collections;
 import java.util.Map;
 
 import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.TASK_LEVEL_GROUP;
-import static org.easymock.EasyMock.expect;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replay;
-import static org.powermock.api.easymock.PowerMock.verify;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamsMetricsImpl.class, Sensor.class, ThreadMetrics.class, StateStoreMetrics.class, ProcessorNodeMetrics.class})
+@RunWith(MockitoJUnitRunner.class)
 public class TaskMetricsTest {
 
     private final static String THREAD_ID = "test-thread";
     private final static String TASK_ID = "test-task";
 
-    private final StreamsMetricsImpl streamsMetrics = createMock(StreamsMetricsImpl.class);
-    private final Sensor expectedSensor = createMock(Sensor.class);
+    private final StreamsMetricsImpl streamsMetrics = Mockito.mock(StreamsMetricsImpl.class);
+    private final Sensor expectedSensor = Mockito.mock(Sensor.class);
     private final Map<String, String> tagMap = Collections.singletonMap("hello", "world");
 
-    @Before
-    public void setUp() {
-        expect(streamsMetrics.version()).andStubReturn(Version.LATEST);
-        mockStatic(StreamsMetricsImpl.class);

Review comment:
       I'm not seeing where these moved to. How do the tests still work without them?




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

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



[GitHub] [kafka] ijuma commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648315914



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -21,39 +21,28 @@
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
 import org.apache.kafka.streams.KafkaStreams.State;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
 import java.util.Collections;
 import java.util.Map;
 
 import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.CLIENT_LEVEL_GROUP;
 import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replay;
-import static org.powermock.api.easymock.PowerMock.verify;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamsMetricsImpl.class, Sensor.class, ClientMetrics.class})
+@RunWith(MockitoJUnitRunner.class)

Review comment:
       Why do we need the runner? In other modules, we use mockito without the runner. The reason why we want to avoid this runner is that it's a JUnit 4 thing and we want to migrate to JUnit 5 soonish.




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

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



[GitHub] [kafka] ijuma commented on pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#issuecomment-859697426






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

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



[GitHub] [kafka] wycccccc commented on pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
wycccccc commented on pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#issuecomment-859328616






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

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



[GitHub] [kafka] ijuma commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648316500



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -125,8 +114,8 @@ public void shouldAddAliveStreamThreadsMetric() {
     public void shouldGetFailedStreamThreadsSensor() {
         final String name = "failed-stream-threads";
         final String description = "The number of failed stream threads since the start of the Kafka Streams client";
-        expect(streamsMetrics.clientLevelSensor(name, RecordingLevel.INFO)).andReturn(expectedSensor);
-        expect(streamsMetrics.clientLevelTagMap()).andReturn(tagMap);
+        Mockito.when(streamsMetrics.clientLevelSensor(name, RecordingLevel.INFO)).thenReturn(expectedSensor);
+        Mockito.when(streamsMetrics.clientLevelTagMap()).thenReturn(tagMap);

Review comment:
       From a style perspective, you can import `when` via a static import to make it more readable.




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

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



[GitHub] [kafka] wycccccc commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
wycccccc commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r649082247



##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ProcessorNodeMetricsTest.java
##########
@@ -18,87 +18,73 @@
 
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
+
 
 import java.util.Collections;
 import java.util.Map;
 import java.util.function.Supplier;
 
 import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_LEVEL_GROUP;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.mock;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replay;
-import static org.powermock.api.easymock.PowerMock.verify;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamsMetricsImpl.class, Sensor.class})
+@RunWith(MockitoJUnitRunner.class)
 public class ProcessorNodeMetricsTest {
 
     private static final String THREAD_ID = "test-thread";
     private static final String TASK_ID = "test-task";
     private static final String PROCESSOR_NODE_ID = "test-processor";
 
-    private final Sensor expectedSensor = mock(Sensor.class);
-    private final Sensor expectedParentSensor = mock(Sensor.class);
-    private final StreamsMetricsImpl streamsMetrics = createMock(StreamsMetricsImpl.class);
     private final Map<String, String> tagMap = Collections.singletonMap("hello", "world");
     private final Map<String, String> parentTagMap = Collections.singletonMap("hi", "universe");
 
-    @Before
-    public void setUp() {
-        expect(streamsMetrics.version()).andStubReturn(Version.LATEST);
-        mockStatic(StreamsMetricsImpl.class);
-    }
+    private final Sensor expectedSensor = Mockito.mock(Sensor.class);

Review comment:
       Thanks for the reminder, I will pay attention it in next pr.




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

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



[GitHub] [kafka] wycccccc commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
wycccccc commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648533278



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -21,39 +21,28 @@
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
 import org.apache.kafka.streams.KafkaStreams.State;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
 import java.util.Collections;
 import java.util.Map;
 
 import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.CLIENT_LEVEL_GROUP;
 import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.powermock.api.easymock.PowerMock.createMock;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replay;
-import static org.powermock.api.easymock.PowerMock.verify;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamsMetricsImpl.class, Sensor.class, ClientMetrics.class})
+@RunWith(MockitoJUnitRunner.class)

Review comment:
       I did not find this detail, thanks for the reminder. It has been modified.




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

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



[GitHub] [kafka] vvcephei commented on pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#issuecomment-859022980


   Thanks for the updates, @wycccccc ! Merged.


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

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



[GitHub] [kafka] wycccccc commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
wycccccc commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648535313



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -125,8 +114,8 @@ public void shouldAddAliveStreamThreadsMetric() {
     public void shouldGetFailedStreamThreadsSensor() {
         final String name = "failed-stream-threads";
         final String description = "The number of failed stream threads since the start of the Kafka Streams client";
-        expect(streamsMetrics.clientLevelSensor(name, RecordingLevel.INFO)).andReturn(expectedSensor);
-        expect(streamsMetrics.clientLevelTagMap()).andReturn(tagMap);
+        Mockito.when(streamsMetrics.clientLevelSensor(name, RecordingLevel.INFO)).thenReturn(expectedSensor);
+        Mockito.when(streamsMetrics.clientLevelTagMap()).thenReturn(tagMap);

Review comment:
       In the case of mockito only,I also think the static import is better. Already changed.




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

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



[GitHub] [kafka] vvcephei merged pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams (metrics)

Posted by GitBox <gi...@apache.org>.
vvcephei merged pull request #10850:
URL: https://github.com/apache/kafka/pull/10850


   


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

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



[GitHub] [kafka] wycccccc commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
wycccccc commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648531262



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -135,45 +124,38 @@ public void shouldGetFailedStreamThreadsSensor() {
             false,
             description
         );
-        replay(StreamsMetricsImpl.class, streamsMetrics);
 
         final Sensor sensor = ClientMetrics.failedStreamThreadSensor(streamsMetrics);
-
-        verify(StreamsMetricsImpl.class, streamsMetrics);

Review comment:
       I think t`PowerMock.verify()` is used to verify whether` PowerMock.replay()` has been executed.
   `replay()`method is not needed in mockito, so there is no need to verify it.




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

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



[GitHub] [kafka] ijuma commented on a change in pull request #10850: KAFKA-12924 Replace EasyMock and PowerMock with Mockito in streams mo…

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10850:
URL: https://github.com/apache/kafka/pull/10850#discussion_r648316500



##########
File path: streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java
##########
@@ -125,8 +114,8 @@ public void shouldAddAliveStreamThreadsMetric() {
     public void shouldGetFailedStreamThreadsSensor() {
         final String name = "failed-stream-threads";
         final String description = "The number of failed stream threads since the start of the Kafka Streams client";
-        expect(streamsMetrics.clientLevelSensor(name, RecordingLevel.INFO)).andReturn(expectedSensor);
-        expect(streamsMetrics.clientLevelTagMap()).andReturn(tagMap);
+        Mockito.when(streamsMetrics.clientLevelSensor(name, RecordingLevel.INFO)).thenReturn(expectedSensor);
+        Mockito.when(streamsMetrics.clientLevelTagMap()).thenReturn(tagMap);

Review comment:
       From a style perspective, you can import `when` via a static import to make it more readable. Similarly for `eq` and so on.




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

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