lianetm commented on code in PR #16826:
URL: https://github.com/apache/kafka/pull/16826#discussion_r1711936398
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1460,10 +1460,14 @@ public void handle(OffsetFetchResponse response,
RequestFuture<Map<TopicPartitio
if (responseError == Errors.COORDINATOR_LOAD_IN_PROGRESS) {
// just retry
future.raise(responseError);
- } else if (responseError == Errors.NOT_COORDINATOR) {
+ } else if (responseError == Errors.COORDINATOR_NOT_AVAILABLE ||
+ responseError == Errors.NOT_COORDINATOR) {
// re-discover the coordinator and retry
markCoordinatorUnknown(responseError);
future.raise(responseError);
+ } else if (responseError.exception() instanceof
RetriableException) {
+ // retry
+ future.raise(responseError);
Review Comment:
This will introduce a change in behaviour at the API level for some
retriable errors (the ones that were not being handled before)
Basically, for retriable errors other than COORDINATOR_LOAD_IN_PROGRESS and
NOT_COORDINATOR, in the case that they never recover in time, these 2 api calls
change a bit in behaviour:
1. committed: will throw a `TimeoutException`, vs the `KafkaException` it
used to throw before this PR (imo this aligns with the API contract, that
states KafkaException for unrecoverable and Timeout if it cannot complete in
time)
2. poll: will simply keep retrying and end up returning empty records, vs.
throwing a `KafkaException` wrapping the retriable as it used to do before this
PR (also aligns with the contract, that states KafkaException for
unrecoverable, and "If the timeout expires, an empty record set will be
returned"
It's definitely a change in behaviour that will be noticed if those
retriable errors happen, but both changes actually look like fixes to me, that
align with the current API contract for both calls, and are just handling all
retriables in the same way. Thoughts?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]