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/11/15 09:26:17 UTC

[GitHub] [kafka] cadonna commented on a diff in pull request #12821: KAFKA-14132: Replace PowerMock and EasyMock with Mockito in streams tests

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


##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerUtilTest.java:
##########
@@ -42,77 +39,54 @@
 
 import static java.util.Collections.emptyList;
 import static java.util.Collections.singletonList;
-import static org.easymock.EasyMock.createStrictControl;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThrows;
-import static org.powermock.api.easymock.PowerMock.mockStatic;
-import static org.powermock.api.easymock.PowerMock.replayAll;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mockStatic;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(Utils.class)
+@RunWith(MockitoJUnitRunner.StrictStubs.class)
 public class StateManagerUtilTest {
 
-    @Mock(type = MockType.NICE)
+    @Mock
     private ProcessorStateManager stateManager;
 
-    @Mock(type = MockType.NICE)
+    @Mock
     private StateDirectory stateDirectory;
 
-    @Mock(type = MockType.NICE)
+    @Mock
     private ProcessorTopology topology;
 
-    @Mock(type = MockType.NICE)
+    @Mock
     private InternalProcessorContext processorContext;
 
-    private IMocksControl ctrl;
-
     private Logger logger = new LogContext("test").logger(AbstractTask.class);
 
     private final TaskId taskId = new TaskId(0, 0);
 
-    @Before
-    public void setup() {
-        ctrl = createStrictControl();
-        topology = ctrl.createMock(ProcessorTopology.class);
-        processorContext = ctrl.createMock(InternalProcessorContext.class);
-
-        stateManager = ctrl.createMock(ProcessorStateManager.class);
-        stateDirectory = ctrl.createMock(StateDirectory.class);
-    }
-
     @Test
     public void testRegisterStateStoreWhenTopologyEmpty() {
-        expect(topology.stateStores()).andReturn(emptyList());
-
-        ctrl.checkOrder(true);

Review Comment:
   The original author wanted to make sure some calls are done in a specific order. You should migrate the tests to Mockito and preserve the verification of the intended order. I see that some test do not really verify an order. For those tests, you do not need to migrate the order verifications. 



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerUtilTest.java:
##########
@@ -187,151 +139,92 @@ public void testCloseStateManagerThrowsExceptionWhenClean() {
         // Thrown stateMgr exception will not be wrapped.
         assertEquals("state manager failed to close", thrown.getMessage());
 
-        ctrl.verify();
+        // The unlock logic should still be executed.
+        verify(stateDirectory).unlock(taskId);
     }
 
     @Test
     public void testCloseStateManagerThrowsExceptionWhenDirty() {
-        expect(stateManager.taskId()).andReturn(taskId);
+        when(stateManager.taskId()).thenReturn(taskId);
 
-        expect(stateDirectory.lock(taskId)).andReturn(true);
+        when(stateDirectory.lock(taskId)).thenReturn(true);
 
-        stateManager.close();
-        expectLastCall().andThrow(new ProcessorStateException("state manager failed to close"));
-
-        stateDirectory.unlock(taskId);
-
-        ctrl.checkOrder(true);
-        ctrl.replay();
+        doThrow(new ProcessorStateException("state manager failed to close")).when(stateManager).close();
 
         assertThrows(
             ProcessorStateException.class,
             () -> StateManagerUtil.closeStateManager(
                 logger, "logPrefix:", false, false, stateManager, stateDirectory, TaskType.ACTIVE));
 
-        ctrl.verify();
+        verify(stateDirectory).unlock(taskId);
     }
 
     @Test
     public void testCloseStateManagerWithStateStoreWipeOut() {
-        expect(stateManager.taskId()).andReturn(taskId);
-        expect(stateDirectory.lock(taskId)).andReturn(true);
-
-        stateManager.close();
-        expectLastCall();
+        when(stateManager.taskId()).thenReturn(taskId);
+        when(stateDirectory.lock(taskId)).thenReturn(true);
 
         // The `baseDir` will be accessed when attempting to delete the state store.
-        expect(stateManager.baseDir()).andReturn(TestUtils.tempDirectory("state_store"));
-
-        stateDirectory.unlock(taskId);
-        expectLastCall();
-
-        ctrl.checkOrder(true);
-        ctrl.replay();
+        when(stateManager.baseDir()).thenReturn(TestUtils.tempDirectory("state_store"));
 
         StateManagerUtil.closeStateManager(logger,
             "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE);
 
-        ctrl.verify();
+        verify(stateManager).close();
+        verify(stateDirectory).unlock(taskId);
     }
 
     @Test
     public void  shouldStillWipeStateStoresIfCloseThrowsException() throws IOException {
         final File randomFile = new File("/random/path");
-        mockStatic(Utils.class);
-
-        expect(stateManager.taskId()).andReturn(taskId);
-        expect(stateDirectory.lock(taskId)).andReturn(true);
 
-        stateManager.close();
-        expectLastCall().andThrow(new ProcessorStateException("Close failed"));
+        when(stateManager.taskId()).thenReturn(taskId);
+        when(stateDirectory.lock(taskId)).thenReturn(true);
 
-        expect(stateManager.baseDir()).andReturn(randomFile);
+        doThrow(new ProcessorStateException("Close failed")).when(stateManager).close();
 
-        Utils.delete(randomFile);
+        when(stateManager.baseDir()).thenReturn(randomFile);
 
-        stateDirectory.unlock(taskId);
-        expectLastCall();
+        try (MockedStatic<Utils> utils = mockStatic(Utils.class)) {
+            assertThrows(ProcessorStateException.class, () ->
+                    StateManagerUtil.closeStateManager(logger, "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE));
+        }
 
-        ctrl.checkOrder(true);
-        ctrl.replay();
-
-        replayAll();
-
-        assertThrows(ProcessorStateException.class, () ->
-            StateManagerUtil.closeStateManager(logger, "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE));
-
-        ctrl.verify();
+        verify(stateDirectory).unlock(taskId);
     }
 
     @Test
     public void testCloseStateManagerWithStateStoreWipeOutRethrowWrappedIOException() throws IOException {
         final File unknownFile = new File("/unknown/path");
-        mockStatic(Utils.class);
-
-        expect(stateManager.taskId()).andReturn(taskId);
-        expect(stateDirectory.lock(taskId)).andReturn(true);
-
-        stateManager.close();
-        expectLastCall();
-
-        expect(stateManager.baseDir()).andReturn(unknownFile);
 
-        Utils.delete(unknownFile);
-        expectLastCall().andThrow(new IOException("Deletion failed"));
+        when(stateManager.taskId()).thenReturn(taskId);
+        when(stateDirectory.lock(taskId)).thenReturn(true);
+        when(stateManager.baseDir()).thenReturn(unknownFile);
 
-        stateDirectory.unlock(taskId);
-        expectLastCall();
+        try (MockedStatic<Utils> utils = mockStatic(Utils.class)) {
+            utils.when(() -> Utils.delete(unknownFile)).thenThrow(new IOException("Deletion failed"));
 
-        ctrl.checkOrder(true);
-        ctrl.replay();
+            final ProcessorStateException thrown = assertThrows(
+                    ProcessorStateException.class, () -> StateManagerUtil.closeStateManager(logger,
+                            "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE));
 
-        replayAll();
+            assertEquals(IOException.class, thrown.getCause().getClass());
+        }
 
-        final ProcessorStateException thrown = assertThrows(
-            ProcessorStateException.class, () -> StateManagerUtil.closeStateManager(logger,
-                "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE));
-
-        assertEquals(IOException.class, thrown.getCause().getClass());
-
-        ctrl.verify();
+        verify(stateManager).close();
+        verify(stateDirectory).unlock(taskId);
     }
 
     @Test
-    public void shouldNotCloseStateManagerIfUnableToLockTaskDirectory() {
-        expect(stateManager.taskId()).andReturn(taskId);
-
-        expect(stateDirectory.lock(taskId)).andReturn(false);
-
-        stateManager.close();
-        expectLastCall().andThrow(new AssertionError("Should not be trying to close state you don't own!"));
-
-        ctrl.checkOrder(true);
-        ctrl.replay();
-
-        replayAll();
+    public void shouldNotWipeStateStoresIfUnableToLockTaskDirectory() {
+        when(stateManager.taskId()).thenReturn(taskId);
+        when(stateDirectory.lock(taskId)).thenReturn(false);
 
         StateManagerUtil.closeStateManager(
-            logger, "logPrefix:", true, false, stateManager, stateDirectory, TaskType.ACTIVE);
-    }
-
-    @Test
-    public void shouldNotWipeStateStoresIfUnableToLockTaskDirectory() throws IOException {

Review Comment:
   As far as I understand the test, the author wanted to fail the test with an `AssertionError` if `Utils.delete` is called after the lock could not be acquired. With Mockito, you can verify that `Utils.delete` is never called.



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