vamossagar12 commented on code in PR #13726:
URL: https://github.com/apache/kafka/pull/13726#discussion_r1218190316


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java:
##########
@@ -458,13 +458,8 @@ boolean sendRecords() {
     }
 
     protected List<SourceRecord> poll() throws InterruptedException {
-        try {
-            return task.poll();
-        } catch (RetriableException | 
org.apache.kafka.common.errors.RetriableException e) {
-            log.warn("{} failed to poll records from SourceTask. Will retry 
operation.", this, e);
-            // Do nothing. Let the framework poll whenever it's ready.
-            return null;
-        }
+        retryWithToleranceOperator.reset();
+        return retryWithToleranceOperator.execute(task::poll, Stage.TASK_POLL, 
this.getClass());

Review Comment:
   Coming back to this, this is the 
[method](https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperator.java#L173)
 which gets invoked eventually for any retry operations . And in this case, the 
only way to get out of a string of loop of RetriableExceptions is for the 
error.tolerance to be set to 0.
   
   I don't see how having error.tolerance set to none impacts in this case. 
This holds true for other retry operations as well like headers converters etc.
   Also, if you change the tolerance type to NONE in this method 
https://github.com/apache/kafka/blob/trunk/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java#L227
 and run any one of testExecute** tests that invoke those methods, the tests 
pass which is in line with what I said above.
   
   I appears to me that the none and all behaviour is applicable only for non 
retriable exceptions. RetriableExceptions would be retried none the less. 
Should we improve the documentation for this?



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

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to