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

Greg Harris commented on KAFKA-17186:
-------------------------------------

[~yitian998] Relevant logs are here:

 
{noformat}
[2024-07-24 09:26:28,124] INFO [Worker clientId=B->A, groupId=B-mm2] Joined 
group at generation 87 with protocol version 2 and got assignment: 
Assignment{error=0, leader='B->A-73c5abfd-32cc-4699-ac31-db4f9d246170', 
leaderUrl='NOTUSED', offset=1376, connectorIds=[], taskIds=[], 
revokedConnectorIds=[], revokedTaskIds=[], delay=248995} with rebalance delay: 
248995 (org.apache.kafka.connect.runtime.distributed.DistributedHerder:2580)

[2024-07-24 09:26:28,698] INFO [Worker clientId=A->B, groupId=A-mm2] Joined 
group at generation 184 with protocol version 2 and got assignment: 
Assignment{error=0, leader='A->B-72f5fd41-a9fb-489b-aedd-e88f995322a8', 
leaderUrl='NOTUSED', offset=1369, connectorIds=[], taskIds=[], 
revokedConnectorIds=[], revokedTaskIds=[], delay=251371} with rebalance delay: 
251371 (org.apache.kafka.connect.runtime.distributed.DistributedHerder:2580)

[2024-07-24 09:26:28,703] DEBUG [Worker clientId=A->B, groupId=A-mm2] Scheduled 
rebalance at: 1721784640069 (now: 1721784388703 scheduledTick: 1721784640069)  
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:496)
[2024-07-24 09:30:37,119] DEBUG [Worker clientId=B->A, groupId=B-mm2] Handling 
completed but unresolved rebalance 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1725)
[2024-07-24 09:30:37,119] DEBUG [Worker clientId=B->A, groupId=B-mm2] 
Requesting rebalance because scheduled rebalance timeout has been reached (now: 
1721784637119 scheduledRebalance: 1721784637119 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1760)
[2024-07-24 09:30:37,129] INFO [Worker clientId=B->A, groupId=B-mm2] Joined 
group at generation 88 with protocol version 2 and got assignment: 
Assignment{error=0, leader='B->A-73c5abfd-32cc-4699-ac31-db4f9d246170', 
leaderUrl='NOTUSED', offset=1376, connectorIds=[MirrorSourceConnector], 
taskIds=[MirrorSourceConnector-0, MirrorSourceConnector-1, 
MirrorSourceConnector-2], revokedConnectorIds=[], revokedTaskIds=[], delay=0} 
with rebalance delay: 0 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:2580)
[2024-07-24 09:30:37,131] INFO [Worker clientId=B->A, groupId=B-mm2] Starting 
task MirrorSourceConnector-0 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1964)
[2024-07-24 09:30:37,131] INFO [Worker clientId=B->A, groupId=B-mm2] Starting 
task MirrorSourceConnector-1 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1964)
[2024-07-24 09:30:37,131] INFO [Worker clientId=B->A, groupId=B-mm2] Starting 
task MirrorSourceConnector-2 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1964)
[2024-07-24 09:30:37,131] INFO [Worker clientId=B->A, groupId=B-mm2] Starting 
connector MirrorSourceConnector 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:2039)

[2024-07-24 09:30:40,069] DEBUG [Worker clientId=A->B, groupId=A-mm2] 
Requesting rebalance because scheduled rebalance timeout has been reached (now: 
1721784640069 scheduledRebalance: 1721784640069 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1760)
[2024-07-24 09:30:40,073] INFO [Worker clientId=A->B, groupId=A-mm2] Joined 
group at generation 185 with protocol version 2 and got assignment: 
Assignment{error=0, leader='A->B-72f5fd41-a9fb-489b-aedd-e88f995322a8', 
leaderUrl='NOTUSED', offset=1369, connectorIds=[MirrorSourceConnector], 
taskIds=[MirrorSourceConnector-0, MirrorSourceConnector-1, 
MirrorSourceConnector-2], revokedConnectorIds=[], revokedTaskIds=[], delay=0} 
with rebalance delay: 0 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:2580)
[2024-07-24 09:30:40,074] INFO [Worker clientId=A->B, groupId=A-mm2] Starting 
connector MirrorSourceConnector 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:2039)
[2024-07-24 09:30:40,075] INFO [Worker clientId=A->B, groupId=A-mm2] Starting 
task MirrorSourceConnector-0 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1964)
[2024-07-24 09:30:40,075] INFO [Worker clientId=A->B, groupId=A-mm2] Starting 
task MirrorSourceConnector-1 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1964)
[2024-07-24 09:30:40,075] INFO [Worker clientId=A->B, groupId=A-mm2] Starting 
task MirrorSourceConnector-2 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:1964)

{noformat}
The cluster is identifying a lost worker (itself, i guess) and is intentionally 
delaying re-assigning tasks. You can see the "with rebalance delay" section in 
the "Joined Group" message start high, and then go to 0 in the subsequent 
rebalance. This delay is controlled by the scheduled.rebalance.max.delay 
configuration: 
[https://kafka.apache.org/documentation.html#connectconfigs_scheduled.rebalance.max.delay.ms]
 
By default it's 5 minutes, but you can lower this if workers typically restart 
much faster and you would like your tasks re-assigned more quickly. I think 
this single-worker situation is an edge case that may be undesirable to trigger 
the rebalance delay, but that may be difficult to change.

 

> Cannot receive message after stopping Source Mirror Maker 2
> -----------------------------------------------------------
>
>                 Key: KAFKA-17186
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17186
>             Project: Kafka
>          Issue Type: Bug
>          Components: mirrormaker
>    Affects Versions: 3.7.1
>         Environment: Source Kafka Cluster per Node:
> CPU(s): 32
> Memory: 32G/1.1G free
> Target Kafka Cluster standalone Node:
> CPU(s): 24
> Memory: 30G/909M free
> Kafka Version 3.7
> Mirrormaker Version 3.7.1
>            Reporter: George Yang
>            Priority: Major
>         Attachments: mirrorMaker.out
>
>
> Deploy nodes 1, 2, and 3 in Data Center A, with MM2 service deployed on node 
> 1. Deploy node 1 in Data Center B, with MM2 service also deployed on node 1. 
> Currently, a service on node 1 in Data Center A acts as a producer sending 
> messages to the `myTest` topic. A service in Data Center B acts as a consumer 
> listening to `A.myTest`. 
> The issue arises when MM2 on node 1 in Data Center A is stopped: the consumer 
> in Data Center B ceases to receive messages. Even after I restarting MM2 in 
> Data Center A, the consumer in Data Center B still does not receive messages 
> until approximately 5 minutes later when a rebalance occurs, at which point 
> it begins receiving messages again.
>  
> [Logs From Consumer on Data Center B]
> [2024-07-23 17:29:17,270] INFO [MirrorCheckpointConnector|worker] refreshing 
> consumer groups took 185 ms (org.apache.kafka.connect.mirror.Scheduler:95)
> [2024-07-23 17:29:19,189] INFO [MirrorCheckpointConnector|worker] refreshing 
> consumer groups took 365 ms (org.apache.kafka.connect.mirror.Scheduler:95)
> [2024-07-23 17:29:22,271] INFO [MirrorCheckpointConnector|worker] refreshing 
> consumer groups took 186 ms (org.apache.kafka.connect.mirror.Scheduler:95)
> [2024-07-23 17:29:24,193] INFO [MirrorCheckpointConnector|worker] refreshing 
> consumer groups took 369 ms (org.apache.kafka.connect.mirror.Scheduler:95)
> [2024-07-23 17:29:25,377] INFO [Worker clientId=B->A, groupId=B-mm2] 
> Rebalance started 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:242)
> [2024-07-23 17:29:25,377] INFO [Worker clientId=B->A, groupId=B-mm2] 
> (Re-)joining group 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:604)
> [2024-07-23 17:29:25,386] INFO [Worker clientId=B->A, groupId=B-mm2] 
> Successfully joined group with generation Generation\{generationId=52, 
> memberId='B->A-adc19038-a8b6-40fb-9bf6-249f866944ab', protocol='sessioned'} 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:665)
> [2024-07-23 17:29:25,390] INFO [Worker clientId=B->A, groupId=B-mm2] 
> Successfully synced group in generation Generation\{generationId=52, 
> memberId='B->A-adc19038-a8b6-40fb-9bf6-249f866944ab', protocol='sessioned'} 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:842)
> [2024-07-23 17:29:25,390] INFO [Worker clientId=B->A, groupId=B-mm2] Joined 
> group at generation 52 with protocol version 2 and got assignment: 
> Assignment\{error=0, leader='B->A-adc19038-a8b6-40fb-9bf6-249f866944ab', 
> leaderUrl='NOTUSED', offset=1360, connectorIds=[MirrorCheckpointConnector], 
> taskIds=[MirrorCheckpointConnector-0, MirrorCheckpointConnector-1, 
> MirrorCheckpointConnector-2], revokedConnectorIds=[], revokedTaskIds=[], 
> delay=0} with rebalance delay: 0 
> (org.apache.kafka.connect.runtime.distributed.DistributedHerder:2580)
> [2024-07-23 17:29:25,390] INFO [Worker clientId=B->A, groupId=B-mm2] Starting 
> connectors and tasks using config offset 1360 
> (org.apache.kafka.connect.runtime.distributed.DistributedHerder:1921)
> [2024-07-23 17:29:25,390] INFO [Worker clientId=B->A, groupId=B-mm2] Finished 
> starting connectors and tasks 
> (org.apache.kafka.connect.runtime.distributed.DistributedHerder:1950)
> [2024-07-23 17:29:26,883] INFO [Worker clientId=A->B, groupId=A-mm2] 
> Rebalance started 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:242)
> [2024-07-23 17:29:26,883] INFO [Worker clientId=A->B, groupId=A-mm2] 
> (Re-)joining group 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:604)
> [2024-07-23 17:29:26,890] INFO [Worker clientId=A->B, groupId=A-mm2] 
> Successfully joined group with generation Generation\{generationId=143, 
> memberId='A->B-0d04e6c1-f12a-4121-89af-e9992a167a01', protocol='sessioned'} 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:665)
> [2024-07-23 17:29:26,893] INFO [Worker clientId=A->B, groupId=A-mm2] 
> Successfully synced group in generation Generation\{generationId=143, 
> memberId='A->B-0d04e6c1-f12a-4121-89af-e9992a167a01', protocol='sessioned'} 
> (org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:842)
> [Configuration]
> name=MCS-MM2
> clusters = A, B
> A.bootstrap.servers = [kafka1]:[port],[kafka2]:[port],[kafka3]:[port]
> B.bootstrap.servers = [kafka]:[port]
> # enable and configure individual replication flows
> A->B.enabled = true
> A->B.topics = .*
> B->A.enabled = true
> B->A.topics = .*
> replication.factor=2
> tasks.max=3
> emit.checkpoints.interval.seconds=5
> A.producer.acks=all
> A.producer.batch.size=50000
> A.consumer.auto.offset.reset=latest
> B.consumer.auto.offset.reset=latest
> A.consumer.enable.auto.commit=true
> B.consumer.enable.auto.commit=true
> A.consumer.max.poll.interval.ms=20000
> B.consumer.max.poll.interval.ms=20000
> checkpoints.topic.replication.factor=1
> heartbeats.topic.replication.factor=1
> offset-syncs.topic.replication.factor=1
> offset.storage.replication.factor=1
> status.storage.replication.factor=1
> config.storage.replication.factor=1
> refresh.topics.enabled=true
> refresh.topics.interval.seconds=5
> refresh.groups.enabled=true
> refresh.groups.interval.seconds=5
>  



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

Reply via email to