Hi Rajini, RS13: Updated.
RS14: Updated. RS15: Agreed. Renamed to mirror.state.topic.num.partitions and mirror.state.topic.replication.factor. RS16: The destination is treated as a full replica of the source's ACL state. There is no concept of "destination‑only" ACLs (except for CLUSTER_MIRROR resource type ACLs, which are explicitly skipped). A DENY ACL created directly on the destination would be wiped on the next sync cycle. That said, you can always use mirror.acl.include configuration to alter this behavior. RS17: Correct, updated. Thanks Fede On Mon, May 18, 2026 at 9:23 PM Rajini Sivaram <[email protected]> wrote: > > Hi Federico, > > Thanks for the updates! The KIP is looking good. A few more small comments. > > > RS13: A couple of places still refer to `kafka-mirror.sh` like under > `Failover Process`. Can we change them to `*kafka-cluster-mirrors.sh*`? > > RS14: Should we change `--entity-type mirrors` for kafka-configs to be ` > --entity-type *cluster-mirrors*` to be consistent? Also, CLUSTER_MIRROR((byte) > 64, "mirror"); could be `*cluster-mirror*`? > > RS15: It may be useful to rename `mirror.topic.num.partitions` and ` > mirror.topic.replication.factor` since they are very similar to ` > mirror.topic.properties.exclude`, but the `mirror.topic` prefix refers to > different topics (the internal topic for the first two and actual mirror > topics for the other one). > > RS16: ACL Sync: KIP says "Deletes ACLs that exist in destination but not in > source using DeleteAcls request." > What happens if someone creates an ACL on the destination to deny > User:Alice access to all topics? > > 1. If that ACL also existed on the source cluster and then it was > removed, will it get removed from the destination? > 2. If that ACL never existed on the source cluster, will it get removed > from the destination? > > RS17: The table in the Source ACLs section says: "DescribeClusterMirrors > MC Read Cluster Log truncation" > Should that be `ClusterMirror:Read` instead of `Cluster:Read`? > > Regards, > > Rajini > > > > On Fri, May 15, 2026 at 8:41 AM Federico Valeri <[email protected]> > wrote: > > > Hi Rajini, we finally addressed the API and tool naming refactoring as > > you suggested in RS6. Please take a look when you have time. Thanks. > > > > > > On Mon, May 11, 2026 at 6:17 PM Federico Valeri <[email protected]> > > wrote: > > > > > > Hello all, I want to highlight a couple of new paragraphs: > > > > > > 1. Leader Epoch Invariant: Cluster mirroring enforces the invariant > > > that the destination leader epoch must always be greater than or equal > > > to the source leader epoch (DLE>=SLE). Without this, consumers on the > > > destination cluster can get stuck in an infinite metadata refresh loop > > > when they encounter committed offsets carrying source epochs higher > > > than the local epoch. The invariant is maintained through three > > > mechanisms: reactive bumping (epoch fencing triggered when SLE > DLE > > > during fetch), proactive bumping (scheduled when SLE approaches DLE > > > within a threshold), and periodic bumping (checked during coordinator > > > metadata sync). > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=406620973#KIP1279:ClusterMirroring-LeaderEpochInvariant > > > > > > 2. Group Offsets: The coordinator periodically syncs consumer and > > > share group offsets from the source cluster to the destination for all > > > mirrored topics. Groups are filtered by configurable include/exclude > > > patterns, and offsets are only synced for groups that are not > > > currently active on the destination cluster, preventing overwrites of > > > local consumer progress. Because source and destination share the same > > > topic offsets (no offset translation), synced offsets can be used > > > directly without mapping. > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=406620973#KIP1279:ClusterMirroring-GroupOffsets > > > > > > These new paragraphs directly address some of your questions, but let > > > me list them here: > > > > > > JR2: Yes, we removed the incorrect phrase and added more details to > > > the paragraph. > > > > > > JR4: When source cluster topic has tiered storage enabled, CM works by > > > mirroring remote and local log into destination cluster. When > > > destination cluster topic has tiered storage enabled, CM fails in > > > PREPARING state because the LME may be in remote storage, but works > > > fine if already MIRRORING because no truncation is needed. > > > > > > JR11: See "Leader Epoch Invariant" paragraph mentioned above. > > > > > > JR13: We can't support stateful Streams application because > > > asynchronous replication cannot preserve the transactional boundaries > > > between input offset commits, state store mutations written to > > > changelog topics, and intermediate records written to repartition > > > topics. The synchronous extension of this design will be able to > > > support them. Existing Features Integration paragraph updated. > > > > > > JR18: See "Group Offsets" paragraph mentioned above. > > > > > > IY1: See "Group Offsets" paragraph mentioned above. > > > > > > Thanks > > > Fede > > > > > > On Mon, May 11, 2026 at 6:08 PM Federico Valeri <[email protected]> > > wrote: > > > > > > > > Hi Vaquar, > > > > > > > > VK4/VK8: We don't do PID mapping anymore. The KIP was updated some > > > > time ago with the new approach based on the new PID reset control > > > > record. > > > > > > > > VK11: The transaction index is always built locally during log append, > > > > never copied. > > > > > > > > VK1: The 50,000 * 1MB = 50GB calculation misunderstands the fetch > > > > model. Fetcher threads don't allocate one buffer per partition. Actual > > > > peak memory is roughly num_fetcher_threads * response_max_bytes, not > > > > num_partitions * partition_max_bytes. With 1 fetcher thread and the > > > > default response max, the memory footprint is modest regardless of > > > > partition count. We are leveraging the same proven pattern used by the > > > > internal replication. > > > > > > > > VK3: The __mirror_state topic uses hash-based partitioning based on > > > > mirrorName, topicId and partition number. With the production default > > > > of 50 partitions, 50,000 partition transitions distribute across ~50 > > > > partition leaders on different brokers, not a single broker. This is > > > > the same proven pattern as __consumer_offsets, which handles millions > > > > of commits. > > > > > > > > VK12: The scenario described can only occur if offsets are > > > > force-written to an active group, which the design prevents. > > > > > > > > Cheers > > > > Fede > >
