KAFKA-14732: Use an exponential backoff retry mechanism while reconfiguring connector tasks#13276
KAFKA-14732: Use an exponential backoff retry mechanism while reconfiguring connector tasks#13276C0urante merged 3 commits intoapache:trunkfrom
Conversation
mukkachaitanya
left a comment
There was a problem hiding this comment.
Thanks, @yashmayya! The intent seems right. Had a couple of comments/suggestions
There was a problem hiding this comment.
I am curious if there is a way to not do infinite retries. If we are actually retrying infinitely, esp in the case of startConnector phase, then the connector just doesn't have tasks. Is it possible to somehow bubble up errors as part of connector (not task) status?
There was a problem hiding this comment.
Hm, that's an interesting idea and I don't see the harm in limiting the number of retries to some reasonable value and then marking the connector as failed after that (we could include the last exception's trace in the connector's status).
Is it possible to somehow bubble up errors as part of connector (not task) status?
The AbstractHerder (DistributedHerder's parent class) implements the ConnectorStatus.Listener interface and so we should be able to update the connector's status to failed via
For instance, here we use the onFailure hook to update a connector's status as failed if there is an exception thrown during startup.
C0urante
left a comment
There was a problem hiding this comment.
Thanks Yash, this looks great.
RE non-infinite retries: probably a good idea, needs a KIP though IMO.
RE Javadoc comment updates: a separate PR would be great. Unless they're required by a functional change, they should be separated, which makes it easier to review and merge.
There was a problem hiding this comment.
This test is great. I think it'd be worth it to perform a third and fourth tick. The third can be used to simulate successfully generating task configs after the two failed attempts, and the fourth can be used to ensure that we don't retry any further.
It's also worth noting that we're only testing the case where Connector::taskConfigs (or really, Worker::connectorTaskConfigs) fails, but the logic that's being added here applies if intra-cluster communication fails as well (which may happen if the leader of the cluster is temporarily unavailable, for example). It'd be nice if we could have test coverage for that too, but I won't block this PR on that.
There was a problem hiding this comment.
Thanks, I've updated the test to add another herder tick which runs a successful task reconfiguration request (I skipped the addition of another tick because the no further retries bit can be verified by the poll timeout at the end of the previous tick).
Regarding the test case for the task reconfiguration REST request to the leader - I did consider that initially but while trying to add one, there were some complications (timing related issues) arising from the use of the forwardRequestExecutor at which point I felt like it was more trouble than it was worth. However, your comment made me revisit it and I've made some changes to drop in a simple mock executor service which runs requests synchronously (on the same thread as the caller). Let me know what you think?
There was a problem hiding this comment.
Ah, fair point about the fourth tick!
I don't love using a synchronous executor here since it diverges significantly from the non-testing behavior of the herder. But, I can't think of a better way to test this without going overboard in complexity, and it does give us decent coverage.
So, good enough 👍
388eae4 to
f53d0ad
Compare
…check extra herder tick after task configs generated successfully; Add testTaskReconfigurationRetriesWithLeaderRequestForwardingException; Revert Javadoc changes to ExponentialBackoff
f53d0ad to
a75630b
Compare
250 ms) in case of errors arising during connector task reconfiguration.DistributedHerder::reconfigureConnectorTasksWithRetrymethod. The initial retry backoff is retained as250 mswith a chosen maximum backoff of60000 ms.Committer Checklist (excluded from commit message)