You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/09/28 22:41:38 UTC

[GitHub] [kafka] rhauch commented on a change in pull request #8910: KAFKA-10188: Prevent SinkTask::preCommit from being called after SinkTask::stop

rhauch commented on a change in pull request #8910:
URL: https://github.com/apache/kafka/pull/8910#discussion_r496239771



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java
##########
@@ -138,6 +139,7 @@ public WorkerSinkTask(ConnectorTaskId id,
         this.sinkTaskMetricsGroup.recordOffsetSequenceNumber(commitSeqno);
         this.consumer = consumer;
         this.isTopicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
+        this.taskStopped = false;

Review comment:
       Shouldn't this be volatile?
   
   Yes, it's true that `WorkerSinkTask.close()` is always and only called from within the `WorkerTask.doRun()` after the tasks determines it will stop. However, the `onPartitionsRevoked(...)` method is called from the consumer thread, and making this volatile is the only way to ensure that the consumer thread doesn't read a previously-cached value.

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
##########
@@ -315,6 +315,56 @@ public void testPause() throws Exception {
         PowerMock.verifyAll();
     }
 
+    @Test
+    public void testShutdown() throws Exception {
+        createTask(initialState);
+
+        expectInitializeTask();
+        expectTaskGetTopic(true);
+
+        // first iteration
+        expectPollInitialAssignment();
+
+        // second iteration
+        EasyMock.expect(sinkTask.preCommit(EasyMock.anyObject())).andReturn(Collections.emptyMap());
+        expectConsumerPoll(1);
+        expectConversionAndTransformation(1);
+        sinkTask.put(EasyMock.<Collection<SinkRecord>>anyObject());
+        EasyMock.expectLastCall();
+
+        // WorkerSinkTask::stop
+        consumer.wakeup();
+        PowerMock.expectLastCall();
+        sinkTask.stop();
+        PowerMock.expectLastCall();
+
+        // WorkerSinkTask::close
+        consumer.close();
+        PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
+            @Override
+            public Object answer() throws Throwable {
+                rebalanceListener.getValue().onPartitionsRevoked(
+                    asList(TOPIC_PARTITION, TOPIC_PARTITION2)
+                );
+                return null;
+            }
+        });
+        transformationChain.close();
+        PowerMock.expectLastCall();
+
+        PowerMock.replayAll();
+
+        workerTask.initialize(TASK_CONFIG);
+        workerTask.initializeAndStart();
+        workerTask.iteration();
+        sinkTaskContext.getValue().requestCommit(); // Force an offset commit
+        workerTask.iteration();
+        workerTask.stop();
+        workerTask.close();
+
+        PowerMock.verifyAll();

Review comment:
       Verified locally that this test fails when the additions to the `onPartitionsRevoked(...)` method above are removed locally. Nice work, @C0urante.




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

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