Josh, if you have two clusters with bidirectional replication, you only get two copies of each record. MM2 won't replicate the data "upstream", cuz it knows it's already there. In particular, MM2 knows not to create topics like B.A.topic1 on cluster A, as this would be an unnecessary cycle.
> is there a reason for MM2 not emitting checkpoint data for the source topic AND the remote topic No, not really! I think it would be surprising if one-directional flows insisted on writing checkpoints both ways -- but it's also surprising that you need to explicitly allow a remote topic to be checkpointed. I'd support changing this, fwiw. Ryanne On Wed, Aug 19, 2020 at 2:30 PM Josh C <josh.fl.c...@gmail.com> wrote: > Sorry, correction -- I am realizing now it would be 3 copies of the same > topic data as A.topic1 has different data than B.topic1. However, that > would still be 3 copies as opposed to just 2 with something like topic1 and > A.topic1. > > As well, if I were to explicitly replicate the remote topic back to the > source cluster by adding it to the topic whitelist, would I also need to > update the topic blacklist and remove ".*\.replica" (since the blacklists > take precedence over the whitelists)? > > Josh > > On Wed, Aug 19, 2020 at 11:46 AM Josh C <josh.fl.c...@gmail.com> wrote: > > > Thanks for the clarification Ryanne. In the context of active/active > > clusters, does this mean there would be 6 copies of the same topic data? > > > > A topics: > > - topic1 > > - B.topic1 > > - B.A.topic1 > > > > B topics: > > - topic1 > > - A.topic1 > > - A.B.topic1 > > > > Out of curiosity, is there a reason for MM2 not emitting checkpoint data > > for the source topic AND the remote topic as a pair as opposed to having > to > > explicitly replicate the remote topic back to the source cluster just to > > have the checkpoints emitted upstream? > > > > Josh > > > > On Wed, Aug 19, 2020 at 6:16 AM Ryanne Dolan <ryannedo...@gmail.com> > > wrote: > > > >> Josh, yes it's possible to migrate the consumer group back to the source > >> topic, but you need to explicitly replicate the remote topic back to the > >> source cluster -- otherwise no checkpoints will flow "upstream": > >> > >> A->B.topics=test1 > >> B->A.topics=A.test1 > >> > >> After the first checkpoint is emitted upstream, > >> RemoteClusterUtils.translateOffsets() will translate B's A.test1 offsets > >> into A's test1 offsets for you. > >> > >> Ryanne > >> > >> On Tue, Aug 18, 2020 at 5:56 PM Josh C <josh.fl.c...@gmail.com> wrote: > >> > >> > Hi there, > >> > > >> > I'm currently exploring MM2 and having some trouble with the > >> > RemoteClusterUtils.translateOffsets() method. I have been successful > in > >> > migrating a consumer group from the source cluster to the target > >> cluster, > >> > but was wondering how I could migrate this consumer group back to the > >> > original source topic? > >> > > >> > It is my understanding that there isn't any checkpoint data being > >> > emitted for this consumer group since it is consuming from a mirrored > >> topic > >> > in the target cluster. I'm currently getting an empty map since there > >> isn't > >> > any checkpoint data for 'target.checkpoints.internal' in the source > >> > cluster. So, I was wondering how would I get these new translated > >> offsets > >> > to migrate the consumer group back to the source cluster? > >> > > >> > Please let me know if my question was unclear or if you require > further > >> > clarification! Appreciate the help. > >> > > >> > Thanks, > >> > Josh > >> > > >> > > >