divijvaidya commented on code in PR #12735:
URL: https://github.com/apache/kafka/pull/12735#discussion_r993621273
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -224,85 +231,68 @@ public void tearDown() {
if (metrics != null) {
metrics.stop();
}
+ mockitoSession.finishMocking();
+
}
@Test
public void testSinkTasksCloseErrorReporters() throws Exception {
- ErrorReporter reporter = EasyMock.mock(ErrorReporter.class);
+ ErrorReporter reporter = mock(ErrorReporter.class);
RetryWithToleranceOperator retryWithToleranceOperator = operator();
retryWithToleranceOperator.reporters(singletonList(reporter));
createSinkTask(initialState, retryWithToleranceOperator);
-
- expectInitializeTask();
- reporter.close();
- EasyMock.expectLastCall();
- sinkTask.stop();
- EasyMock.expectLastCall();
-
- consumer.close();
- EasyMock.expectLastCall();
-
- headerConverter.close();
- EasyMock.expectLastCall();
-
- PowerMock.replayAll();
-
+ doNothing().when(consumer).subscribe(
+ eq(singletonList(TOPIC)),
+ rebalanceListener.capture());
+ doNothing().when(sinkTask).initialize(sinkTaskContext.capture());
Review Comment:
Where are we using the argument that was captured here? Usually, when we
capture an argument, we have a place in the code after capturing where we do
`sinkTaskContext.getValue()` to use the captured value.
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -139,7 +139,9 @@ public class ErrorHandlingTaskTest {
@SuppressWarnings("unused")
@Mock
private SourceTask sourceTask;
- private Capture<WorkerSinkTaskContext> sinkTaskContext =
EasyMock.newCapture();
+
+ private ArgumentCaptor<WorkerSinkTaskContext> sinkTaskContext =
Review Comment:
You can use @Captor here since you are already using annotations such as
`@mock`
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -165,7 +167,8 @@ public class ErrorHandlingTaskTest {
@Mock
private ConnectorOffsetBackingStore offsetStore;
- private Capture<ConsumerRebalanceListener> rebalanceListener =
EasyMock.newCapture();
+ private ArgumentCaptor<ConsumerRebalanceListener> rebalanceListener =
Review Comment:
same comment as earlier about `@captor`
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -494,72 +502,29 @@ private void assertErrorHandlingMetricValue(String name,
double expected) {
assertEquals(expected, measured, 0.001d);
}
- private void expectInitializeTask() {
- consumer.subscribe(EasyMock.eq(singletonList(TOPIC)),
EasyMock.capture(rebalanceListener));
- PowerMock.expectLastCall();
-
- sinkTask.initialize(EasyMock.capture(sinkTaskContext));
- PowerMock.expectLastCall();
- sinkTask.start(TASK_PROPS);
- PowerMock.expectLastCall();
- }
-
- private void expectTaskGetTopic(boolean anyTimes) {
- final Capture<String> connectorCapture = EasyMock.newCapture();
- final Capture<String> topicCapture = EasyMock.newCapture();
- IExpectationSetters<TopicStatus> expect =
EasyMock.expect(statusBackingStore.getTopic(
- EasyMock.capture(connectorCapture),
- EasyMock.capture(topicCapture)));
- if (anyTimes) {
- expect.andStubAnswer(() -> new TopicStatus(
- topicCapture.getValue(),
- new ConnectorTaskId(connectorCapture.getValue(), 0),
- Time.SYSTEM.milliseconds()));
- } else {
- expect.andAnswer(() -> new TopicStatus(
- topicCapture.getValue(),
- new ConnectorTaskId(connectorCapture.getValue(), 0),
- Time.SYSTEM.milliseconds()));
- }
- if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) {
- assertEquals("job", connectorCapture.getValue());
- assertEquals(TOPIC, topicCapture.getValue());
- }
- }
-
- private void expectClose() {
- producer.close(EasyMock.anyObject(Duration.class));
- EasyMock.expectLastCall();
-
- admin.close(EasyMock.anyObject(Duration.class));
- EasyMock.expectLastCall();
+ private void expectClose() throws IOException {
- offsetReader.close();
- EasyMock.expectLastCall();
-
- offsetStore.stop();
- EasyMock.expectLastCall();
-
- try {
- headerConverter.close();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- EasyMock.expectLastCall();
+ verify(producer).close(any(Duration.class));
+ verify(admin).close(any(Duration.class));
+ verify(offsetReader).close();
+ verify(offsetStore).stop();
+ // headerConverter.close() can throw IOException
+ verify(headerConverter).close();
}
private void expectTopicCreation(String topic) {
if (workerConfig.topicCreationEnable()) {
-
EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap());
- Capture<NewTopic> newTopicCapture = EasyMock.newCapture();
+
when(admin.describeTopics(topic)).thenReturn(Collections.emptyMap());
+ ArgumentCaptor<NewTopic> newTopicCapture =
+ ArgumentCaptor.forClass(NewTopic.class);
if (enableTopicCreation) {
Set<String> created = Collections.singleton(topic);
Set<String> existing = Collections.emptySet();
TopicAdmin.TopicCreationResponse response = new
TopicAdmin.TopicCreationResponse(created, existing);
-
EasyMock.expect(admin.createOrFindTopics(EasyMock.capture(newTopicCapture))).andReturn(response);
+
when(admin.createOrFindTopics(newTopicCapture.capture())).thenReturn(response);
Review Comment:
Let's remove them in that case please. I understand that it complicates the
code review and changes more than a pure migration, but I am sure reviewers
wouldn't mind some code simplification here.
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -177,16 +180,14 @@ public class ErrorHandlingTaskTest {
private ErrorHandlingMetrics errorHandlingMetrics;
- private boolean enableTopicCreation;
+ private boolean enableTopicCreation = false;
Review Comment:
`= false` could be removed. The default value of primitives (such as
boolean, int, long etc. is automatically set in Java and default value for
boolean is false).
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -197,10 +198,16 @@ public void setup() {
workerProps.put("value.converter",
"org.apache.kafka.connect.json.JsonConverter");
workerProps.put("offset.storage.file.filename",
"/tmp/connect.offsets");
workerProps.put(TOPIC_CREATION_ENABLE_CONFIG,
String.valueOf(enableTopicCreation));
- pluginLoader = PowerMock.createMock(PluginClassLoader.class);
+ pluginLoader = mock(PluginClassLoader.class);
workerConfig = new StandaloneConfig(workerProps);
sourceConfig = new SourceConnectorConfig(plugins,
sourceConnectorProps(TOPIC), true);
errorHandlingMetrics = new ErrorHandlingMetrics(taskId, metrics);
+
+ // Use strict mode to detect unused mocks
+ mockitoSession = Mockito.mockitoSession()
+ .initMocks(this)
+ .strictness(Strictness.STRICT_STUBS)
+ .startMocking();
Review Comment:
You can remove this and instead use
`@RunWith(MockitoJUnitRunner.StrictStubs.class)` as a class level annotation.
Also, you don't need to init mocks with Mockito. As an inspiration, you can
search for other tests we have recently migrated to Mockito look for such
examples there.
##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java:
##########
@@ -374,30 +367,37 @@ public void testErrorHandlingInSourceTasks() throws
Exception {
Struct struct2 = new Struct(valSchema).put("val", 6789);
SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC,
PARTITION1, valSchema, struct2);
- EasyMock.expect(workerSourceTask.isStopping()).andReturn(false);
- EasyMock.expect(workerSourceTask.isStopping()).andReturn(false);
- EasyMock.expect(workerSourceTask.isStopping()).andReturn(true);
+ when(workerSourceTask.isStopping()).thenReturn(false);
+ when(workerSourceTask.isStopping()).thenReturn(false);;
Review Comment:
nit
extra semicolon
--
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]