Skip to content
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

KAFKA-10188: Prevent SinkTask::preCommit from being called after SinkTask::stop #8910

Merged
merged 1 commit into from
Oct 6, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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;
C0urante marked this conversation as resolved.
Show resolved Hide resolved
Copy link
Contributor

@rhauch rhauch Sep 28, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 the field volatile is the only way to ensure that the consumer thread reads a non-cached value.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Javadocs for the ConsumerRebalanceLister state that the callback "will only execute in the user thread as part of the poll(long) call"; I think we have a guarantee here that onPartitionsRevoked will be called on the same thread that sets taskStopped to false. A fun way to verify this is to view the exceptions that get thrown by this bug; the stack traces include these lines:

	at org.apache.kafka.connect.runtime.WorkerSinkTask$HandleRebalance.onPartitionsRevoked(WorkerSinkTask.java:695)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.invokePartitionsRevoked(ConsumerCoordinator.java:312)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onLeavePrepare(ConsumerCoordinator.java:744)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.close(AbstractCoordinator.java:976)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.close(ConsumerCoordinator.java:888)
	at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:2368)
	at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:2335)
	at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:2285)
	at org.apache.kafka.common.utils.Utils.closeQuietly(Utils.java:933)
	at org.apache.kafka.connect.runtime.WorkerSinkTask.close(WorkerSinkTask.java:174)
	at org.apache.kafka.connect.runtime.WorkerTask.doClose(WorkerTask.java:164)
	at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:191)
	at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:235)

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

Expand Down Expand Up @@ -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");
Expand Down Expand Up @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}

@Test
public void testPollRedelivery() throws Exception {
createTask(initialState);
Expand Down