-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-13348: Allow Source Tasks to Handle Producer Exceptions #11382
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
34d93a8
b956c19
22daf36
878b131
ad098eb
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 |
|---|---|---|
|
|
@@ -42,6 +42,7 @@ | |
| import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; | ||
| import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; | ||
| import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; | ||
| import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; | ||
| import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; | ||
| import org.apache.kafka.connect.runtime.isolation.Plugins; | ||
| import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; | ||
|
|
@@ -219,18 +220,32 @@ public void tearDown() { | |
| } | ||
|
|
||
| private void createWorkerTask() { | ||
| createWorkerTask(TargetState.STARTED); | ||
| createWorkerTask(TargetState.STARTED, RetryWithToleranceOperatorTest.NOOP_OPERATOR); | ||
| } | ||
|
|
||
| private void createWorkerTaskWithErrorToleration() { | ||
|
Member
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. Can we reuse the
Contributor
Author
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. +1 I have refactored the constructors to be cleaner with various parameter lists. |
||
| createWorkerTask(TargetState.STARTED, RetryWithToleranceOperatorTest.ALL_OPERATOR); | ||
| } | ||
|
|
||
| private void createWorkerTask(TargetState initialState) { | ||
| createWorkerTask(initialState, keyConverter, valueConverter, headerConverter); | ||
| createWorkerTask(initialState, RetryWithToleranceOperatorTest.NOOP_OPERATOR); | ||
| } | ||
|
|
||
| private void createWorkerTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { | ||
| createWorkerTask(initialState, keyConverter, valueConverter, headerConverter, retryWithToleranceOperator); | ||
| } | ||
|
|
||
| private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { | ||
| private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, | ||
| HeaderConverter headerConverter) { | ||
| createWorkerTask(initialState, keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.NOOP_OPERATOR); | ||
| } | ||
|
|
||
| private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, | ||
| HeaderConverter headerConverter, RetryWithToleranceOperator retryWithToleranceOperator) { | ||
| workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, | ||
| transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), | ||
| offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, | ||
| RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore, Runnable::run); | ||
| retryWithToleranceOperator, statusBackingStore, Runnable::run); | ||
| } | ||
|
|
||
| @Test | ||
|
|
@@ -815,6 +830,32 @@ public void testSendRecordsTaskCommitRecordFail() throws Exception { | |
| PowerMock.verifyAll(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSourceTaskIgnoresProducerException() throws Exception { | ||
| createWorkerTaskWithErrorToleration(); | ||
| expectTopicCreation(TOPIC); | ||
|
|
||
| // send two records | ||
| // record 1 will succeed | ||
| // record 2 will invoke the producer's failure callback, but ignore the exception via retryOperator | ||
| // 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, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); | ||
|
|
||
|
|
||
| expectSendRecordOnce(); | ||
| expectSendRecordProducerCallbackFail(); | ||
| sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), EasyMock.isNull()); | ||
| EasyMock.expectLastCall(); | ||
|
|
||
| PowerMock.replayAll(); | ||
|
|
||
| Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); | ||
| Whitebox.invokeMethod(workerTask, "sendRecords"); | ||
|
|
||
| PowerMock.verifyAll(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSlowTaskStart() throws Exception { | ||
| final CountDownLatch startupLatch = new CountDownLatch(1); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we have a debug/trace log in this path?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Previously it was suggested to have the tolerance operator handle via the logging report. I would personally find it useful to have it in the connect log regardless of tolerance error logging configuration. I've moved the error/debug log lines to above the tolerance check to log in all instances.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should not be logging at
ERRORlevel for every single record if we aren't failing the task unless the user has explicitly enabled this by settingerrors.log.enabletotruein their connector config.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's keep the existing
traceanderrorlog lines in theelseblock.My suggestion is to add a line at the debug or trace level in the
ifblock so users can know if an error is ignored.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My misunderstanding, thank you both for the feedback. Update made.