[ https://issues.apache.org/jira/browse/KAFKA-15262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17895920#comment-17895920 ]
Kanalas Vidor commented on KAFKA-15262: --------------------------------------- [~blacktooth] technically you are right, the MirrorHearbeatConnector is producing into the target topic. However, if you check [Code Ref,|https://github.com/apache/kafka/blob/ed44bcd71b3b9926c474033882eaa6c1cf35cfa4/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java#L127] by default (if global heartbeating is enabled), a herder is created for all cluster pairs, regardless whether there is a replication flow enabled for those two cluster pairs. In other words, the MirrorHearbeatConnector of the replication flow A->B will produce into B, however by the way MirrorMaker is working, there will be also a MirrorHearbeatConnector for a (possibly not even enabled) replication flow B->A which will produce into A. So the heartbeats topic on both A and B will be populated. > MirrorHeartbeatConnector is not working as documented > ----------------------------------------------------- > > Key: KAFKA-15262 > URL: https://issues.apache.org/jira/browse/KAFKA-15262 > Project: Kafka > Issue Type: Bug > Components: mirrormaker > Affects Versions: 2.8.0, 3.4.0, 3.5.0 > Reporter: Ravindranath Kakarla > Priority: Major > > As per the MM2 > [KIP-382|[https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0]] > the MirrorHeartbeatConnector should emit pings to heartbeat topic on the > source cluster which then gets replicated to the target cluster. This can be > used to demonstrate that MM2 is replicating the data. > However, this is not happening right now. To the contrary, the > MirrorHeartbeatConnector is producing heartbeat pings to target cluster > instead of source. This is not much useful as it won't help detect problems > connecting to source cluster or with the data replication. > Is my understanding of the MirrorHeartbeatConnector accurate? > *Reference:* > _MM2 emits a *heartbeat* *topic* in each source cluster, which is replicated > to demonstrate connectivity through the connectors. Downstream consumers can > use this topic to verify that a) the connector is running and b) the > corresponding source cluster is available. Heartbeats will get propagated by > source and sink connectors s.t. chains like backup.us-west.us-east.heartbeat > are possible._ > [Code > Ref|https://github.com/apache/kafka/blob/ed44bcd71b3b9926c474033882eaa6c1cf35cfa4/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java#L65] > -- This message was sent by Atlassian Jira (v8.20.10#820010)