gharris1727 commented on code in PR #12781:
URL: https://github.com/apache/kafka/pull/12781#discussion_r1006005644


##########
connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java:
##########
@@ -98,7 +98,7 @@
     private Optional<Producer<K, V>> producer;
     private TopicAdmin admin;
 
-    private Thread thread;
+    Thread thread;

Review Comment:
   ```suggestion
       // visible for testing
       Thread thread;
   ```



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java:
##########
@@ -547,32 +538,18 @@ public void 
testReadEndOffsetsUsingAdminThatFailsWithRetriable() throws Exceptio
     private void setupWithAdmin() {
         Supplier<TopicAdmin> adminSupplier = () -> admin;
         java.util.function.Consumer<TopicAdmin> initializer = admin -> { };
-        store = PowerMock.createPartialMock(KafkaBasedLog.class, new 
String[]{"createConsumer", "createProducer"},
-                TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, adminSupplier, 
consumedCallback, time, initializer);
+        store = new MockedKafkaBasedLog(TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, 
adminSupplier, consumedCallback, time, initializer);
     }
 
-    private void expectProducerAndConsumerCreate() throws Exception {
-        PowerMock.expectPrivate(store, "createProducer")
-                 .andReturn(producer);
-        PowerMock.expectPrivate(store, "createConsumer")
-                 .andReturn(consumer);
-    }
-
-    private void expectStart() throws Exception {
+    private void expectStart() {
         initializer.run();
-        EasyMock.expectLastCall().times(1);
-
-        expectProducerAndConsumerCreate();
+        verify(initializer, times(1)).run();

Review Comment:
   Does the initializer.run() call trivially satisfy the following verify() 
call?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java:
##########
@@ -135,8 +161,7 @@ public class KafkaBasedLogTest {
     @SuppressWarnings("unchecked")
     @Before
     public void setUp() {
-        store = PowerMock.createPartialMock(KafkaBasedLog.class, new 
String[]{"createConsumer", "createProducer"},
-                TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, consumedCallback, time, 
initializer);
+        store = new MockedKafkaBasedLog(TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, 
() -> null, consumedCallback, time, null);

Review Comment:
   I think the `@Mock private Runnable initializer` is the incorrect signature 
here, and should be changed to be a Consumer mock. Otherwise the initializer 
value isn't behaving like a mock for the mocked kafka based log.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java:
##########
@@ -547,32 +538,18 @@ public void 
testReadEndOffsetsUsingAdminThatFailsWithRetriable() throws Exceptio
     private void setupWithAdmin() {
         Supplier<TopicAdmin> adminSupplier = () -> admin;
         java.util.function.Consumer<TopicAdmin> initializer = admin -> { };
-        store = PowerMock.createPartialMock(KafkaBasedLog.class, new 
String[]{"createConsumer", "createProducer"},
-                TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, adminSupplier, 
consumedCallback, time, initializer);
+        store = new MockedKafkaBasedLog(TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, 
adminSupplier, consumedCallback, time, initializer);
     }
 
-    private void expectProducerAndConsumerCreate() throws Exception {
-        PowerMock.expectPrivate(store, "createProducer")
-                 .andReturn(producer);
-        PowerMock.expectPrivate(store, "createConsumer")
-                 .andReturn(consumer);
-    }
-
-    private void expectStart() throws Exception {
+    private void expectStart() {
         initializer.run();
-        EasyMock.expectLastCall().times(1);
-
-        expectProducerAndConsumerCreate();
+        verify(initializer, times(1)).run();
     }
 
     private void expectStop() {
         producer.close();
-        PowerMock.expectLastCall();
+        verify(producer, times(1)).close();

Review Comment:
   Also it looks like this method is not really translated from easymock yet, 
as it is separate from the store.stop() calls throughout the test. I think that 
instead of calling store.stop(), the test should call expectStop, and then the 
expectStop calls store.stop and asserts that the shutdown procedure happens.



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java:
##########
@@ -547,32 +538,18 @@ public void 
testReadEndOffsetsUsingAdminThatFailsWithRetriable() throws Exceptio
     private void setupWithAdmin() {
         Supplier<TopicAdmin> adminSupplier = () -> admin;
         java.util.function.Consumer<TopicAdmin> initializer = admin -> { };
-        store = PowerMock.createPartialMock(KafkaBasedLog.class, new 
String[]{"createConsumer", "createProducer"},
-                TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, adminSupplier, 
consumedCallback, time, initializer);
+        store = new MockedKafkaBasedLog(TOPIC, PRODUCER_PROPS, CONSUMER_PROPS, 
adminSupplier, consumedCallback, time, initializer);
     }
 
-    private void expectProducerAndConsumerCreate() throws Exception {
-        PowerMock.expectPrivate(store, "createProducer")
-                 .andReturn(producer);
-        PowerMock.expectPrivate(store, "createConsumer")
-                 .andReturn(consumer);
-    }
-
-    private void expectStart() throws Exception {
+    private void expectStart() {
         initializer.run();
-        EasyMock.expectLastCall().times(1);
-
-        expectProducerAndConsumerCreate();
+        verify(initializer, times(1)).run();
     }
 
     private void expectStop() {
         producer.close();
-        PowerMock.expectLastCall();
+        verify(producer, times(1)).close();

Review Comment:
   Does the producer.close() call trivially satisfy the following verify() call?



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