Edward Vaisman created KAFKA-12664:
--------------------------------------
Summary: Mirrormaker 2.0 infinate rebalance loop when dealing with
more than 2 clusters in standalone mode
Key: KAFKA-12664
URL: https://issues.apache.org/jira/browse/KAFKA-12664
Project: Kafka
Issue Type: Bug
Components: mirrormaker
Affects Versions: 2.7.0, 2.6.0, 2.4.1, 2.5.0
Reporter: Edward Vaisman
Attachments: connect.log.tar.gz, docker-compose-multi.yml,
mm2.properties
Hi Folks, I came across this issue when trying to aggregate data from two
separate data centres into one data centre.
In the configuration below, you can see I am trying to replicate a topic from
dc1 (named test_topic_dc1) to dc3 as well as replicate a topic from dc2
(test_topic_dc2) to dc3.
However, when I try to replicate both topics from those datacenters at the same
time i notice that connect gets stuck in a rebalance loop (see attachment for
logs)
[^connect.log.tar.gz]
exerpt of connect.log
{code:java}
2021-04-13 17:03:06,360] INFO [Worker clientId=connect-3, groupId=mm2-dc2]
Successfully synced group in generation Generation{generationId=347,
memberId='connect-3-c59342c3-ca62-41cc-964c-41a0f98351c0',
protocol='sessioned'}
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator:756)[2021-04-13
17:03:06,360] INFO [Worker clientId=connect-4, groupId=mm2-dc2] Rebalance
started
(org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:225)[2021-04-13
17:03:06,362] INFO [Worker clientId=connect-4, groupId=mm2-dc2] (Re-)joining
group
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator:540)[2021-04-13
17:03:06,368] INFO [Worker clientId=connect-2, groupId=mm2-dc3] Rebalance
started
(org.apache.kafka.connect.runtime.distributed.WorkerCoordinator:225)[2021-04-13
17:03:06,369] INFO [Worker clientId=connect-2, groupId=mm2-dc3] (Re-)joining
group
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator:540)[2021-04-13
17:03:06,370] INFO [Worker clientId=connect-3, groupId=mm2-dc2] Joined group
at generation 347 with protocol version 2 and got assignment:
Assignment{error=1, leader='connect-3-c59342c3-ca62-41cc-964c-41a0f98351c0',
leaderUrl='NOTUSED/dc1', offset=13, connectorIds=[MirrorSourceConnector],
taskIds=[], revokedConnectorIds=[], revokedTaskIds=[], delay=0} with rebalance
delay: 0 (org.apache.kafka.connect.runtime.distributed.DistributedHerder:1688)
{code}
To replicate the issue here is what I used:
[^mm2.properties]
{code:java}
clusters = dc1, dc2, dc3
dc1.bootstrap.servers = kafka-dc1:19092
dc2.bootstrap.servers = kafka-dc2:19093
dc3.bootstrap.servers = kafka-dc3:19094
dc1.group.id=mm2-dc1
dc2.group.id=mm2-dc2
dc3.group.id=mm2-dc3
replication.factor=1
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
dc1->dc3.enabled = true
dc1->dc3.topics = test_topic_dc1
dc2->dc3.enabled = true
dc2->dc3.topics = test_topic_dc2
dc3->dc2 = falsedc3->dc1 = false
{code}
This [^docker-compose-multi.yml] file to create local kafka clusters
(dc1,dc2,dc3)
(I set docker to use 6 cpus, 8gb mem, swap 2gb)
I then ran an interactive shell to run mirror maker within the same
docker-compose network (change network to match yours)
{code:java}
docker run --network kafka-examples_default -it wurstmeister/kafka:latest bash
# Upload mm2 properties on server
/opt/kafka/bin/connect-mirror-maker.sh mm2.properties{code}
Kafkacat commands to produce to dc1, dc2
{code:java}
kafkacat -b localhost:9092 -t test_topic_dc1 -P
Hello World from DC1!{code}
{code:java}
kafkacat -b localhost:9093 -t test_topic_dc2 -P
Hello World from DC2{code}
I then tried to remove one of the datacenters to confirm if it was a
configuration problem, however mirror maker ran successfully with the below
configuration
mm2.properties
{code:java}
clusters = dc2, dc3
dc2.bootstrap.servers = kafka-dc2:19093
dc3.bootstrap.servers = kafka-dc3:19094
dc2.group.id=mm2-dc2
dc3.group.id=mm2-dc3
replication.factor=1
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
dc2->dc3.enabled = true
dc2->dc3.topics = test_topic_dc2
{code}
Any help would be appreciated!
--
This message was sent by Atlassian Jira
(v8.3.4#803005)