regarding consumer group = null, https://github.com/apache/kafka/blob/trunk/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java#L89 is where the consumer is created in MirrorSourceTask and people could override any consumer-level config (including group.id) at https://github.com/apache/kafka/blob/b44d32dffedb368b888e3431257d68abb1e62b9f/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java#L249
as you may have tried something like `source.consumer.group.id`, `<source_cluster.alias>.consumer.group.id`, or `<source_cluster.alias>.group.id`, if all of them do not 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, <amit.srivas...@cognizant.com> wrote: > 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 > (org.apache.kafka.clients.consumer.KafkaConsumer:1545) > > [2020-12-08 05:11:07,113] INFO [Consumer clientId=onprem-aws-replicator-0, > groupId=onprem-aws-replicator] Seeking to offset 49 for partition > ONPREM.AWS.REPLICA.TOPIC.P3R3-1 > (org.apache.kafka.clients.consumer.KafkaConsumer:1545) > > [2020-12-08 05:11:07,615] INFO [Consumer clientId=onprem-aws-replicator-0, > groupId=onprem-aws-replicator] Seeking to offset 53 for partition > ONPREM.AWS.REPLICA.TOPIC.P3R3-2 > (org.apache.kafka.clients.consumer.KafkaConsumer:1545) > > > > Mirror Maker 2.0: > > [2020-12-08 06:10:51,385] INFO [Consumer clientId=consumer-4, groupId=null] > Seeking to offset 80 for partition ONPREM.AWS.REPLICA.TOPIC.P3R3-0 > (org.apache.kafka.clients.consumer.KafkaConsumer:1545) > > [2020-12-08 06:10:51.385] INFO [Consumer clientId= consumer-4, groupId=null] > Seeking to offset 49 for partition ONPREM.AWS.REPLICA.TOPIC.P3R3-0 > (org.apache.kafka.clients.consumer.KafkaConsumer:1545) > > [2020-12-08 06:10:51.386] INFO [Consumer clientId= consumer-4, groupId=null] > Seeking to offset 52 for partition ONPREM.AWS.REPLICA.TOPIC.P3R3-0 > (org.apache.kafka.clients.consumer.KafkaConsumer:1545) > > > > You can see that groupId is still null in MM2 and the offsets are previous > offset meaning it will replicate those messages as well which has been > already replicated by Replicator. > > > > Thanks and regards, > > Amit > > > > -----Original Message----- > From: Ning Zhang <ning2008w...@gmail.com> > Sent: Monday, December 7, 2020 10:29 PM > To: users@kafka.apache.org > Subject: Re: RE: Maintaining same offset while migrating from Confluent > Replicator to Apache Mirror Maker 2.0 > > > > [External] > > > > > > Hi Amit, > > > > After looking into a little bit, do you mind to override > `connector.consumer.group`? The default consumer group may be called > 'connect-MirrorSourceConnector' or similar > > > > On 2020/12/07 03:32:30, > <amit.srivas...@cognizant.com<mailto:amit.srivas...@cognizant.com>> wrote: > > > Hi Ning, > > > > > > Thank you for the response. > > > > > > I probably tried to change every possible consumer group id in all ways MM2 > > can run. But I went through the code and it seems consumer config is using > > assign method which does not requires group id to be assigned and also > > could not find anywhere that property being read and set. > > > > > > One crude way we found and is in testing is to manually change the offset > > of the topic in the internal topics which MM2 reads to get the latest > > offset of the message. > > > > > > Thanks, > > > Amit > > > > > > -----Original Message----- > > > From: Ning Zhang <ning2008w...@gmail.com<mailto:ning2008w...@gmail.com>> > > > Sent: Monday, December 7, 2020 3:46 AM > > > To: users@kafka.apache.org<mailto: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:11, > > <amit.srivas...@cognizant.com<mailto:amit.srivas...@cognizant.com>> wrote: > > > > 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 as > > > messages are getting duplicated at the target cluster. Here are more > > > details: > > > > > > > > > > > > 1. RCA: replicator assign a consumer group for replicating > > > messages. This consumer group maintains the offset of the source topic. > > > But we are not able to assign same consumer group to the Consumer config > > > in mirror maker 2. > > > > > > > > 2. Mirror Maker 1.0 : working as same consumer group can be > > > assigned in consumer.properties file and the messages are picked right > > > after where replicator was stopped. > > > > > > > > 3. Tried running and configuring source.cluster.consumer.group.id > > > in mirror maker 2.0 in all available options (in cluster mode, in > > > connect-standalone and connect-distributed mode) but mirror maker 2.0 is > > > assigning consumer group id as null while replicating messages. > > > > > > > > > > > > Any pointers if anyone has done same and tried to maintain the same > > > offset with mirror maker 2.0. > > > > > > > > Thanks and regards, > > > > Amit > > > > This e-mail and any files transmitted with it are for the sole use of the > > > intended recipient(s) and may contain confidential and privileged > > > information. If you are not the intended recipient(s), please reply to > > > the sender and destroy all copies of the original message. Any > > > unauthorized review, use, disclosure, dissemination, forwarding, printing > > > or copying of this email, and/or any action taken in reliance on the > > > contents of this e-mail is strictly prohibited and may be unlawful. Where > > > permitted by applicable law, this e-mail and other e-mail communications > > > sent to and from Cognizant e-mail addresses may be monitored. > > > > > > > This e-mail and any files transmitted with it are for the sole use of the > > intended recipient(s) and may contain confidential and privileged > > information. If you are not the intended recipient(s), please reply to the > > sender and destroy all copies of the original message. Any unauthorized > > review, use, disclosure, dissemination, forwarding, printing or copying of > > this email, and/or any action taken in reliance on the contents of this > > e-mail is strictly prohibited and may be unlawful. Where permitted by > > applicable law, this e-mail and other e-mail communications sent to and > > from Cognizant e-mail addresses may be monitored. > > > > > This e-mail and any files transmitted with it are for the sole use of the > intended recipient(s) and may contain confidential and privileged > information. If you are not the intended recipient(s), please reply to the > sender and destroy all copies of the original message. Any unauthorized > review, use, disclosure, dissemination, forwarding, printing or copying of > this email, and/or any action taken in reliance on the contents of this > e-mail is strictly prohibited and may be unlawful. Where permitted by > applicable law, this e-mail and other e-mail communications sent to and from > Cognizant e-mail addresses may be monitored. >