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


##########
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java:
##########
@@ -706,95 +662,72 @@ public void testSourceTaskIgnoresProducerException() 
throws Exception {
         // and no ConnectException will be thrown
         SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, 
KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD);
         SourceRecord record2 = new SourceRecord(PARTITION, offset2, TOPIC, 2, 
KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD);
-        expectOffsetFlush(true);
-        expectSendRecordOnce();
-        expectSendRecordProducerCallbackFail();
-        sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), 
EasyMock.isNull());
 
-        //As of KAFKA-14079 all offsets should be committed, even for failed 
records (if ignored)
-        //Only the last offset will be passed to the method as everything up 
to that point is committed
-        //Before KAFKA-14079 offset 12 would have been passed and not 13 as it 
would have been unacked
-        offsetWriter.offset(PARTITION, offset2);
-        PowerMock.expectLastCall();
+        expectOffsetFlush();
+        expectPreliminaryCalls();
 
-        PowerMock.replayAll();
+        when(producer.send(any(ProducerRecord.class), any(Callback.class)))
+                .thenAnswer(producerSendAnswer(true))
+                .thenAnswer(producerSendAnswer(false));
 
         //Send records and then commit offsets and verify both were committed 
and no exception
-        Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, 
record2));
-        Whitebox.invokeMethod(workerTask, "sendRecords");
-        Whitebox.invokeMethod(workerTask, "updateCommittableOffsets");
+        workerTask.toSend = Arrays.asList(record1, record2);
+        workerTask.sendRecords();
+        workerTask.updateCommittableOffsets();
         workerTask.commitOffsets();
 
-        PowerMock.verifyAll();
+        //As of KAFKA-14079 all offsets should be committed, even for failed 
records (if ignored)
+        //Only the last offset will be passed to the method as everything up 
to that point is committed
+        //Before KAFKA-14079 offset 12 would have been passed and not 13 as it 
would have been unacked
+        verify(offsetWriter).offset(PARTITION, offset2);
+        verify(sourceTask).commitRecord(any(SourceRecord.class), isNull());
 
         //Double check to make sure all submitted records were cleared
-        assertEquals(0, ((SubmittedRecords) 
Whitebox.getInternalState(workerTask,
-                "submittedRecords")).records.size());
+        assertEquals(0, workerTask.submittedRecords.records.size());
     }
 
     @Test
     public void testSlowTaskStart() throws Exception {
         final CountDownLatch startupLatch = new CountDownLatch(1);
         final CountDownLatch finishStartupLatch = new CountDownLatch(1);
-
         createWorkerTask();
 
-        offsetStore.start();
-        EasyMock.expectLastCall();
-        sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
-        EasyMock.expectLastCall();
-        sourceTask.start(TASK_PROPS);
-        EasyMock.expectLastCall().andAnswer(() -> {
+        doAnswer((Answer<Object>) invocation -> {
             startupLatch.countDown();
-            assertTrue(awaitLatch(finishStartupLatch));
+            ConcurrencyUtils.awaitLatch(finishStartupLatch, "Timeout waiting 
for task to stop");

Review Comment:
   The purpose of `finishStartupLatch` is to ensure that we invoke 
`WorkerTask::stop` while the `WorkerTask` (on its separate thread) is in the 
middle of invoking `SourceTask::start`.
   
   We should add that logic back. I was only commenting on the error message; 
the rest of the test case looked correct.



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