Hey everyone, Bumping this topic to see if anyone has experience migrating Streams jobs between clusters with Mirror Maker, and how you handled repartition topics.
Thanks! Greg On Mon, Aug 12, 2024 at 8:55 AM Greg Harris <greg.har...@aiven.io> wrote: > Hi all, > > We were recently working to set up a MirrorMaker2 flow to migrate a > Streams job between an existing Kafka cluster and a new Kafka cluster. We > noticed the following behavior: > > * Streams sets infinite retention on the repartition topic, and uses the > AdminClient deleteRecords call to implement "active retention" by deleting > data that is no longer necessary. > * MirrorMaker2 mirrors the data to topic with infinite retention, but > _without_ the active AdminClient deleteRecords calls running, as it is not > known to be a repartition topic and the destination job is not yet active. > * The target topic grows without bound and exhausts disks on the > destination. > > Could any Streams folks give their recommendations for this situation? > > 1. Should repartition topics have only streams-managed consumers, is > inspecting the repartition topic intended to be undefined behavior? > 2. If Streams deletes some records before MM2 mirrors them, could that > "lossy" replication of the repartition topic cause corruption or data loss? > 3. If someone truncates the repartition topic or MM2 doesn't replicate it, > are at-least-once and/or exactly-once semantics preserved for the Streams > job? > > And the more general question I'm trying to answer is: should MM2 have an > option to explicitly "sync" the deleteRecords calls across the cluster > boundaries? If "active retention" is a common pattern that just happens to > show up in the repartition topic, maybe it would be desirable to try and > sync the retention behaviors, and not just the retention properties. > > Thanks, > Greg >