How to setup Kafka cluster replication with MirrorMaker 2 - covering deployment model, connector tuning, topic drift gotchas, and a step-by-step failback playbook for Kafka disaster recovery.

Kafka cluster replication is a common requirement for disaster recovery, geographic distribution, and data migration. There are only a few options for that: MirrorMaker 2 (MM2), Brooklin, Confluent Cluster Linking, and more recently the emerging "Iceberg topics" pattern, which leverages Apache Iceberg as a storage layer so that cross-cluster access can happen via shared object storage rather than traditional topic mirroring. For most teams, MM2 is the practical path forward - it ships as part of the official Apache Kafka Connect distribution, requires no additional licensing, and is the community standard for Kafka replication.

This post covers the deployment model, the connector configuration choices that matter, and the production gotchas that tend to surface quietly over time. It closes with a step-by-step failback playbook - the piece most DR plans underspecify.

Deployment Model

We typically recommend deploying MM2 connectors on or near the remote (DR/passive) cluster, pulling data from the source. This placement means MM2's consumer group offsets and internal state are local to the target cluster, which simplifies reasoning about the system during a failover. MM2 itself runs as a set of Kafka Connect connectors; you can deploy it in dedicated Connect workers or use MM2's standalone mode.

A single MM2 source connector handles all matched topics by default, but splitting across multiple connectors makes sense when the source cluster mixes high-throughput and low-throughput topics. A lagging high-volume topic can starve replication of smaller topics - head-of-line blocking within a single connector. Separate connector instances with distinct topic allowlists (one for high-throughput ingestion topics, another for lower-volume event topics) keep replication progress independent across workloads. The right split depends on your traffic profile; a workload-by-workload analysis before deployment pays off.

MM2's MirrorCheckpointConnector handles offset translation between clusters. It periodically maps consumer group offsets from the source into equivalent offsets on the target, storing translations in an internal checkpoint topic. This is what allows consumers on the target cluster to resume from approximately where they left off during a failover. The translation is not perfectly exact - small gaps or duplicates are possible depending on replication lag at the moment of the switch. Consumer applications should be built for at-least-once semantics. MM2 does theoretically support exactly-once delivery through idempotent producers and transactional consumers, but this comes with significant performance overhead and operational complexity - most teams do not find the tradeoff worthwhile across a cluster boundary.

Connector Configuration and Tuning

Two configuration decisions have outsized impact on how painful failover and failback will be.

Replication policy is the first. The default DefaultReplicationPolicy prefixes replicated topics with the source alias (e.g., source.my-topic). This means consumers must be reconfigured after failover to read from the prefixed names, but it keeps the door open for active-active setups later. IdentityReplicationPolicy preserves original topic names, which simplifies the failover switch for consumers but creates infinite replication loops if both MM2 directions are ever active at the same time. Decide on this before going live; switching replication policies later requires topic renames and consumer reconfiguration across the board.

Client tuning is the second. MM2 is fundamentally just another set of Kafka producers and consumers. tasks.max needs to scale relative to partition count and throughput. Standard Kafka client properties - batch.size, linger.ms, fetch.min.bytes, receive.buffer.bytes, and compression settings - should be tuned based on your actual throughput and latency requirements, the same as for any other Kafka client. The defaults are rarely appropriate for high-throughput production workloads.

The Gotchas

Config Drift

sync.topic.configs.enabled is on by default, but the sync is periodic and not all configs propagate reliably across MM2 versions. Retention policies, cleanup policy, and min ISR on the target should be audited independently, not assumed to match the source. Running a scheduled monitoring job to detect config drift between source and target topics is standard practice.

Topic Recreation

If a topic is deleted and recreated on the source cluster - to change partition count, for example, or to clean up corrupted data - MM2 does not handle this cleanly. The mirrored topic on the target retains its old data and configuration. MM2 will begin replicating the new topic's messages, but offsets reset to zero on the source while the target topic retains its prior data, producing a scrambled sequence.

The correct recovery procedure:

  1. Stop the MM2 connector.
  2. Delete the mirrored topic on the target (e.g., source.my-topic with DefaultReplicationPolicy).
  3. Reset or clear the relevant MM2 internal offsets for that topic.
  4. Restart the connector, which triggers a clean re-sync.

Failback: Reversing Replication After a DR Event

Failback is one of the more operationally complex scenarios in any Kafka DR plan, and it is often the part that teams rehearse least. The general process involves confirming the primary cluster is healthy, setting up a temporary reverse replication from the DR cluster back to the primary, letting it converge, and then migrating traffic back. Rehearsing this in a non-production environment before you need it is not optional - the coordination between stopping producers, draining replication, and switching consumers is where mistakes happen.

Step 1 - Validate Primary Cluster Health

Before initiating failback, confirm that the primary cluster is actually ready. All broker pods should be running, KRaft controllers (or ZooKeeper, if still applicable) should have quorum, inter-broker replication should be healthy with no under-replicated partitions, and there should be sufficient disk, CPU, and memory headroom to absorb incoming replication traffic on top of production workloads.

Step 2 - Establish Reverse Replication

Deploy a temporary MM2 instance replicating from the DR cluster (Cluster B) back to the primary (Cluster A), using the same "deploy alongside target" pattern. Configure it to replicate all topics that received writes during the outage period.

By default, the reverse MM2 starts from the earliest available offset on Cluster B, which means it will re-replicate data Cluster A already had before the DR event. With IdentityReplicationPolicy this creates duplicates; with DefaultReplicationPolicy it copies full topic histories. To limit replication to just the DR-period delta, inspect the original MM2's offset-syncs topic to determine where forward replication stopped, then seed the reverse connector's starting offsets from that point. This offset math is operationally complex and getting it wrong risks missing data. Most teams accept the duplication and rely on idempotent consumers to absorb it. For topics in the hundreds of GBs, the offset seeding is worth doing; for modest-sized topics, letting the reverse MM2 replicate from the beginning and absorbing the convergence time is the safer call.

The MirrorCheckpointConnector must be active in the reverse direction as well. Consumers switching back to Cluster A need their offsets translated using the checkpoint data accumulated on Cluster B during the DR period.

Step 3 - Move Workloads Back to Primary

Once the reverse MM2 has caught up and Cluster A is confirmed healthy with current data, consumers move first, producers second. This sequence prevents data from being produced to Cluster A before consumers are ready, and allows consumers to drain any remaining lag from Cluster B before switching. Depending on the replication policy, topic names may need to change after the switch.

Once consumers are confirmed stable on Cluster A, coordinate a brief window where producers stop writing to Cluster B, the reverse MM2 lag drains to zero, and then producers are redirected to Cluster A. Any messages still in flight on the reverse MM2 that haven't landed on Cluster A represent potential data loss if the cutover happens too early - don't skip the drain step. Keep the window as short as possible, but do not abbreviate the drain.

After producers are writing to Cluster A, the reverse MM2 instance can be shut down. The original forward replication (Cluster A to B) resumes from that point.

Key Takeaways

  • Deploy MM2 alongside the target cluster; split connectors by throughput profile when you have a mix of high- and low-volume topics.
  • Choose DefaultReplicationPolicy or IdentityReplicationPolicy before going live. Changing it later is disruptive.
  • Do not rely solely on sync.topic.configs.enabled for config parity - run independent audits of retention, cleanup policy, and min ISR on the target.
  • Topic recreations on the source require manual intervention: stop the connector, delete the mirrored topic, reset MM2 offsets, restart.
  • Failback requires temporary reverse replication; consumers move before producers, and the reverse MirrorCheckpointConnector must be active throughout.
  • Drain replication lag to zero before producer cutover. Any remaining in-flight messages represent data loss if you cut too early.
  • Rehearse the full failback procedure in a non-production environment. The coordination steps are straightforward on paper and error-prone under pressure.