-
Notifications
You must be signed in to change notification settings - Fork 14.8k
KAFKA-10188: Prevent SinkTask::preCommit from being called after SinkTask::stop #8910
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -94,6 +94,7 @@ class WorkerSinkTask extends WorkerTask { | |
| private int commitFailures; | ||
| private boolean pausedForRedelivery; | ||
| private boolean committing; | ||
| private boolean taskStopped; | ||
| private final WorkerErrantRecordReporter workerErrantRecordReporter; | ||
|
|
||
| public WorkerSinkTask(ConnectorTaskId id, | ||
|
|
@@ -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; | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldn't this be volatile? Yes, it's true that There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The Javadocs for the The only edge case I can think of might be with asynchronous offset commits, but fwict those don't trigger asynchronous rebalance listener callbacks (if they trigger rebalances or rebalance listener callbacks at all). |
||
| this.workerErrantRecordReporter = workerErrantRecordReporter; | ||
| } | ||
|
|
||
|
|
@@ -168,6 +170,7 @@ protected void close() { | |
| } catch (Throwable t) { | ||
| log.warn("Could not stop task", t); | ||
| } | ||
| taskStopped = true; | ||
| Utils.closeQuietly(consumer, "consumer"); | ||
| Utils.closeQuietly(transformationChain, "transformation chain"); | ||
| Utils.closeQuietly(retryWithToleranceOperator, "retry operator"); | ||
|
|
@@ -689,6 +692,10 @@ else if (!context.pausedPartitions().isEmpty()) | |
|
|
||
| @Override | ||
| public void onPartitionsRevoked(Collection<TopicPartition> partitions) { | ||
| if (taskStopped) { | ||
C0urante marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| log.trace("Skipping partition revocation callback as task has already been stopped"); | ||
| return; | ||
| } | ||
| log.debug("{} Partitions revoked", WorkerSinkTask.this); | ||
| try { | ||
| closePartitions(); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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(); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Verified locally that this test fails when the additions to the |
||
| } | ||
|
|
||
| @Test | ||
| public void testPollRedelivery() throws Exception { | ||
| createTask(initialState); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.