C0urante commented on code in PR #12418:
URL: https://github.com/apache/kafka/pull/12418#discussion_r951657274


##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java:
##########
@@ -105,34 +112,53 @@ public class KafkaOffsetBackingStoreTest {
 
     @Mock
     KafkaBasedLog<byte[], byte[]> storeLog;
+    @Mock
+    Supplier<TopicAdmin> topicAdminSupplier;
+    @Spy
+    @InjectMocks
     private KafkaOffsetBackingStore store;
 
-    private Capture<String> capturedTopic = EasyMock.newCapture();
-    private Capture<Map<String, Object>> capturedProducerProps = 
EasyMock.newCapture();
-    private Capture<Map<String, Object>> capturedConsumerProps = 
EasyMock.newCapture();
-    private Capture<Supplier<TopicAdmin>> capturedAdminSupplier = 
EasyMock.newCapture();
-    private Capture<NewTopic> capturedNewTopic = EasyMock.newCapture();
-    private Capture<Callback<ConsumerRecord<byte[], byte[]>>> 
capturedConsumedCallback = EasyMock.newCapture();
+    private MockedStatic<ConnectUtils> connectUtilsMockedStatic;
+
+    private final ArgumentCaptor<String> capturedTopic = 
ArgumentCaptor.forClass(String.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Map<String, Object>> capturedProducerProps = 
ArgumentCaptor.forClass(Map.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Map<String, Object>> capturedConsumerProps = 
ArgumentCaptor.forClass(Map.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Supplier<TopicAdmin>> capturedAdminSupplier = 
ArgumentCaptor.forClass(Supplier.class);
+    private final ArgumentCaptor<NewTopic> capturedNewTopic = 
ArgumentCaptor.forClass(NewTopic.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Callback<ConsumerRecord<byte[], byte[]>>> 
capturedConsumedCallback = ArgumentCaptor.forClass(Callback.class);
 
     @Before
-    public void setUp() throws Exception {
-        store = 
PowerMock.createPartialMockAndInvokeDefaultConstructor(KafkaOffsetBackingStore.class,
 "createKafkaBasedLog");
-    }
+    public void setup() {
+        
doReturn(storeLog).when(store).createKafkaBasedLog(capturedTopic.capture(), 
capturedProducerProps.capture(),
+                capturedConsumerProps.capture(), 
capturedConsumedCallback.capture(),
+                capturedNewTopic.capture(), capturedAdminSupplier.capture());
 
-    @Test
-    public void testStartStop() throws Exception {
-        expectConfigure();
-        expectStart(Collections.emptyList());
-        expectStop();
-        expectClusterId();
+        connectUtilsMockedStatic = mockStatic(ConnectUtils.class, 
Mockito.CALLS_REAL_METHODS);
+        connectUtilsMockedStatic.when(() -> 
ConnectUtils.lookupKafkaClusterId(any())).thenReturn("test-cluster");
+    }
 
-        PowerMock.replayAll();
+    @After
+    public void tearDown() {
+        connectUtilsMockedStatic.close();

Review Comment:
   We should also make sure that none of the tests resulted in unexpected calls 
to start, stop, etc. the underlying `KafkaBasedLog`:
   ```suggestion
           connectUtilsMockedStatic.close();
           Mockito.verifyNoMoreInteractions(storeLog);
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java:
##########
@@ -105,34 +112,53 @@ public class KafkaOffsetBackingStoreTest {
 
     @Mock
     KafkaBasedLog<byte[], byte[]> storeLog;
+    @Mock
+    Supplier<TopicAdmin> topicAdminSupplier;
+    @Spy
+    @InjectMocks
     private KafkaOffsetBackingStore store;
 
-    private Capture<String> capturedTopic = EasyMock.newCapture();
-    private Capture<Map<String, Object>> capturedProducerProps = 
EasyMock.newCapture();
-    private Capture<Map<String, Object>> capturedConsumerProps = 
EasyMock.newCapture();
-    private Capture<Supplier<TopicAdmin>> capturedAdminSupplier = 
EasyMock.newCapture();
-    private Capture<NewTopic> capturedNewTopic = EasyMock.newCapture();
-    private Capture<Callback<ConsumerRecord<byte[], byte[]>>> 
capturedConsumedCallback = EasyMock.newCapture();
+    private MockedStatic<ConnectUtils> connectUtilsMockedStatic;
+
+    private final ArgumentCaptor<String> capturedTopic = 
ArgumentCaptor.forClass(String.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Map<String, Object>> capturedProducerProps = 
ArgumentCaptor.forClass(Map.class);

Review Comment:
   We can use the `@Captor` annotation here and elsewhere to simplify things a 
bit:
   ```suggestion
       @Captor
       private ArgumentCaptor<Map<String, Object>> capturedProducerProps;
   ```



##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStore.java:
##########
@@ -214,7 +214,7 @@ public void configure(final WorkerConfig config) {
         this.offsetLog = createKafkaBasedLog(topic, producerProps, 
consumerProps, consumedCallback, topicDescription, adminSupplier);
     }
 
-    private KafkaBasedLog<byte[], byte[]> createKafkaBasedLog(String topic, 
Map<String, Object> producerProps,

Review Comment:
   Nit: This makes more sense as package-private than protected, since we don't 
intend for this method to be overridden by subclasses at the moment.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java:
##########
@@ -105,34 +112,53 @@ public class KafkaOffsetBackingStoreTest {
 
     @Mock
     KafkaBasedLog<byte[], byte[]> storeLog;
+    @Mock
+    Supplier<TopicAdmin> topicAdminSupplier;
+    @Spy
+    @InjectMocks
     private KafkaOffsetBackingStore store;
 
-    private Capture<String> capturedTopic = EasyMock.newCapture();
-    private Capture<Map<String, Object>> capturedProducerProps = 
EasyMock.newCapture();
-    private Capture<Map<String, Object>> capturedConsumerProps = 
EasyMock.newCapture();
-    private Capture<Supplier<TopicAdmin>> capturedAdminSupplier = 
EasyMock.newCapture();
-    private Capture<NewTopic> capturedNewTopic = EasyMock.newCapture();
-    private Capture<Callback<ConsumerRecord<byte[], byte[]>>> 
capturedConsumedCallback = EasyMock.newCapture();
+    private MockedStatic<ConnectUtils> connectUtilsMockedStatic;
+
+    private final ArgumentCaptor<String> capturedTopic = 
ArgumentCaptor.forClass(String.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Map<String, Object>> capturedProducerProps = 
ArgumentCaptor.forClass(Map.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Map<String, Object>> capturedConsumerProps = 
ArgumentCaptor.forClass(Map.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Supplier<TopicAdmin>> capturedAdminSupplier = 
ArgumentCaptor.forClass(Supplier.class);
+    private final ArgumentCaptor<NewTopic> capturedNewTopic = 
ArgumentCaptor.forClass(NewTopic.class);
+    @SuppressWarnings("unchecked")
+    private final ArgumentCaptor<Callback<ConsumerRecord<byte[], byte[]>>> 
capturedConsumedCallback = ArgumentCaptor.forClass(Callback.class);
 
     @Before
-    public void setUp() throws Exception {
-        store = 
PowerMock.createPartialMockAndInvokeDefaultConstructor(KafkaOffsetBackingStore.class,
 "createKafkaBasedLog");
-    }
+    public void setup() {
+        
doReturn(storeLog).when(store).createKafkaBasedLog(capturedTopic.capture(), 
capturedProducerProps.capture(),
+                capturedConsumerProps.capture(), 
capturedConsumedCallback.capture(),
+                capturedNewTopic.capture(), capturedAdminSupplier.capture());
 
-    @Test
-    public void testStartStop() throws Exception {
-        expectConfigure();
-        expectStart(Collections.emptyList());
-        expectStop();
-        expectClusterId();
+        connectUtilsMockedStatic = mockStatic(ConnectUtils.class, 
Mockito.CALLS_REAL_METHODS);
+        connectUtilsMockedStatic.when(() -> 
ConnectUtils.lookupKafkaClusterId(any())).thenReturn("test-cluster");

Review Comment:
   This is so much cleaner to have in `setup` and `tearDown` instead of having 
to explicitly invoke `expectClusterId()` in individual test cases. Nice! 🎉 



##########
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java:
##########
@@ -105,34 +112,53 @@ public class KafkaOffsetBackingStoreTest {
 
     @Mock
     KafkaBasedLog<byte[], byte[]> storeLog;
+    @Mock
+    Supplier<TopicAdmin> topicAdminSupplier;
+    @Spy
+    @InjectMocks

Review Comment:
   I usually like annotations when they simplify things, but `@InjectMocks` is 
a bit too magical for my taste, and it restricts us from being able to pass in 
non-mocked and non-null constructor arguments. I also don't see it in use 
anywhere else in our tests.
   
   Could we replace this with some logic in `setup()` to explicitly construct 
the `store`?



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