cadonna commented on code in PR #12492:
URL: https://github.com/apache/kafka/pull/12492#discussion_r944375842
##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/MaterializedInternalTest.java:
##########
@@ -73,11 +71,11 @@ public void shouldUseProvidedStoreNameWhenSet() {
@Test
public void shouldUseStoreNameOfSupplierWhenProvided() {
final String storeName = "other-store-name";
- EasyMock.expect(supplier.name()).andReturn(storeName).anyTimes();
- EasyMock.replay(supplier);
+ when(supplier.name()).thenReturn(storeName);
final MaterializedInternal<Object, Object, KeyValueStore<Bytes,
byte[]>> materialized =
new MaterializedInternal<>(Materialized.as(supplier),
nameProvider, prefix);
assertThat(materialized.storeName(), equalTo(storeName));
+ verify(supplier, times(2)).name();
Review Comment:
Why `times(2)`? The original test does not says anything about the number of
calls. I think if you swap line 77 with line 78, you can remove `times(2)`.
##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/TransformerSupplierAdapterTest.java:
##########
@@ -145,10 +141,12 @@ public void shouldAlwaysGetNewAdapterTransformer() {
final Transformer<String, String, Iterable<KeyValue<Integer,
Integer>>> adapterTransformer3 = adapter.get();
adapterTransformer3.init(context);
- verifyAll();
assertThat(adapterTransformer1,
not(sameInstance(adapterTransformer2)));
assertThat(adapterTransformer2,
not(sameInstance(adapterTransformer3)));
assertThat(adapterTransformer3,
not(sameInstance(adapterTransformer1)));
+ verify(transformer1).init(any(ProcessorContext.class));
+ verify(transformer2).init(any(ProcessorContext.class));
+ verify(transformer3).init(any(ProcessorContext.class));
Review Comment:
Why not?
```
verify(transformer1).init(context);
verify(transformer2).init(context);
verify(transformer3).init(context);
```
##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java:
##########
@@ -47,19 +56,14 @@ public class KStreamFlatTransformTest extends
EasyMockSupport {
public void setUp() {
inputKey = 1;
inputValue = 10;
- transformer = mock(Transformer.class);
- context = strictMock(InternalProcessorContext.class);
Review Comment:
In EasyMock a strict mock verifies also the order of the calls. In Mockito
that is done by using `InOrder`
(https://javadoc.io/doc/org.mockito/mockito-core/latest/org/mockito/Mockito.html#in_order_verification).
Currently, this PR does not check the order which softens up the test.
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java:
##########
@@ -148,34 +155,27 @@ private void createMockTaskManager(final Set<TaskId>
activeTasks) {
}
private void createMockTaskManager(final Map<TaskId, Long> taskOffsetSums)
{
- taskManager = EasyMock.createNiceMock(TaskManager.class);
- expect(taskManager.topologyMetadata()).andStubReturn(topologyMetadata);
-
expect(taskManager.getTaskOffsetSums()).andReturn(taskOffsetSums).anyTimes();
Review Comment:
Why was this stub not migrated?
##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java:
##########
@@ -47,19 +56,14 @@ public class KStreamFlatTransformValuesTest extends
EasyMockSupport {
public void setUp() {
inputKey = 1;
inputValue = 10;
- valueTransformer = mock(ValueTransformerWithKey.class);
- context = strictMock(InternalProcessorContext.class);
Review Comment:
n EasyMock a strict mock verifies also the order of the calls. In Mockito
that is done by using `InOrder`
(https://javadoc.io/doc/org.mockito/mockito-core/latest/org/mockito/Mockito.html#in_order_verification).
Currently, this PR does not check the order which softens up the test.
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/ClientUtilsTest.java:
##########
@@ -108,75 +110,73 @@ public class ClientUtilsTest {
@Test
public void
fetchCommittedOffsetsShouldRethrowKafkaExceptionAsStreamsException() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
- expect(consumer.committed(PARTITIONS)).andThrow(new KafkaException());
- replay(consumer);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
+ when(consumer.committed(PARTITIONS)).thenThrow(new KafkaException());
assertThrows(StreamsException.class, () ->
fetchCommittedOffsets(PARTITIONS, consumer));
}
@Test
public void fetchCommittedOffsetsShouldRethrowTimeoutException() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
- expect(consumer.committed(PARTITIONS)).andThrow(new
TimeoutException());
- replay(consumer);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
+ when(consumer.committed(PARTITIONS)).thenThrow(new TimeoutException());
assertThrows(TimeoutException.class, () ->
fetchCommittedOffsets(PARTITIONS, consumer));
}
@Test
public void
fetchCommittedOffsetsShouldReturnEmptyMapIfPartitionsAreEmpty() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
assertTrue(fetchCommittedOffsets(emptySet(), consumer).isEmpty());
}
@Test
public void fetchEndOffsetsShouldReturnEmptyMapIfPartitionsAreEmpty() {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
+ final Admin adminClient = mock(AdminClient.class);
assertTrue(fetchEndOffsets(emptySet(), adminClient).isEmpty());
}
@Test
public void
fetchEndOffsetsShouldRethrowRuntimeExceptionAsStreamsException() throws
Exception {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
- final ListOffsetsResult result =
EasyMock.createNiceMock(ListOffsetsResult.class);
- final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = EasyMock.createMock(KafkaFuture.class);
+ final Admin adminClient = mock(AdminClient.class);
+ final ListOffsetsResult result = mock(ListOffsetsResult.class);
+ @SuppressWarnings("unchecked")
+ final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = mock(KafkaFuture.class);
-
EasyMock.expect(adminClient.listOffsets(EasyMock.anyObject())).andStubReturn(result);
- EasyMock.expect(result.all()).andStubReturn(allFuture);
- EasyMock.expect(allFuture.get()).andThrow(new RuntimeException());
- replay(adminClient, result, allFuture);
+ when(adminClient.listOffsets(any())).thenReturn(result);
+ when(result.all()).thenReturn(allFuture);
+ when(allFuture.get()).thenThrow(new RuntimeException());
assertThrows(StreamsException.class, () -> fetchEndOffsets(PARTITIONS,
adminClient));
- verify(adminClient);
Review Comment:
Verifications of the admin client are missing.
##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/TransformerSupplierAdapterTest.java:
##########
@@ -24,117 +24,113 @@
import org.apache.kafka.streams.kstream.TransformerSupplier;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.state.StoreBuilder;
-import org.easymock.EasyMock;
-import org.easymock.EasyMockSupport;
-import org.junit.Before;
import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
import static org.hamcrest.core.IsEqual.equalTo;
import static org.hamcrest.core.IsSame.sameInstance;
import static org.hamcrest.core.IsNot.not;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
-public class TransformerSupplierAdapterTest extends EasyMockSupport {
+@RunWith(MockitoJUnitRunner.StrictStubs.class)
+public class TransformerSupplierAdapterTest {
+ @Mock
private ProcessorContext context;
+ @Mock
private Transformer<String, String, KeyValue<Integer, Integer>>
transformer;
+ @Mock
private TransformerSupplier<String, String, KeyValue<Integer, Integer>>
transformerSupplier;
+ @Mock
private Set<StoreBuilder<?>> stores;
final String key = "Hello";
final String value = "World";
- @Before
- public void before() {
- context = mock(ProcessorContext.class);
- transformer = mock(Transformer.class);
- transformerSupplier = mock(TransformerSupplier.class);
- stores = mock(Set.class);
- }
-
@Test
public void shouldCallInitOfAdapteeTransformer() {
- EasyMock.expect(transformerSupplier.get()).andReturn(transformer);
- transformer.init(context);
- replayAll();
+ when(transformerSupplier.get()).thenReturn(transformer);
final TransformerSupplierAdapter<String, String, Integer, Integer>
adapter =
new TransformerSupplierAdapter<>(transformerSupplier);
final Transformer<String, String, Iterable<KeyValue<Integer,
Integer>>> adaptedTransformer = adapter.get();
adaptedTransformer.init(context);
- verifyAll();
+ verify(transformerSupplier).get();
+ verify(transformer).init(any(ProcessorContext.class));
Review Comment:
Is it possible to use `verify(transformer).init(context)`?
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/ClientUtilsTest.java:
##########
@@ -108,75 +110,73 @@ public class ClientUtilsTest {
@Test
public void
fetchCommittedOffsetsShouldRethrowKafkaExceptionAsStreamsException() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
- expect(consumer.committed(PARTITIONS)).andThrow(new KafkaException());
- replay(consumer);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
+ when(consumer.committed(PARTITIONS)).thenThrow(new KafkaException());
assertThrows(StreamsException.class, () ->
fetchCommittedOffsets(PARTITIONS, consumer));
}
@Test
public void fetchCommittedOffsetsShouldRethrowTimeoutException() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
- expect(consumer.committed(PARTITIONS)).andThrow(new
TimeoutException());
- replay(consumer);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
+ when(consumer.committed(PARTITIONS)).thenThrow(new TimeoutException());
assertThrows(TimeoutException.class, () ->
fetchCommittedOffsets(PARTITIONS, consumer));
}
@Test
public void
fetchCommittedOffsetsShouldReturnEmptyMapIfPartitionsAreEmpty() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
assertTrue(fetchCommittedOffsets(emptySet(), consumer).isEmpty());
}
@Test
public void fetchEndOffsetsShouldReturnEmptyMapIfPartitionsAreEmpty() {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
+ final Admin adminClient = mock(AdminClient.class);
assertTrue(fetchEndOffsets(emptySet(), adminClient).isEmpty());
}
@Test
public void
fetchEndOffsetsShouldRethrowRuntimeExceptionAsStreamsException() throws
Exception {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
- final ListOffsetsResult result =
EasyMock.createNiceMock(ListOffsetsResult.class);
- final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = EasyMock.createMock(KafkaFuture.class);
+ final Admin adminClient = mock(AdminClient.class);
+ final ListOffsetsResult result = mock(ListOffsetsResult.class);
+ @SuppressWarnings("unchecked")
+ final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = mock(KafkaFuture.class);
-
EasyMock.expect(adminClient.listOffsets(EasyMock.anyObject())).andStubReturn(result);
- EasyMock.expect(result.all()).andStubReturn(allFuture);
- EasyMock.expect(allFuture.get()).andThrow(new RuntimeException());
- replay(adminClient, result, allFuture);
+ when(adminClient.listOffsets(any())).thenReturn(result);
+ when(result.all()).thenReturn(allFuture);
+ when(allFuture.get()).thenThrow(new RuntimeException());
assertThrows(StreamsException.class, () -> fetchEndOffsets(PARTITIONS,
adminClient));
- verify(adminClient);
}
@Test
public void
fetchEndOffsetsShouldRethrowInterruptedExceptionAsStreamsException() throws
Exception {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
- final ListOffsetsResult result =
EasyMock.createNiceMock(ListOffsetsResult.class);
- final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = EasyMock.createMock(KafkaFuture.class);
+ final Admin adminClient = mock(AdminClient.class);
+ final ListOffsetsResult result = mock(ListOffsetsResult.class);
+ @SuppressWarnings("unchecked")
+ final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = mock(KafkaFuture.class);
-
EasyMock.expect(adminClient.listOffsets(EasyMock.anyObject())).andStubReturn(result);
- EasyMock.expect(result.all()).andStubReturn(allFuture);
- EasyMock.expect(allFuture.get()).andThrow(new InterruptedException());
- replay(adminClient, result, allFuture);
+ when(adminClient.listOffsets(any())).thenReturn(result);
+ when(result.all()).thenReturn(allFuture);
+ when(allFuture.get()).thenThrow(new InterruptedException());
assertThrows(StreamsException.class, () -> fetchEndOffsets(PARTITIONS,
adminClient));
- verify(adminClient);
Review Comment:
Verifications of the admin client are missing.
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/ClientUtilsTest.java:
##########
@@ -108,75 +110,73 @@ public class ClientUtilsTest {
@Test
public void
fetchCommittedOffsetsShouldRethrowKafkaExceptionAsStreamsException() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
- expect(consumer.committed(PARTITIONS)).andThrow(new KafkaException());
- replay(consumer);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
+ when(consumer.committed(PARTITIONS)).thenThrow(new KafkaException());
assertThrows(StreamsException.class, () ->
fetchCommittedOffsets(PARTITIONS, consumer));
}
@Test
public void fetchCommittedOffsetsShouldRethrowTimeoutException() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
- expect(consumer.committed(PARTITIONS)).andThrow(new
TimeoutException());
- replay(consumer);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
+ when(consumer.committed(PARTITIONS)).thenThrow(new TimeoutException());
assertThrows(TimeoutException.class, () ->
fetchCommittedOffsets(PARTITIONS, consumer));
}
@Test
public void
fetchCommittedOffsetsShouldReturnEmptyMapIfPartitionsAreEmpty() {
- final Consumer<byte[], byte[]> consumer =
EasyMock.createMock(Consumer.class);
+ @SuppressWarnings("unchecked")
+ final Consumer<byte[], byte[]> consumer = mock(Consumer.class);
assertTrue(fetchCommittedOffsets(emptySet(), consumer).isEmpty());
}
@Test
public void fetchEndOffsetsShouldReturnEmptyMapIfPartitionsAreEmpty() {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
+ final Admin adminClient = mock(AdminClient.class);
assertTrue(fetchEndOffsets(emptySet(), adminClient).isEmpty());
}
@Test
public void
fetchEndOffsetsShouldRethrowRuntimeExceptionAsStreamsException() throws
Exception {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
- final ListOffsetsResult result =
EasyMock.createNiceMock(ListOffsetsResult.class);
- final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = EasyMock.createMock(KafkaFuture.class);
+ final Admin adminClient = mock(AdminClient.class);
+ final ListOffsetsResult result = mock(ListOffsetsResult.class);
+ @SuppressWarnings("unchecked")
+ final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = mock(KafkaFuture.class);
-
EasyMock.expect(adminClient.listOffsets(EasyMock.anyObject())).andStubReturn(result);
- EasyMock.expect(result.all()).andStubReturn(allFuture);
- EasyMock.expect(allFuture.get()).andThrow(new RuntimeException());
- replay(adminClient, result, allFuture);
+ when(adminClient.listOffsets(any())).thenReturn(result);
+ when(result.all()).thenReturn(allFuture);
+ when(allFuture.get()).thenThrow(new RuntimeException());
assertThrows(StreamsException.class, () -> fetchEndOffsets(PARTITIONS,
adminClient));
- verify(adminClient);
}
@Test
public void
fetchEndOffsetsShouldRethrowInterruptedExceptionAsStreamsException() throws
Exception {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
- final ListOffsetsResult result =
EasyMock.createNiceMock(ListOffsetsResult.class);
- final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = EasyMock.createMock(KafkaFuture.class);
+ final Admin adminClient = mock(AdminClient.class);
+ final ListOffsetsResult result = mock(ListOffsetsResult.class);
+ @SuppressWarnings("unchecked")
+ final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = mock(KafkaFuture.class);
-
EasyMock.expect(adminClient.listOffsets(EasyMock.anyObject())).andStubReturn(result);
- EasyMock.expect(result.all()).andStubReturn(allFuture);
- EasyMock.expect(allFuture.get()).andThrow(new InterruptedException());
- replay(adminClient, result, allFuture);
+ when(adminClient.listOffsets(any())).thenReturn(result);
+ when(result.all()).thenReturn(allFuture);
+ when(allFuture.get()).thenThrow(new InterruptedException());
assertThrows(StreamsException.class, () -> fetchEndOffsets(PARTITIONS,
adminClient));
- verify(adminClient);
}
@Test
public void
fetchEndOffsetsShouldRethrowExecutionExceptionAsStreamsException() throws
Exception {
- final Admin adminClient = EasyMock.createMock(AdminClient.class);
- final ListOffsetsResult result =
EasyMock.createNiceMock(ListOffsetsResult.class);
- final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = EasyMock.createMock(KafkaFuture.class);
+ final Admin adminClient = mock(AdminClient.class);
+ final ListOffsetsResult result = mock(ListOffsetsResult.class);
+ @SuppressWarnings("unchecked")
+ final KafkaFuture<Map<TopicPartition, ListOffsetsResultInfo>>
allFuture = mock(KafkaFuture.class);
-
EasyMock.expect(adminClient.listOffsets(EasyMock.anyObject())).andStubReturn(result);
- EasyMock.expect(result.all()).andStubReturn(allFuture);
- EasyMock.expect(allFuture.get()).andThrow(new ExecutionException(new
RuntimeException()));
- replay(adminClient, result, allFuture);
+ when(adminClient.listOffsets(any())).thenReturn(result);
+ when(result.all()).thenReturn(allFuture);
+ when(allFuture.get()).thenThrow(new ExecutionException(new
RuntimeException()));
assertThrows(StreamsException.class, () -> fetchEndOffsets(PARTITIONS,
adminClient));
- verify(adminClient);
Review Comment:
Verifications of the admin client are missing.
--
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]