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 2022/09/09 07:58:07 UTC

[GitHub] [kafka] cadonna commented on a diff in pull request #12505: KAFKA-14133: Replace EasyMock with Mockito in streams tests

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


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java:
##########
@@ -258,58 +266,45 @@ public void shouldPutBackwardFetchAllKeysFromCache() {
     @Test
     public void shouldCloseWrappedStoreAndCacheAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on flush"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+        doThrow(new RuntimeException("Simulating an error on flush")).doNothing().when(cache).flush(anyString());
 
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
+        verify(cache).flush(eq(CACHE_NAMESPACE));
+        verify(cache).close(eq(CACHE_NAMESPACE));
+        verify(underlyingStore).close();
     }
 
     @Test
     public void shouldCloseWrappedStoreAfterErrorDuringCacheClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        cache.close(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on close"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+

Review Comment:
   nit: Please remove this line here and in other parts of this PR. Stubbing is part of the setup IMO.  



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java:
##########
@@ -105,26 +112,20 @@ protected <K, V> KeyValueStore<K, V> createKeyValueStore(final StateStoreContext
     @SuppressWarnings("deprecation")
     @Test
     public void shouldDelegateDeprecatedInit() {
-        final KeyValueStore<Bytes, byte[]> inner = EasyMock.mock(InMemoryKeyValueStore.class);
+        final KeyValueStore<Bytes, byte[]> inner = mock(InMemoryKeyValueStore.class);
         final CachingKeyValueStore outer = new CachingKeyValueStore(inner, false);
-        EasyMock.expect(inner.name()).andStubReturn("store");
-        inner.init((ProcessorContext) context, outer);
-        EasyMock.expectLastCall();
-        EasyMock.replay(inner);
+        when(inner.name()).thenReturn("store");
         outer.init((ProcessorContext) context, outer);
-        EasyMock.verify(inner);
+        verify(inner).init(eq((ProcessorContext) context), eq(outer));
     }
 
     @Test
     public void shouldDelegateInit() {
-        final KeyValueStore<Bytes, byte[]> inner = EasyMock.mock(InMemoryKeyValueStore.class);
+        final KeyValueStore<Bytes, byte[]> inner = mock(InMemoryKeyValueStore.class);
         final CachingKeyValueStore outer = new CachingKeyValueStore(inner, false);
-        EasyMock.expect(inner.name()).andStubReturn("store");
-        inner.init((StateStoreContext) context, outer);
-        EasyMock.expectLastCall();
-        EasyMock.replay(inner);
+        when(inner.name()).thenReturn("store");
         outer.init((StateStoreContext) context, outer);
-        EasyMock.verify(inner);
+        verify(inner).init(eq((StateStoreContext) context), eq(outer));

Review Comment:
   Out of curiosity: Why do you need `eq()` here?
   This comment applies also to other parts of the PR.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java:
##########
@@ -119,29 +125,24 @@ public void closeStore() {
         cachingStore.close();
     }
 
-    @SuppressWarnings("deprecation")
+    @SuppressWarnings({"deprecation", "unchecked"})
     @Test
     public void shouldDelegateDeprecatedInit() {
-        final WindowStore<Bytes, byte[]> inner = EasyMock.mock(WindowStore.class);
+        final WindowStore<Bytes, byte[]> inner = mock(WindowStore.class);

Review Comment:
   You could move the mock specification to the object fields and use `@Mock` to avoid suppression. Same is true for other places in this PR where suppression is applied.



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListenerTest.java:
##########
@@ -22,164 +22,143 @@
 import org.apache.kafka.streams.errors.TaskAssignmentException;
 import org.apache.kafka.streams.processor.internals.StreamThread.State;
 import org.apache.kafka.streams.processor.internals.assignment.AssignorError;
-import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
 import org.slf4j.LoggerFactory;
 
 import java.util.Collection;
 import java.util.Collections;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.mock;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
+@RunWith(MockitoJUnitRunner.StrictStubs.class)
 public class StreamsRebalanceListenerTest {
 
-    private final TaskManager taskManager = mock(TaskManager.class);
-    private final StreamThread streamThread = mock(StreamThread.class);
+    @Mock
+    private TaskManager taskManager;
+    @Mock
+    private StreamThread streamThread;
     private final AtomicInteger assignmentErrorCode = new AtomicInteger();
     private final MockTime time = new MockTime();
-    private final StreamsRebalanceListener streamsRebalanceListener = new StreamsRebalanceListener(
-        time,
-        taskManager,
-        streamThread,
-        LoggerFactory.getLogger(StreamsRebalanceListenerTest.class),
-        assignmentErrorCode
-    );
+    private StreamsRebalanceListener streamsRebalanceListener;
 
     @Before
-    public void before() {
-        expect(streamThread.state()).andStubReturn(null);
-        expect(taskManager.activeTaskIds()).andStubReturn(null);
-        expect(taskManager.standbyTaskIds()).andStubReturn(null);
+    public void setup() {
+        streamsRebalanceListener = new StreamsRebalanceListener(time,
+                taskManager,
+                streamThread,
+                LoggerFactory.getLogger(StreamsRebalanceListenerTest.class),
+                assignmentErrorCode
+        );

Review Comment:
   Is there a specific reason to move the creation of the listener to the setup method? I think it is fine to leave the creation at the field.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java:
##########
@@ -153,57 +154,48 @@ public void shouldAvoidFlushingDeletionsWithoutDirtyKeys() {
     @Test
     public void shouldCloseWrappedStoreAndCacheAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on flush"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+
+        doThrow(new RuntimeException("Simulating an error on flush")).when(cache).flush(CACHE_NAMESPACE);
 
         assertThrows(RuntimeException.class, store::close);
-        EasyMock.verify(cache, underlyingStore);
+
+        verify(cache).flush(eq(CACHE_NAMESPACE));
+        verify(cache).close(eq(CACHE_NAMESPACE));
+        verify(underlyingStore).close();
     }
 
     @Test
     public void shouldCloseWrappedStoreAfterErrorDuringCacheClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        cache.close(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on close"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+
+        doThrow(new RuntimeException("Simulating an error on close")).when(cache).close(CACHE_NAMESPACE);
 
         assertThrows(RuntimeException.class, store::close);
-        EasyMock.verify(cache, underlyingStore);
+
+        verify(cache).flush(eq(CACHE_NAMESPACE));
+        verify(cache).close(eq(CACHE_NAMESPACE));

Review Comment:
   Same question as above about verifying a stubbing.
   I saw the same situation also in other parts of this PR. 



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java:
##########
@@ -153,57 +154,48 @@ public void shouldAvoidFlushingDeletionsWithoutDirtyKeys() {
     @Test
     public void shouldCloseWrappedStoreAndCacheAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on flush"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+
+        doThrow(new RuntimeException("Simulating an error on flush")).when(cache).flush(CACHE_NAMESPACE);
 
         assertThrows(RuntimeException.class, store::close);
-        EasyMock.verify(cache, underlyingStore);
+
+        verify(cache).flush(eq(CACHE_NAMESPACE));

Review Comment:
   Do we need to verify this since it is already stubbed and we use `StrictStubbed`?



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingInMemoryKeyValueStoreTest.java:
##########
@@ -153,57 +154,48 @@ public void shouldAvoidFlushingDeletionsWithoutDirtyKeys() {
     @Test
     public void shouldCloseWrappedStoreAndCacheAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on flush"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+
+        doThrow(new RuntimeException("Simulating an error on flush")).when(cache).flush(CACHE_NAMESPACE);
 
         assertThrows(RuntimeException.class, store::close);
-        EasyMock.verify(cache, underlyingStore);
+
+        verify(cache).flush(eq(CACHE_NAMESPACE));
+        verify(cache).close(eq(CACHE_NAMESPACE));

Review Comment:
   Again, why do we need `eq()` here?



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java:
##########
@@ -1039,58 +1041,42 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() {
     @Test
     public void shouldCloseCacheAndWrappedStoreAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an error on flush"));
-        cache.close(CACHE_NAMESPACE);
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.replay(underlyingStore);
+        doThrow(new RuntimeException("Simulating an error on flush")).doNothing().when(cache).flush(CACHE_NAMESPACE);
 
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
+        verify(cache).flush(CACHE_NAMESPACE);
+        verify(cache).close(CACHE_NAMESPACE);
+        verify(underlyingStore).close();

Review Comment:
   I think it makes sense to use `InOrder` here since we want to flush a cache/store before closing it.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueStoreBuilderTest.java:
##########
@@ -126,27 +121,15 @@ public void shouldThrowNullPointerIfInnerIsNull() {
             Serdes.String(), new MockTime()));
     }
 
-    @Test

Review Comment:
   That is unfortunate! Yes, we should add null checks! 
   Lesson learned:
   If you verify an exception always verify the exception message as well. 🙂 



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

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