Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-14132: Replace PowerMock and EasyMock with Mockito in streams tests #12821

Merged
merged 6 commits into from Jan 19, 2023

Conversation

clolov
Copy link
Collaborator

@clolov clolov commented Nov 4, 2022

Batch 1 of the tests detailed in https://issues.apache.org/jira/browse/KAFKA-14132 which use PowerMock/EasyMock and need to be moved to Mockito.

@clolov clolov changed the title Replace PowerMock and EasyMock with Mockito in streams tests KAFKA-14132: Replace PowerMock and EasyMock with Mockito in streams tests Nov 4, 2022
}

@Test
public void shouldNotWipeStateStoresIfUnableToLockTaskDirectory() throws IOException {
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test appears to be just wrong, unless I am reading the code incorrectly. Because we use expect(stateDirectory.lock(taskId)).andReturn(false); we never reach the code path which would be using Utils.delete. As such, I just removed the test.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I do not understand. I think this test and the one above can be collapsed in the test I have written. I still verify that deletion did not happen by checking that a few methods were never interacted with.

Copy link
Collaborator Author

@clolov clolov Nov 16, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As in, it appears a bit strange to mock everything on a path to the Utils.delete when this logic short-circuits if we use expect(stateDirectory.lock(taskId)).andReturn(false);.

        try {
            if (stateDirectory.lock(id)) { <---- WE JUMP FROM HERE...
                try {
                    stateMgr.close(); <---- WE VERIFY THIS IS NOT CALLED
                } catch (final ProcessorStateException e) {
                    firstException.compareAndSet(null, e);
                } finally {
                    try {
                        if (wipeStateStore) {
                            log.debug("Wiping state stores for {} task {}", taskType, id);
                            // we can just delete the whole dir of the task, including the state store images and the checkpoint files,
                            // and then we write an empty checkpoint file indicating that the previous close is graceful and we just
                            // need to re-bootstrap the restoration from the beginning
                            Utils.delete(stateMgr.baseDir() <---- WE VERIFY THIS IS NOT CALLED);
                        }
                    } finally {
                        stateDirectory.unlock(id); <---- WE VERIFY THIS IS NOT CALLED
                    }
                }
            }
        } catch (final IOException e) {
            final ProcessorStateException exception = new ProcessorStateException(
                String.format("%sFatal error while trying to close the state manager for task %s", logPrefix, id), e
            );
            firstException.compareAndSet(null, exception);
        } <---- ...ALL THE WAY TO HERE

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see what you mean, but unit tests should also document the behavior of production code and be robust against code changes. In this case, we have indeed to separate cases:

  1. When exactly-once is enabled and the state manager is closed dirty and the lock is not owned then the state manager should not be closed, the directory should not be wiped and the directory should not be unlocked.
  2. When exactly-once is disabled and the lock is not owned then the state manager should not be closed and the directory should not be unlocked.

So, I think what you did is fine, although would not verify that stateManager.baseDir() is never called but that Utils.delete() is never called. However, you need to run that test once with:

StateManagerUtil.closeStateManager(
            logger, "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE);

and once with

StateManagerUtil.closeStateManager(
            logger, "logPrefix:", true, false, stateManager, stateDirectory, TaskType.ACTIVE);

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay, I agree, I will create a new commit testing both scenarios.

@clolov
Copy link
Collaborator Author

clolov commented Nov 4, 2022

Hey @cadonna 👋! Adding you for visibility as these are Streams-related tests.

Copy link
Contributor

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clolov Thanks for the PR!

Here my feedback!

}

@Test
public void shouldNotWipeStateStoresIfUnableToLockTaskDirectory() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

@clolov
Copy link
Collaborator Author

clolov commented Nov 15, 2022

Thank you for the review @cadonna. I will aim to address it today!

@divijvaidya
Copy link
Contributor

@clolov please fix the checkstyle failures in this PR!

Copy link
Contributor

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clolov Thanks for the updates and sorry for the long silence on this PR!

I had one major comment and some nits.

}

@Test
public void shouldNotWipeStateStoresIfUnableToLockTaskDirectory() throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see what you mean, but unit tests should also document the behavior of production code and be robust against code changes. In this case, we have indeed to separate cases:

  1. When exactly-once is enabled and the state manager is closed dirty and the lock is not owned then the state manager should not be closed, the directory should not be wiped and the directory should not be unlocked.
  2. When exactly-once is disabled and the lock is not owned then the state manager should not be closed and the directory should not be unlocked.

So, I think what you did is fine, although would not verify that stateManager.baseDir() is never called but that Utils.delete() is never called. However, you need to run that test once with:

StateManagerUtil.closeStateManager(
            logger, "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE);

and once with

StateManagerUtil.closeStateManager(
            logger, "logPrefix:", true, false, stateManager, stateDirectory, TaskType.ACTIVE);

@clolov
Copy link
Collaborator Author

clolov commented Dec 14, 2022

Hello @cadonna and thanks for the review! I will aim to address the comments tomorrow.

@mjsax mjsax added the tests Test fixes (including flaky tests) label Dec 28, 2022
Copy link
Contributor

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clolov Thanks for the update and as always for your patience!

LGTM!

I have just minor comment! I will leave it up to you, if you want to change it. Just let me know so that I can merge.

Comment on lines 223 to 225
inOrder.verify(stateManager, never()).close();
inOrder.verify(stateManager, never()).baseDir();
inOrder.verify(stateDirectory, never()).unlock(taskId);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do not know how much it makes sense to verify the call order of methods that should not be called.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a fair point. I used inOrder because the original test wanted to be strict. This being said it wanted to be strict on the actual calls rather than the calls which weren't made. Does

    @Test
    public void shouldNotCloseStateManagerIfUnableToLockTaskDirectory() {
        final InOrder inOrder = inOrder(stateManager, stateDirectory);
        when(stateManager.taskId()).thenReturn(taskId);
        when(stateDirectory.lock(taskId)).thenReturn(false);

        StateManagerUtil.closeStateManager(
                logger, "logPrefix:", true, false, stateManager, stateDirectory, TaskType.ACTIVE);

        inOrder.verify(stateManager).taskId(); <- ENSURE ORDER
        inOrder.verify(stateDirectory).lock(taskId); <- ENSURE ORDER
        verify(stateManager, never()).close();
        verify(stateManager, never()).baseDir();
        verify(stateDirectory, never()).unlock(taskId);
        verifyNoMoreInteractions(stateManager, stateDirectory);
    }

    @Test
    public void shouldNotWipeStateStoresIfUnableToLockTaskDirectory() {
        final InOrder inOrder = inOrder(stateManager, stateDirectory);
        when(stateManager.taskId()).thenReturn(taskId);
        when(stateDirectory.lock(taskId)).thenReturn(false);

        StateManagerUtil.closeStateManager(
                logger, "logPrefix:", false, true, stateManager, stateDirectory, TaskType.ACTIVE);

        inOrder.verify(stateManager).taskId(); <- ENSURE ORDER
        inOrder.verify(stateDirectory).lock(taskId); <- ENSURE ORDER
        verify(stateManager, never()).close(); 
        verify(stateManager, never()).baseDir();
        verify(stateDirectory, never()).unlock(taskId);
        verifyNoMoreInteractions(stateManager, stateDirectory);
    }

look better to you?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks great!

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay, I pushed the latest rebased version!

Comment on lines 238 to 240
inOrder.verify(stateManager, never()).close();
inOrder.verify(stateManager, never()).baseDir();
inOrder.verify(stateDirectory, never()).unlock(taskId);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

See my comment above about call order.

@cadonna
Copy link
Contributor

cadonna commented Jan 19, 2023

Build failures are unrelated:

Build / JDK 11 and Scala 2.13 / org.apache.kafka.clients.consumer.KafkaConsumerTest.testReturnRecordsDuringRebalance()
Build / JDK 11 and Scala 2.13 / org.apache.kafka.connect.integration.BlockingConnectorTest.testBlockInSinkTaskStart
Build / JDK 11 and Scala 2.13 / org.apache.kafka.connect.integration.ConnectWorkerIntegrationTest.testBrokerCoordinator
Build / JDK 17 and Scala 2.13 / org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()
Build / JDK 17 and Scala 2.13 / org.apache.kafka.trogdor.coordinator.CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated()

@cadonna cadonna merged commit e235e1a into apache:trunk Jan 19, 2023
@clolov
Copy link
Collaborator Author

clolov commented Jan 20, 2023

Thank you very much for the review and merge!

guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
…ests (apache#12821)

Batch 1 of the tests detailed in https://issues.apache.org/jira/browse/KAFKA-14132 which use PowerMock/EasyMock and need to be moved to Mockito.

Reviewer: Bruno Cadonna <cadonna@apache.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
streams tests Test fixes (including flaky tests)
Projects
None yet
4 participants