[ 
https://issues.apache.org/jira/browse/KAFKA-17232?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17898035#comment-17898035
 ] 

Greg Harris commented on KAFKA-17232:
-------------------------------------

Hi [~kakhramanov] thanks for the bug report!

That error appearing multiple times is the intended behavior of the patch; It 
should eventually resolve once the initial load of consumer groups finishes, 
and this log message is printed: 
[https://github.com/apache/kafka/blob/b6b2c9ebc45bd60572c24355886620dbdc406ce9/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java#L217]
 
Are you seeing the task configuration error appearing continuously without ever 
resolving? Are you seeing the log messages indicating loading has finished?

Also, are you seeing any Scheduler logs mentioning `loading initial consumer 
groups`? That would tell you if the load was actually timing out. The 
RetriableException infers that the load operation timed out, but perhaps there 
is a mistake that causes the exception to be thrown even if the timeout has not 
elapsed yet.

I also wonder if these log messages could be coming from cancelled tasks by 
accident.

> MirrorCheckpointConnector does not generate task configs if initial consumer 
> group load times out
> -------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-17232
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17232
>             Project: Kafka
>          Issue Type: Bug
>          Components: mirrormaker
>    Affects Versions: 3.9.0
>            Reporter: Greg Harris
>            Assignee: TengYao Chi
>            Priority: Major
>             Fix For: 3.9.0
>
>
> The MirrorCheckpointConnector has two operations that read the source 
> consumer groups:
>  * loadInitialConsumerGroups
>  * refreshConsumerGroups
> loadInitialConsumerGroups blocks the start() method of the connector, while 
> refreshConsumerGroups is asynchronous and runs periodically while the 
> connector is running.
> loadInitialConsumerGroups may take a long time to execute, and may exceed the 
> configured "admin.timeout.ms" used by the Scheduler. This timeout is logged 
> and the start() method returns normally. If this happens, the framework will 
> generate task configs immediately after start(), before 
> loadInitialConsumerGroups can finish, and will generate an empty set of task 
> configs: 
> [https://github.com/apache/kafka/blob/e2494e6ffb89f8288ed2aeb9b5596c755210bffd/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java#L118-L121].
> Later, when loadInitialConsumerGroups completes, it will not request task 
> reconfiguration, believing it is the initial load operation.
> Later still, when refreshConsumerGroups completes, it will not request task 
> reconfiguration, as the set of consumer groups has not changed since the 
> initial load: 
> [https://github.com/apache/kafka/blob/e2494e6ffb89f8288ed2aeb9b5596c755210bffd/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java#L173-L180]
>  
> This leads to a situation where the MirrorCheckpointConnector believes it has 
> converged with nothing to update, but actually has consumer groups that are 
> not allocated to tasks.
> This happens particularly for large, stable Kafka clusters with many consumer 
> groups that are not being actively created or deleted.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to