Damien Gasparina created KAFKA-13109:
----------------------------------------

             Summary: WorkerSourceTask is not enforcing the 
errors.retry.timeout and errors.retry.delay.max.ms parameters in case of a 
RetriableException during task.poll()
                 Key: KAFKA-13109
                 URL: https://issues.apache.org/jira/browse/KAFKA-13109
             Project: Kafka
          Issue Type: Bug
          Components: KafkaConnect
    Affects Versions: 2.8.0
            Reporter: Damien Gasparina


It seems that the {{errors.retry.timeout}} timeout is not enforced if 
{{RetriableException}} is thrown in the {{poll()}} of a SourceTask.

Looking at Kafka Connect source code:
 * If a task throws a {{RetriableException}} during a {{poll()}}, the connect 
runtime catch it and returns null: 
[https://github.com/apache/kafka/blob/2.8.0/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java#L273-L277]
 * Then, {{toSend}} is set to null, and the runtime continues the loop and 
re-execute the next iteration of poll without any delay 
[https://github.com/apache/kafka/blob/2.8.0/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java#L240-L246]

 

This implies that, if the {{poll()}} is throwing a {{RetriableException}}:
 * {{errors.retry.timeout}} is ignored and the task will retry indefinitely
 * there would be no delay between each retry, {{errors.retry.delay.max.ms}} is 
ignored, causing potential high resource utilization and log flooding

 

My understanding of 
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-298%3A+Error+Handling+in+Connect]
 is that {{errors.retry.timeout}} and {{errors.retry.delay.max.ms}} should have 
been respected in case of a {{RetriableException}} during a Source Task 
{{poll()}}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to