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


##########
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java:
##########
@@ -68,69 +72,60 @@ public void 
shouldTransformInputRecordToMultipleOutputValues() {
                 "Hello",
                 "Blue",
                 "Planet");
+
         processor.init(context);
-        EasyMock.reset(valueTransformer);
 
-        EasyMock.expect(valueTransformer.transform(inputKey, 
inputValue)).andReturn(outputValues);
-        for (final String outputValue : outputValues) {
-            context.forward(new Record<>(inputKey, outputValue, 0L));
-        }
-        replayAll();
+        when(valueTransformer.transform(inputKey, 
inputValue)).thenReturn(outputValues);
 
         processor.process(new Record<>(inputKey, inputValue, 0L));
 
-        verifyAll();
+        for (final String outputValue : outputValues) {
+            verify(context).forward(new Record<>(inputKey, outputValue, 0L));
+        }
     }
 
     @Test
     public void shouldEmitNoRecordIfTransformReturnsEmptyList() {
         processor.init(context);
-        EasyMock.reset(valueTransformer);
 
-        EasyMock.expect(valueTransformer.transform(inputKey, 
inputValue)).andReturn(Collections.emptyList());
-        replayAll();
+        when(valueTransformer.transform(inputKey, 
inputValue)).thenReturn(Collections.emptyList());
 
         processor.process(new Record<>(inputKey, inputValue, 0L));
 
-        verifyAll();
+        verify(context, never()).forward(ArgumentMatchers.<Record<Integer, 
String>>any());
     }
 
     @Test
     public void shouldEmitNoRecordIfTransformReturnsNull() {
         processor.init(context);
-        EasyMock.reset(valueTransformer);
 
-        EasyMock.expect(valueTransformer.transform(inputKey, 
inputValue)).andReturn(null);
-        replayAll();
+        when(valueTransformer.transform(inputKey, 
inputValue)).thenReturn(null);
 
         processor.process(new Record<>(inputKey, inputValue, 0L));
 
-        verifyAll();
+        verify(context, never()).forward(ArgumentMatchers.<Record<Integer, 
String>>any());
     }
 
     @Test
     public void shouldCloseFlatTransformValuesProcessor() {
-        valueTransformer.close();
-        replayAll();
-
         processor.close();
 
-        verifyAll();
+        verify(valueTransformer).close();
     }
 
     @Test
     public void shouldGetFlatTransformValuesProcessor() {
+        @SuppressWarnings("unchecked")
         final ValueTransformerWithKeySupplier<Integer, Integer, 
Iterable<String>> valueTransformerSupplier =
             mock(ValueTransformerWithKeySupplier.class);
         final KStreamFlatTransformValues<Integer, Integer, String> 
processorSupplier =
             new KStreamFlatTransformValues<>(valueTransformerSupplier);
 
-        
EasyMock.expect(valueTransformerSupplier.get()).andReturn(valueTransformer);
-        replayAll();
+        when(valueTransformerSupplier.get()).thenReturn(valueTransformer);
 
         final Processor<Integer, Integer, Integer, String> processor = 
processorSupplier.get();
 
-        verifyAll();
+        verify(valueTransformerSupplier).get();

Review Comment:
   @clolov I think we do not need to verify all stubs. In the case you pointed 
out above I think that verifying `verify(transformer).init(context)` is enough 
since the transformer is returned by the stub and if it were not called a would 
get a `NullpointerException`.
   
   The following test would be fine with me:
   ```
   @Test
   public void shouldCallInitOfAdapteeTransformer() {
       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);
   
       verify(transformer).init(context);
   }
   ```
   However, there might be cases where it makes sense to verify the call of the 
stub.
   
   I also think that we do not need to use 
`verifyNoMoreInteractions(transformer);` each and every time. We have a similar 
behavior with EasyMock and that leads to tests that are harder to maintain 
because even the slightest change in production code might lead to changes in 
the tests. We suffered a bit because of that in the past. Admittedly, without 
checking additional interactions you lose a bit of control in the test. I think 
it is a trade-off.   
   
   What I do not understand is why the following test does not fail when run 
with `@RunWith(MockitoJUnitRunner.StrictStubs.class)`:
   
   ```
   @Test
   public void shouldCallInitOfAdapteeTransformer() {
       final Record record = mock(Record.class);
       when(transformerSupplier.get()).thenReturn(transformer);
       when(record.hasKey()).thenReturn(true);
   
       final TransformerSupplierAdapter<String, String, Integer, Integer> 
adapter =
           new TransformerSupplierAdapter<>(transformerSupplier);
       final Transformer<String, String, Iterable<KeyValue<Integer, Integer>>> 
adaptedTransformer = adapter.get();
       adaptedTransformer.init(context);
   
       verify(transformer).init(context);
   }
   ``` 
   `record.hasKey()` is never called and so the corresponding stubbing is 
unnecessary. According to Mockito's doc 
(https://javadoc.io/static/org.mockito/mockito-core/4.6.1/org/mockito/quality/Strictness.html#STRICT_STUBS):
   
   >   Cleaner tests without unnecessary stubbings: the test fails when unused 
stubs are present (see UnnecessaryStubbingException).
   
   Am I missing something?



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

Reply via email to