divijvaidya commented on code in PR #12739:
URL: https://github.com/apache/kafka/pull/12739#discussion_r997087704


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java:
##########
@@ -1159,58 +1183,42 @@ public void 
shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() {
     @Test
     public void shouldCloseCacheAndWrappedStoreAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on flush"));
+        reset(cache);
+        doThrow(new RuntimeException("Simulating an error on 
flush2")).when(cache).flush(CACHE_NAMESPACE);
         cache.close(CACHE_NAMESPACE);
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
+        reset(underlyingStore);
         underlyingStore.close();
-        EasyMock.replay(underlyingStore);
-
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     @Test
     public void shouldCloseWrappedStoreAfterErrorDuringCacheClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
+        reset(cache);
         cache.flush(CACHE_NAMESPACE);
-        cache.close(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on close"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
+        doThrow(new RuntimeException("Simulating an error on 
close")).when(cache).close(CACHE_NAMESPACE);
+        reset(underlyingStore);
         underlyingStore.close();
-        EasyMock.replay(underlyingStore);
-
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     @Test
     public void shouldCloseCacheAfterErrorDuringStateStoreClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
+        reset(cache);
         cache.flush(CACHE_NAMESPACE);
         cache.close(CACHE_NAMESPACE);
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();

Review Comment:
   please verify invocation for this



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java:
##########
@@ -1159,58 +1183,42 @@ public void 
shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() {
     @Test
     public void shouldCloseCacheAndWrappedStoreAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on flush"));
+        reset(cache);
+        doThrow(new RuntimeException("Simulating an error on 
flush2")).when(cache).flush(CACHE_NAMESPACE);
         cache.close(CACHE_NAMESPACE);
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
+        reset(underlyingStore);
         underlyingStore.close();
-        EasyMock.replay(underlyingStore);
-
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     @Test
     public void shouldCloseWrappedStoreAfterErrorDuringCacheClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
+        reset(cache);
         cache.flush(CACHE_NAMESPACE);
-        cache.close(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on close"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
+        doThrow(new RuntimeException("Simulating an error on 
close")).when(cache).close(CACHE_NAMESPACE);
+        reset(underlyingStore);
         underlyingStore.close();
-        EasyMock.replay(underlyingStore);
-
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     @Test
     public void shouldCloseCacheAfterErrorDuringStateStoreClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
+        reset(cache);
         cache.flush(CACHE_NAMESPACE);

Review Comment:
   This and the following line 1210 could be removed since they are void 
returning methods on mocks. Instead verify the invocation of these methods 
using the function I suggested in one of the comments above.
   
   same for other 2 close tests.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java:
##########
@@ -138,51 +143,70 @@ public void setUp() {
 
     @After
     public void closeStore() {
-        cachingStore.close();
+        try {
+            cachingStore.close();
+        } catch (final RuntimeException runtimeException) {

Review Comment:
   This may hide cases when the exception is genuine.
   
   Alternatively, create a function:
   ```
   private void verifyAndTearDownCloseTests() {
           verify(underlyingStore).close();
           verify(cache).flush(CACHE_NAMESPACE);
           verify(cache).close(CACHE_NAMESPACE);
   
           // resets the mocks created in #setUpCloseTests(). It is necessary to
           // ensure that @After works correctly.
           reset(cache);
           reset(underlyingStore);
       }
   ```
   
   And call it at the end of each close test.



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java:
##########
@@ -90,8 +88,15 @@
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
-
-@RunWith(Parameterized.class)

Review Comment:
   We are removing the ability to run the tests in a parameterized manner. 
Hence, this test will not for parameters defined in 
   ```
   @Parameterized.Parameters(name = "{0}")
       public static Collection<Object[]> data() {
           return asList(new Object[][] {
               {true},
               {false}
           });
       }
   ```
   
   I understand that you did this because we cannot have two `RunWith` 
annotations. You can alternatively use 
   ```
   @Rule
       public MockitoRule mockito = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
   ```
   instead of the `@RunWith(MockitoJUnitRunner.StrictStubs.class)`



##########
streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java:
##########
@@ -1159,58 +1183,42 @@ public void 
shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() {
     @Test
     public void shouldCloseCacheAndWrappedStoreAfterErrorDuringCacheFlush() {
         setUpCloseTests();
-        EasyMock.reset(cache);
-        cache.flush(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on flush"));
+        reset(cache);
+        doThrow(new RuntimeException("Simulating an error on 
flush2")).when(cache).flush(CACHE_NAMESPACE);
         cache.close(CACHE_NAMESPACE);
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
+        reset(underlyingStore);
         underlyingStore.close();
-        EasyMock.replay(underlyingStore);
-
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     @Test
     public void shouldCloseWrappedStoreAfterErrorDuringCacheClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
+        reset(cache);
         cache.flush(CACHE_NAMESPACE);
-        cache.close(CACHE_NAMESPACE);
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on close"));
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
+        doThrow(new RuntimeException("Simulating an error on 
close")).when(cache).close(CACHE_NAMESPACE);
+        reset(underlyingStore);
         underlyingStore.close();
-        EasyMock.replay(underlyingStore);
-
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     @Test
     public void shouldCloseCacheAfterErrorDuringStateStoreClose() {
         setUpCloseTests();
-        EasyMock.reset(cache);
+        reset(cache);
         cache.flush(CACHE_NAMESPACE);
         cache.close(CACHE_NAMESPACE);
-        EasyMock.replay(cache);
-        EasyMock.reset(underlyingStore);
-        underlyingStore.close();
-        EasyMock.expectLastCall().andThrow(new RuntimeException("Simulating an 
error on close"));
-        EasyMock.replay(underlyingStore);
-
+        reset(underlyingStore);
+        doThrow(new RuntimeException("Simulating an error on 
close")).when(underlyingStore).close();
         assertThrows(RuntimeException.class, cachingStore::close);
-        EasyMock.verify(cache, underlyingStore);
     }
 
     private void setUpCloseTests() {
-        underlyingStore = 
EasyMock.createNiceMock(RocksDBTimeOrderedWindowStore.class);
-        EasyMock.expect(underlyingStore.name()).andStubReturn("store-name");
-        EasyMock.expect(underlyingStore.isOpen()).andStubReturn(true);
-        EasyMock.replay(underlyingStore);
-        cachingStore = new TimeOrderedCachingWindowStore(underlyingStore, 
WINDOW_SIZE, SEGMENT_INTERVAL);
-        cache = EasyMock.createNiceMock(ThreadCache.class);
+        underlyingStore = mock(RocksDBTimeOrderedWindowStore.class);
+        when(underlyingStore.name()).thenReturn("store-name");
+        cachingStore = spy(new TimeOrderedCachingWindowStore(underlyingStore,

Review Comment:
   I wasn't able to understand this change. Why do we need to spy this?



-- 
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-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to