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

Attila Sasvari commented on KAFKA-6703:
---------------------------------------

There is a call to 
[ensureCoordinatorReady()|https://github.com/apache/kafka/blob/f0a29a693548efe539cba04807e21862c8dfc1bf/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L279]
 in {{ConsumerCoordinator}}. It tries to poll the coordinator in a 
[loop|https://github.com/apache/kafka/blob/f0a29a693548efe539cba04807e21862c8dfc1bf/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java#L217-L219].
 It does not succeed, so it will retry the connection, and other matched topics 
are ignored until the failing coordinator becomes healthy.


> MirrorMaker cannot make progress when any matched topic from a whitelist 
> regexp has -1 leader
> ---------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-6703
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6703
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 1.1.0
>            Reporter: Attila Sasvari
>            Priority: Major
>
> Scenario:
>  - MM whitelabel regexp matches multiple topics
>  - destination cluster has 5 brokers with multiple topics replication factor 3
>  - without partition reassign shut down 2 brokers
>  - suppose a topic has no leader any more because it was off-sync and the 
> leader and the rest of the replicas are hosted on the downed brokers.
>  - so we have 1 topic with some partitions with leader -1
>  - the rest of the matching topics has 3 replicas with leaders
> MM will not produce into any of the matched topics until:
>  - the "orphaned" topic removed or
>  - the partition reassign carried out from the downed brokers (suppose you 
> can turn these back on)
> In the MirrorMaker logs, there are a lot of messages like the following ones:
> {code}
> [2018-03-22 18:59:07,781] DEBUG [Consumer clientId=1-1, groupId=1] Sending 
> FindCoordinator request to broker 192.168.1.102:9092 (id: 0 rack: null) 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,781] DEBUG [Consumer clientId=1-0, groupId=1] Sending 
> FindCoordinator request to broker 192.168.1.102:9092 (id: 0 rack: null) 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,783] DEBUG [Consumer clientId=1-0, groupId=1] Received 
> FindCoordinator response ClientResponse(receivedTimeMs=1521741547782, 
> latencyMs=1, disconnected=false, 
> requestHeader=RequestHeader(apiKey=FIND_COORDINATOR, apiVersion=1, 
> clientId=1-0, correlationId=71), 
> responseBody=FindCoordinatorResponse(throttleTimeMs=0, errorMessage='null', 
> error=COORDINATOR_NOT_AVAILABLE, node=:-1 (id: -1 rack: null))) 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,783] DEBUG [Consumer clientId=1-1, groupId=1] Received 
> FindCoordinator response ClientResponse(receivedTimeMs=1521741547782, 
> latencyMs=1, disconnected=false, 
> requestHeader=RequestHeader(apiKey=FIND_COORDINATOR, apiVersion=1, 
> clientId=1-1, correlationId=71), 
> responseBody=FindCoordinatorResponse(throttleTimeMs=0, errorMessage='null', 
> error=COORDINATOR_NOT_AVAILABLE, node=:-1 (id: -1 rack: null))) 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,783] DEBUG [Consumer clientId=1-0, groupId=1] Group 
> coordinator lookup failed: The coordinator is not available. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,783] DEBUG [Consumer clientId=1-1, groupId=1] Group 
> coordinator lookup failed: The coordinator is not available. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,783] DEBUG [Consumer clientId=1-0, groupId=1] 
> Coordinator discovery failed, refreshing metadata 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2018-03-22 18:59:07,783] DEBUG [Consumer clientId=1-1, groupId=1] 
> Coordinator discovery failed, refreshing metadata 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> {code}
> Interestingly, if MirrorMaker uses {{zookeeper.connect}} in its consumer 
> properties file, then an OldConsumer is created, and it can make progress.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to