question: mirror maker 2

2024-01-31 Thread M. Lim
Hello all, I am setting up mm2 to replicate messages, consumer groups, and consumer offset from a->b. I believe I am replicating those 3 items from a->b. my mm2 prop file is as followed: ``` # specify any number of cluster aliases clusters = a,b b.group.id=mm2-request # replication settings ta

Mirror Maker 2 | 2.7.1 | Some partitions are not synched since target cluster patching

2024-01-13 Thread Akshaya Rawat
Hi Team, We are using MM2 v2.7.1 to replicate Kafka. We noticed that the topic replication was working fine. However due to security patching the target cluster was restarted and since then some of the partitions are not synched. Can you help us identify the issue ? Regards, Akshaya

Re: Mirror Maker bidirectional offset sync

2024-01-12 Thread Greg Harris
t; fails back to the primary cluster. This consumer will then consume > > > > > messages > > > > > > from the prefixed topic in the secondary cluster, and I'd like to > > > have > > > > > > those offsets replicated back to the non-prefixed

Re: Mirror Maker bidirectional offset sync

2024-01-12 Thread Greg Harris
> > coupling between producers and consumers so we're able to test > > failovers > > > > > for individual applications without the need for all > > producers/consumers > > > > to > > > > > failover and failback at once. > > &g

Re: Mirror Maker bidirectional offset sync

2024-01-12 Thread Jeroen Schutrup
eam(replication,a.replicate-me-0,25): Skipped > > > (offset > > > > sync not found) (org.apache.kafka.connect.mirror.OffsetSyncStore) > > > > > > > > I wrote a small program to produce these offset syncs for the > prefixed > > > > topic, and t

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Ryanne Dolan
prefixed topic not being pushed to mm2-offset-syncs.b.internal is > likely > > > > the reason the checkpoint connector doesn't replicate consumer > offsets: > > > > DEBUG translateDownstream(replication,a.replicate-me-0,25): Skipped > > > (offset > >

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Greg Harris
gt; > OffsetSync{topicPartition=replicate-me-0, upstreamOffset=28, > > > downstreamOffset=28} > > > OffsetSync{topicPartition=replicate-me-0, upstreamOffset=29, > > > downstreamOffset=29} > > > OffsetSync{topicPartition=a.replicate-me-0, upstreamOffset=29, &g

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Jeroen Schutrup
; of > > whether this is a wise thing to do and if it would have any negative side > > effects. I'd need to spend some more time in the MM2 source, though I > > welcome any feedback on this hack :-) > > > > On the two complications you're mentioning Greg, the seco

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Greg Harris
downstreamOffset=29} > > > OffsetSync{topicPartition=a.replicate-me-0, upstreamOffset=29, > > > downstreamOffset=29} <-- the artificially generated offset-sync > > > > > > At this point it goes a bit beyond my understanding of the MM2 internals > > of > &

Re: Mirror Maker bidirectional offset sync

2024-01-10 Thread Ryanne Dolan
e > > effects. I'd need to spend some more time in the MM2 source, though I > > welcome any feedback on this hack :-) > > > > On the two complications you're mentioning Greg, the second one is > > something we should figure out regardless, as any given

Re: Mirror Maker bidirectional offset sync

2024-01-10 Thread Greg Harris
cations you're mentioning Greg, the second one is > something we should figure out regardless, as any given consumer group may > not be active on both the primary and secondary cluster as it would block > MM2 from replicating its offsets from primary to the cluster-prefixed topic >

Re: Mirror Maker bidirectional offset sync

2024-01-09 Thread Jeroen Schutrup
eady. On the first point, I think it would be a good practice to only allow MM2 to produce to any cluster-prefixed topic by using topic ACLs. In other words, the only application producing to a cluster-prefixed (or downstream) topic would be mirror maker and I think that prevents this kind of messag

Re: Mirror Maker bidirectional offset sync

2024-01-08 Thread Ryanne Dolan
Jeroen, MirrorClient will correctly translate offsets for both failover and failback, exactly as you describe. It's possible to automate failover and failback using that logic. The integration tests automatically fail over and fail back, for example. I've seen it done two ways: during startup withi

Re: Mirror Maker bidirectional offset sync

2024-01-08 Thread Greg Harris
Hi Jeroen, Thanks for looking into MM2 for your use-case! I believe the "active-active" model that MM2 uses is really more of a pair of opposing "active-passive" replication flows, and this extends to the offset sync mechanism too. MM2 doesn't change the state of the source topic or consumer grou

Mirror Maker bidirectional offset sync

2024-01-08 Thread Jeroen Schutrup
Hi all, I'm exploring using the MirrorSourceConnector and MirrorCheckpointConnector on Kafka Connect to setup active/active replication between two Kafka clusters. Using the DefaultReplicationPolicy replication policy class, messages originating from the source cluster get replicated as expected to

Re: Mirror Maker 2 - offset sync from source to target

2023-10-24 Thread Greg Harris
ocumentation explaining the same ? > I tried to find a lot in the past > > > From: Greg Harris > Sent: Monday, October 23, 2023 11:23:26 PM > To: users@kafka.apache.org > Subject: Re: Mirror Maker 2 - offset sync from source to tar

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Alexander Shapiro (ashapiro)
Not a problem Greg. Is there some documentation explaining the same ? I tried to find a lot in the past From: Greg Harris Sent: Monday, October 23, 2023 11:23:26 PM To: users@kafka.apache.org Subject: Re: Mirror Maker 2 - offset sync from source to target [You

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread sherry
larification plz > > > > > > In bullet for You mention "4. The target group does not exist, or has > no active consumers" > > > If group on target does not exist, will it be created without active > consumers ? > > > > > > -Original Mess

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
es not exist, will it be created without active > > consumers ? > > > > -Original Message- > > From: Greg Harris > > Sent: Monday, October 23, 2023 8:56 PM > > To: users@kafka.apache.org > > Subject: Re: Mirror Maker 2 - offset sync from s

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
n "4. The target group does not exist, or has no > active consumers" > If group on target does not exist, will it be created without active > consumers ? > > -Original Message- > From: Greg Harris > Sent: Monday, October 23, 2023 8:56 PM > To: users@kafk

RE: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Alexander Shapiro (ashapiro)
8:56 PM To: users@kafka.apache.org Subject: Re: Mirror Maker 2 - offset sync from source to target [You don't often get email from greg.har...@aiven.io.invalid. Learn why this is important at https://aka.ms/LearnAboutSenderIdentification ] CAUTION: This email is from an external source. Please

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
--Original Message- > From: Greg Harris > Sent: Monday, October 23, 2023 8:42 PM > To: users@kafka.apache.org > Subject: Re: Mirror Maker 2 - offset sync from source to target > > CAUTION: This email is from an external source. Please don’t open any unknown > links or attach

RE: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Alexander Shapiro (ashapiro)
Hi Greg, Thank you very much, it is the most detailed answer I would expect. -Original Message- From: Greg Harris Sent: Monday, October 23, 2023 8:42 PM To: users@kafka.apache.org Subject: Re: Mirror Maker 2 - offset sync from source to target CAUTION: This email is from an external

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
Hi Alexander, Thanks for using MirrorMaker2! If you turn on `sync.group.offsets.enabled`, then the MirrorCheckpointTask will sync the offsets if all of the following is true: 1. The source group exists 2. The source group name matches the configured group filter (group.filter.class, groups, group

Mirror Maker 2 - offset sync from source to target

2023-10-22 Thread Alexander Shapiro (ashapiro)
Hi Can someone advise please if sync.group.offsets.enabled : true to sync offset from source to target for particular consumer group That group must be created on target, even if no actual consumption will be done ? This email and the information contained herein is proprietary and confidentia

Re: mirror-maker

2023-10-07 Thread Denny Fuchs
hi Richard, Am 02.10.23 um 16:46 schrieb Richard Bosch: CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you recognize the sender and know the content is safe. Hi Denny, If you use mirrormaker to replicate data between clusters th

Re: mirror-maker

2023-10-02 Thread Richard Bosch
gt; I try to use mirror maker to replicate the data, and I have one trivial > question: > > Do I need access to / add **all** Kafka Brokers ? > > I have a 3.1 Cluster with 7 brokers and a new 3.5 with 5 brokers and I > want to migrate from the 3.1 (Zoo) to 3.5 (KRaft). Both cluste

Re: mirror-maker

2023-10-02 Thread Denny Fuchs
hi, it seems to be pretty clear .. TCPDump says, it tries to connect to the other hosts too, so I need to allow it. cu denny

mirror-maker

2023-10-02 Thread Denny Fuchs
hi, I try to use mirror maker to replicate the data, and I have one trivial question: Do I need access to / add  **all** Kafka Brokers ? I have a 3.1 Cluster with 7 brokers and a new 3.5 with 5 brokers and I want to migrate from the 3.1 (Zoo) to 3.5 (KRaft). Both clusters are in different

Re: Consumer group offset translation in Mirror Maker 2

2023-09-03 Thread Greg Harris
; specific differences between syncing offsets using RemoteClusterUtils and > the inbuilt MM2 consumer offset sync feature. Although I read through KIP > 545 regarding Mirror Maker 2, I didn't find much context to explain why > this approach is preferable. > > Additionally, woul

Re: Consumer group offset translation in Mirror Maker 2

2023-09-03 Thread Hemanth Savasere
trying to understand the specific differences between syncing offsets using RemoteClusterUtils and the inbuilt MM2 consumer offset sync feature. Although I read through KIP 545 regarding Mirror Maker 2, I didn't find much context to explain why this approach is preferable. Additionally, wo

Re: Consumer group offset translation in Mirror Maker 2

2023-08-29 Thread Greg Harris
Hey Hemanth! Thank you for asking about Mirror Maker 2! Offset translation is not so simple, so I'll summarize the main functionality and leave some pointers into the code for you to examine yourself. 1. After MirrorSourceTask writes a record, it receives a commitRecord callback [1]

Consumer group offset translation in Mirror Maker 2

2023-08-28 Thread Hemanth Savasere
Hi, We're using Mirror Maker 2 for replicating messages from our primary to secondary cluster, it's one way replication. We're also replicating by the consumer group offsets by adding the below properties sync.group.offsets.enabled=true sync.group.offsets.int

Mirror Maker 2 Query

2023-03-29 Thread Tech Guy
Dear Users, I have a general query regarding MM2. Can MM2 version 3.0 work with Kafka 2.5? Thanks

Re: Kafka Mirror maker stops replicating

2023-02-23 Thread Arpit Jain
debug it, and possibly auto-restart it. > > > Could you tell me how can I enable more logging ? > > I believe you can configure the logging by changing the KAFKA_LOG4J_OPTS > environment variable before running the mirror maker script. > For example, you could copy and mod

Re: Mirror maker worker can't issue with REST uri

2023-02-08 Thread Greg Harris
ffect of this is there is enormous amount of log generated. >> Is there a quick solution possible to slow down the logs. >> >> Cheers. >> >> From: Greg Harris >> Date: Wednesday, 8 February 2023 at 1:08 pm >> To: users@kafka.apache.org >> Subject: R

Re: Kafka Mirror maker stops replicating

2023-02-08 Thread Greg Harris
he replication from progressing. Good luck with your investigation! Greg On Wed, Feb 8, 2023 at 2:20 PM Arpit Jain wrote: > Hi Greg, > > Thanks for getting back to me. Please find more details below > > 1. Are you using MirrorMaker, or MirrorMaker 2.0? > Mirror maker > 2. Wh

Re: Kafka Mirror maker stops replicating

2023-02-08 Thread Arpit Jain
Hi Greg, Thanks for getting back to me. Please find more details below 1. Are you using MirrorMaker, or MirrorMaker 2.0? Mirror maker 2. What version of MM or MM2 are you using, and with what Kafka broker version? 3.2.3 3. How is your replication flow configured? We have upstream brokers (3 node

Re: Kafka Mirror maker stops replicating

2023-02-08 Thread Greg Harris
, Greg Harris On Tue, Feb 7, 2023 at 6:27 AM Arpit Jain wrote: > Hi, > > Hope this is the right forum to ask for Kafka mirror maker issues. > We are facing an issue where the mirror maker replicates the trades and > then doesn't work for long time and again replicates. >

Re: Mirror maker worker can't issue with REST uri

2023-02-08 Thread Greg Harris
k solution possible to slow down the logs. > > Cheers. > > From: Greg Harris > Date: Wednesday, 8 February 2023 at 1:08 pm > To: users@kafka.apache.org > Subject: Re: Mirror maker worker can't issue with REST uri > NetApp Security WARNING: This is an external email. Do

Re: Mirror maker worker can't issue with REST uri

2023-02-08 Thread Shirolkar, Anup
Yes, that makes sense thanks. But the side effect of this is there is enormous amount of log generated. Is there a quick solution possible to slow down the logs. Cheers. From: Greg Harris Date: Wednesday, 8 February 2023 at 1:08 pm To: users@kafka.apache.org Subject: Re: Mirror maker worker

Re: Mirror maker worker can't issue with REST uri

2023-02-07 Thread Greg Harris
nup wrote: > Hi, > > I have deployed a 3-node mirror maker cluster version 3.2.1 > I have configured the connect-mirror-maker.properties file and started the > mirror service using connect-mirror-maker.sh > > It runs fine but one of the three workers always gets below except

Mirror maker worker can't issue with REST uri

2023-02-07 Thread Shirolkar, Anup
Hi, I have deployed a 3-node mirror maker cluster version 3.2.1 I have configured the connect-mirror-maker.properties file and started the mirror service using connect-mirror-maker.sh It runs fine but one of the three workers always gets below exception. If I restart the connect worker with the

Kafka Mirror maker stops replicating

2023-02-07 Thread Arpit Jain
Hi, Hope this is the right forum to ask for Kafka mirror maker issues. We are facing an issue where the mirror maker replicates the trades and then doesn't work for long time and again replicates. Also seeing the warning message to increase the poll interval or decrease the maximum batch

Re: Mirror Maker 2 with different Avro Schema Registries

2022-12-29 Thread andreasvdber...@avathar.be
hi, I guess this would mean a deserialisation error, meaning he data you're trying to read, is not as expected by the Confluent Avro deserializer. https://www.youtube.com/watch?v=GWBk9LIzy-A > On 29 Dec 2022, at 16:47, Дмитрий Лушников wrote: >

Mirror Maker 2 with different Avro Schema Registries

2022-12-29 Thread Дмитрий Лушников
Anders, maybe you can help me? MirrorMaker 2 stops transfer messages after setting up Schema Registry Transfer SMT. The logs show these errors: Caused by: org.apache.kafka.common.errors.SerializationException: Unknown magic byte! Caused by: cricket.jmoore.confluent.kafka.schemaregistry.client.r

Delete Kafka topics replicated by mirror maker 2

2022-12-21 Thread Shirolkar, Anup
Hi Team, I have configured two Kafka clusters with 2-way replication using MirrorMaker2 (mm2). I want to delete a topic from source cluster and its replica from destination cluster. When I delete a topic in the source cluster and then delete the replica in the destination cluster, the desti

Mirror Maker: copy data from source cluster to target with different number of partitions

2022-09-26 Thread Mikhail Dubrovin
We have a cluster with topics that contain many partitions. We are going to decrease the number of partitions and looking for any solution. I t would be very useful to make it with MirrorMaker 2 but as we can see, it doesn't allow us to do it. MM2 creates missing partitions for a topic, e.g. source

Re: Mirror maker disable auto topic creation

2022-09-16 Thread Mcs Vemuri
Thanks Ryanne Sent from Yahoo Mail for iPhone On Thursday, September 15, 2022, 12:00 AM, Ryanne Dolan wrote: The link is correct. Connect and mm2 create those internal topics at startup, whether or not auto topic creation is enabled. On Wed, Sep 14, 2022, 11:15 PM Mcs Vemuri wrote: > Hello,

Mirror maker disable topic creation

2022-09-15 Thread Mcs Vemuri
Hello, Is there any way to disable topic creation in MM2? I tried setting the topic.creation.enable to False but the MM-offsets/configs/status topics are still created(broker auto topic creation is also set to false) I found this- https://groups.google.com/g/confluent-platform/c/JRZmpCEZElo  whi

Re: Mirror Maker is not replicating WRITE ACLs for topics and consumer group ACLs

2022-09-15 Thread John Lewis
Hello, this is a gentle reminder. Any response will be appreciated. John ср, 7 сент. 2022 г. в 15:03, John Lewis : > Hi everyone, > > I installed and ran Mirror Maker 2 to replicate all possible data from > Active cluster to Standby one. It's replicating topics, consumer gro

Re: Mirror maker disable auto topic creation

2022-09-14 Thread Ryanne Dolan
The link is correct. Connect and mm2 create those internal topics at startup, whether or not auto topic creation is enabled. On Wed, Sep 14, 2022, 11:15 PM Mcs Vemuri wrote: > Hello, > Is there any way to disable topic creation in MM2? I tried setting the > topic.creation.enable to False but the

Mirror maker disable auto topic creation

2022-09-14 Thread Mcs Vemuri
Hello, Is there any way to disable topic creation in MM2? I tried setting the topic.creation.enable to False but the MM-offsets/configs/status topics are still created(broker auto topic creation is also set to false) I found this- https://groups.google.com/g/confluent-platform/c/JRZmpCEZElo  whi

Mirror Maker is not replicating WRITE ACLs for topics and consumer group ACLs

2022-09-07 Thread John Lewis
Hi everyone, I installed and ran Mirror Maker 2 to replicate all possible data from Active cluster to Standby one. It's replicating topics, consumer groups, topic READ ACLs fine, but it is not replicating WRITE ACLs for topics and consumer group ACLs (where resourceType=GROUP) at all, for ex

How to measure/monitor lag for Mirror Maker 2

2022-08-02 Thread Ashish Gupta
What is the best way to monitor the lag for Mirror Maker 2? so that I can know if I need to add more nodes to the Mirror Maker 2 if it is falling behind.

what is Best Practices of kafka mirror maker 2 #12375

2022-07-04 Thread Phongtat Yaemsomphong
Now, i config mm2,properties this below clusters=DC1,DC2 DC1.bootstrap.servers = broker1-dev1:30011, broker2-dev1:30012, broker3-dev1:30013 DC2.bootstrap.servers = broker1-dev2:30011, broker2-dev2:30012, broker3-dev2:30013 DC1.config.storage.replication.factor = 3 DC2.config.storage.replication.f

Mirror Maker 2.0

2022-06-08 Thread Josée Brodeur
Good morning, I would like to have more information about the acquisition of the licence Mirror Maker 2.0 and the commercial support that is offered Thank you very much ! [cid:image001.png@01D87B03.2452BCB0] Josée Brodeur Conseillère Approvisionnement stratégique, Technologies d'aff

Re: Mirror Maker 2 - High Throughput Identity Mirroring

2022-03-02 Thread Ryanne Dolan
Henry Cai has a PR somewhere. Ryanne On Wed, Mar 2, 2022, 3:36 AM Antón Rodríguez Yuste wrote: > Hi Ryanne, > > Is there a PR or code I could take a look at or just the KIP? "shallow > mirroring" seems very interesting for our use cases, and I would like to > evaluate it internally. > > Thanks,

Re: Mirror Maker 2 - High Throughput Identity Mirroring

2022-03-02 Thread Antón Rodríguez Yuste
Hi Ryanne, Is there a PR or code I could take a look at or just the KIP? "shallow mirroring" seems very interesting for our use cases, and I would like to evaluate it internally. Thanks, Antón On Thu, Jul 29, 2021 at 7:02 PM Ryanne Dolan wrote: > Jamie, this would depend on KIP-712 (or simila

Re: Mirror Maker Issue : Huge difference in topic size on disk

2022-02-06 Thread Samuel Cantero
fka Experts , Need your advice please > > I am running a mirror maker in kafka 2.8 to replicate a topic from kafka > 0.11 instance. > The size of each partition for a topic on 0.11 is always in 5 to 6 GB but > the replicated topic in 2.8 instances is in 40 GB for the same partit

Mirror Maker Issue : Huge difference in topic size on disk

2022-02-06 Thread Kafka Life
Dear Kafka Experts , Need your advice please I am running a mirror maker in kafka 2.8 to replicate a topic from kafka 0.11 instance. The size of each partition for a topic on 0.11 is always in 5 to 6 GB but the replicated topic in 2.8 instances is in 40 GB for the same partition. The topic

Re: Mirror Maker 2: use prefix-less topic names?

2021-10-07 Thread Ryanne Dolan
Jake, in the most recent Kafka 3.0 release you will find IdentityReplicationPolicy, which does what you want. Just be careful that you don't try to replicate the same topics in a loop. Ryanne On Thu, Oct 7, 2021, 1:31 PM Jake Mayward wrote: > Hi, > > I have read a bit on MirrorMaker 2 via > htt

Re: Mirror Maker 2: use prefix-less topic names?

2021-10-07 Thread Men Lim
Nope. Mm2 has that prefix. I read that in 3.0, that prefix is going away. On Thu, Oct 7, 2021 at 11:31 AM Jake Mayward wrote: > Hi, > > I have read a bit on MirrorMaker 2 via > https://kafka.apache.org/documentation/#georeplication-overview, and I am > curious whether I can enable replication wi

Mirror Maker 2: use prefix-less topic names?

2021-10-07 Thread Jake Mayward
Hi, I have read a bit on MirrorMaker 2 via https://kafka.apache.org/documentation/#georeplication-overview, and I am curious whether I can enable replication with MirrorMaker 2 without having to prefix the topic with the cluster name. Reasoning behind this is that I would like the client to al

Not able to replicate groups in Mirror Maker 2

2021-09-29 Thread Hemanth Savasere
- - Here, the topic is getting replicated but when I run the command to describe the group in the destination cluster kafka-consumer-groups.sh --bootstrap-server localhost:9093 --describe --group quickstartGroup Getting error as Error: Consumer group 'quickstartGroup' does

Not able to replicate groups in Mirror Maker 2

2021-09-29 Thread Hemanth Savasere
- - - Here, the topic is getting replicated but when I run the command to describe the group in the destination cluster kafka-consumer-groups.sh --bootstrap-server localhost:9093 --describe --group quickstartGroup Getting error as Error: Consumer group 'quickstartGroup'

Re: Mirror Maker 2 with different Avro Schema Registries

2021-09-16 Thread Anders Engström
Quick informal update: Plugging in an SMT works like a charm 👍. Thanks for pointing me in the right direction. Took some time to understand that the MirrorSinkConnector mentioned in the KIP doesn't actually exist in Kafka. When running MM2 as a "pure" MirrorSourceConnector I need to point the conne

Re: Mirror Maker 2 with different Avro Schema Registries

2021-09-14 Thread Ryanne Dolan
Yes it's a consequence of Connect, which has its own serde model. mirror-makers hard-coded ByteArraySerialization for > consumers/producers

Re: Mirror Maker 2 with different Avro Schema Registries

2021-09-14 Thread Anders Engström
On Tue, Sep 14, 2021 at 3:57 PM Ryanne Dolan wrote: > Hey Anders, take a look at Connect's serdes and SMTs. MirrorMaker can be > configured to use them. > > Ryanne > > On Tue, Sep 14, 2021, 3:13 AM Anders Engström wrote: > > > Hi! > > I'm trying to replic

Re: Mirror Maker 2 with different Avro Schema Registries

2021-09-14 Thread Ryanne Dolan
Hey Anders, take a look at Connect's serdes and SMTs. MirrorMaker can be configured to use them. Ryanne On Tue, Sep 14, 2021, 3:13 AM Anders Engström wrote: > Hi! > I'm trying to replicate a few topics using Mirror Maker 2 (2.8). > > Both the source and the target clust

Mirror Maker 2 with different Avro Schema Registries

2021-09-14 Thread Anders Engström
Hi! I'm trying to replicate a few topics using Mirror Maker 2 (2.8). Both the source and the target cluster use Schema Registry (Karapace, on Aiven) and the Confluent Avro serialization format when publishing messages. This means that messages replicated from source->target are not reada

Re: Kafka Mirror Maker 2 - source topic keep getting created

2021-08-13 Thread Men Lim
ue source->target.emit.checkpoints.enabled=true On Tue, Aug 10, 2021 at 7:54 PM AlR wrote: > I have 2 Kafka setup, A and B. A is a cluster with 3 instances running on > the same machine, while B is a stand alone in another machine. I tried to > use Mirror Maker to replicate from A to B. The conf

Kafka Mirror Maker 2 - source topic keep getting created

2021-08-10 Thread AlR
I have 2 Kafka setup, A and B. A is a cluster with 3 instances running on the same machine, while B is a stand alone in another machine. I tried to use Mirror Maker to replicate from A to B. The config file is as follows: - clusters = A, BA.bootstrap.servers = host1:9091, host1:9092, host1

Re: Mirror Maker 2 - High Throughput Identity Mirroring

2021-07-29 Thread Ryanne Dolan
Jamie, this would depend on KIP-712 (or similar) aka "shallow mirroring". This is a work in progress, but I'm optimistic it'll happen at some point. ftr, "IdentityReplicationPolicy" has landed for the upcoming release, tho "identity" in that context just means that topics aren't renamed. Ryanne

Mirror Maker 2 - High Throughput Identity Mirroring

2021-07-29 Thread Jamie
Hi All,  This blog post: https://blog.cloudera.com/a-look-inside-kafka-mirrormaker-2/ mentions that "High Throughput Identity Mirroring" (when the compression is the same in both the source and destination cluster) will soon be coming to MM2 which would avoid the MM2 consumer decompressing the d

Re: Mirror Maker 2: Incoming messages on source and target kafka cluster mismatch after mirroring

2021-05-18 Thread Ning Zhang
t;. kafka connect Producer enter image > > description here <https://i.stack.imgur.com/asp4t.png> > > > > source kafka:enter image description here > > <https://i.stack.imgur.com/JrKmK.png> target kafka: enter image > > description here <https://i.stack.imgur.

Re: Mirror Maker 2: Incoming messages on source and target kafka cluster mismatch after mirroring

2021-04-29 Thread fighter
on here > <https://i.stack.imgur.com/JrKmK.png> target kafka: enter image > description here <https://i.stack.imgur.com/Umtx7.png> > > We notices Mirror maker tasks and workers has no issues. We have three > connect worker nodes only one connect node has this producer &a

Mirror Maker 2: Incoming messages on source and target kafka cluster mismatch after mirroring

2021-04-28 Thread fighter
ur.com/Umtx7.png> We notices Mirror maker tasks and workers has no issues. We have three connect worker nodes only one connect node has this producer & consumer difference. As we don't have logs any insight will help to investigate the issue. Kafka MM Source connector producer and consumer uses default config only.

Re: RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2021-04-05 Thread Ning Zhang
mm2. Any help will be appreciated. Looking forward to > your response. > > Thanks, > Samruddhi > > > On 2020/12/08 06:28:50, wrote: > > Hi Ning,> > > > > > > > > > > > > > > > > It did not worked. Here are

RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2021-04-05 Thread Samruddhi Naik
e logs from the replicator and mirror maker 2 > respectively:> > > > > Replicator:> > > > > [2020-12-08 05:11:06,611] INFO [Consumer clientId=onprem-aws-replicator-0, > groupId=onprem-aws-replicator] See

Re: Mirror Maker 2 - Issues

2021-03-10 Thread Navneeth Krishnan
cation.factor = 2 target.status.storage.replication.factor = 2 source->target.enabled = true target->source.enabled = true # Mirror maker configurations. offset-syncs.topic.replication.factor = 2 heartbeats.topic.replication.factor = 2 checkpoints.topic.replication.factor = 2 topics = .* groups = .* r

Re: Mirror Maker 2 - Issues

2021-03-07 Thread Navneeth Krishnan
storage.replication.factor = 2 > > target.config.storage.replication.factor = 2 > > > > source.offset.storage.replication.factor = 2 > > target.offset.storage.replication.factor = 2 > > > > source.status.storage.replication.factor = 2 > > target.status.storage.r

Re: Mirror Maker 2 - Issues

2021-03-07 Thread Ryanne Dolan
torage.replication.factor = 2 > > source.offset.storage.replication.factor = 2 > target.offset.storage.replication.factor = 2 > > source.status.storage.replication.factor = 2 > target.status.storage.replication.factor = 2 > > source->target.enabled = true > targe

Re: Mirror Maker 2 - Issues

2021-03-07 Thread Navneeth Krishnan
r = 2 source.offset.storage.replication.factor = 2 target.offset.storage.replication.factor = 2 source.status.storage.replication.factor = 2 target.status.storage.replication.factor = 2 source->target.enabled = true target->source.enabled = false # Mirror maker configurati

Re: Mirror Maker 2 - Issues

2021-03-07 Thread Ryanne Dolan
ll, > > I'm trying to use mirror maker 2 to replicate data to our new AWS MSK kafka > cluster and I have been running into so many issues and I couldn't find > proper documentation. Need some help and it's very urgent. Thanks > > Also I don't see any of my

Mirror Maker 2 - Issues

2021-03-06 Thread Navneeth Krishnan
Hi All, I'm trying to use mirror maker 2 to replicate data to our new AWS MSK kafka cluster and I have been running into so many issues and I couldn't find proper documentation. Need some help and it's very urgent. Thanks Also I don't see any of my topics created. Note: Th

Mirror Maker for Avro Messages (with schema registry)

2021-03-03 Thread Adithya Tirumale
below options : * Add _schemas topic to the list of topics to be syncd by mirror maker. This way when the applications in the cloned environment try to look up schema for a schema id it gets the same thing as in production. * Add a special message transformation to the mirror maker to basi

Re: RE: RE: RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-23 Thread Ning Zhang
uot;:0}] > Value: {"offset":24} > > For each topic and partition, whenever a new message is replicated a new > message with same key but increased offset is produced to the connect-offsets > topic. > Convert the key of this message to Mirror Maker 2 format and

RE: RE: RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-20 Thread Amit.SRIVASTAV
. Key : ["replicator-group",{"topic":"TEST","partition":0}] Value: {"offset":24} For each topic and partition, whenever a new message is replicated a new message with same key but increased offset is produced to the connect-offsets topic. Convert the key of

Re: RE: RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-10 Thread Ning Zhang
work, then we should look into more source code and see if your setting is override by other places On 2020/12/08 06:28:50, wrote: > Hi Ning, > > > > It did not worked. Here are the logs from the replicator and mirror maker 2 > respectively: > > Replicator: > >

RE: RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-07 Thread Amit.SRIVASTAV
Hi Ning, It did not worked. Here are the logs from the replicator and mirror maker 2 respectively: Replicator: [2020-12-08 05:11:06,611] INFO [Consumer clientId=onprem-aws-replicator-0, groupId=onprem-aws-replicator] Seeking to offset 83 for partition ONPREM.AWS.REPLICA.TOPIC.P3R3-0

Re: RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-07 Thread Ning Zhang
December 7, 2020 3:46 AM > To: users@kafka.apache.org > Subject: Re: Maintaining same offset while migrating from Confluent > Replicator to Apache Mirror Maker 2.0 > > [External] > > > Hi Amit, I guess you may need to override the actual consumer group config > (probably not co

RE: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-06 Thread Amit.SRIVASTAV
: users@kafka.apache.org Subject: Re: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0 [External] Hi Amit, I guess you may need to override the actual consumer group config (probably not consumer.group.id) that is used in Kafka Connect On 2020/11/26 06:47

Re: Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-12-06 Thread Ning Zhang
Hi Amit, I guess you may need to override the actual consumer group config (probably not consumer.group.id) that is used in Kafka Connect  On 2020/11/26 06:47:11, wrote: > Hi All, > > We are currently trying to migrate Confluent replicator to Apache Open Source > Mirror Maker

Maintaining same offset while migrating from Confluent Replicator to Apache Mirror Maker 2.0

2020-11-25 Thread Amit.SRIVASTAV
Hi All, We are currently trying to migrate Confluent replicator to Apache Open Source Mirror Maker v2.0. We are facing an issue where the messages which are already replicated by replicator is getting replicated again when the mirror maker is started on the same topic. This should not happen

Compression impact on Mirror Maker

2020-11-03 Thread usha rani
compressed data on the topic without decompressing using mirror maker. Thanks and regards, Usha

Re: Kafka Mirror Maker 2: RemoteClusterUtils.translateOffsets() returning empty map

2020-10-13 Thread Sunny Lohani
> I have 2 data centers, each having single node Zookeeper and Kafka >> > > cluster. >> > > > I have a topic (single partition) in both the data center kafka >> > > clusters. I >> > > > am using MM 2.0 as a dedicated cluster for bi-dire

Re: Mirror Maker 2.0 NOT generating checkpoints for consumers running in assign mode

2020-09-11 Thread Ananya Sen
"Ananya Sen" wrote: > > [External] > > > Hello All, > > I was using a mirror maker 2.0. I was testing the consumer > checkpointing functionality. I found that the > RemoteClusterUtils.translateOffsets do not give checkpoints for the > consumer w

Re: Mirror Maker 2.0 NOT generating checkpoints for consumers running in assign mode

2020-09-11 Thread Manoj.Agrawal2
on.ofMillis(5500)); Properties= Bootstraps properties of cluster B TestTopic-123= Topic name at cluster A Thanks On 9/7/20, 8:43 AM, "Ananya Sen" wrote: [External] Hello All, I was using a mirror maker 2.0. I was testing the consumer che

Mirror Maker 2.0 NOT generating checkpoints for consumers running in assign mode

2020-09-07 Thread Ananya Sen
Hello All, I was using a mirror maker 2.0. I was testing the consumer checkpointing functionality. I found that the RemoteClusterUtils.translateOffsets do not give checkpoints for the consumer which run in assign mode. I am using mirror maker 2.0 of Kafka Version 2.5.0 and Scala version 2.12

  1   2   3   4   >