[ 
https://issues.apache.org/jira/browse/KAFKA-15144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17739883#comment-17739883
 ] 

Edoardo Comar edited comment on KAFKA-15144 at 7/4/23 12:12 PM:
----------------------------------------------------------------

again, with {color:#000000}offset.lag.max={color}{color:#a31515}100{color}

producing at irregular rate with a console producer,

the checkpoints are unexpected to me

 

OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0, 
downstreamOffset=0}
OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=101, 
downstreamOffset=101}
OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=202, 
downstreamOffset=202}

 

Checkpoint\{consumerGroupId=edogroup, 
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=1, 
downstreamOffset=1, metadata=}
Checkpoint\{consumerGroupId=edogroup, 
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=118, 
downstreamOffset=102, metadata=}
Checkpoint\{consumerGroupId=edogroup, 
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=227, 
downstreamOffset=203, metadata=}

 

so a consumer using Checkpoint switching from source to target is guaranteed 
not to miss any messages but may reprocess quite a few, correct ?


was (Author: ecomar):
again, with {color:#000000}offset.lag.max={color}{color:#a31515}100{color}

producing at irregular rate with a console producer,

the checkpoints are unexpected to me

 

OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0, 
downstreamOffset=0}
OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=101, 
downstreamOffset=101}
OffsetSync\{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=202, 
downstreamOffset=202}

 

Checkpoint\{consumerGroupId=edogroup, 
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=1, 
downstreamOffset=1, metadata=}
Checkpoint\{consumerGroupId=edogroup, 
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=118, 
downstreamOffset=102, metadata=}
Checkpoint\{consumerGroupId=edogroup, 
topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=227, 
downstreamOffset=203, metadata=}

> MM2 Checkpoint downstreamOffset stuck to 1
> ------------------------------------------
>
>                 Key: KAFKA-15144
>                 URL: https://issues.apache.org/jira/browse/KAFKA-15144
>             Project: Kafka
>          Issue Type: Bug
>          Components: mirrormaker
>            Reporter: Edoardo Comar
>            Assignee: Edoardo Comar
>            Priority: Major
>         Attachments: edo-connect-mirror-maker-sourcetarget.properties
>
>
> Steps to reproduce :
> 1.Start the source cluster
> 2.Start the target cluster
> 3.Start connect-mirror-maker.sh using a config like the attached
> 4.Create a topic in source cluster
> 5.produce a few messages
> 6.consume them all with autocommit enabled
>  
> 7. then dump the Checkpoint topic content e.g.
> {{% bin/kafka-console-consumer.sh --bootstrap-server localhost:9192 --topic 
> source.checkpoints.internal --from-beginning --formatter 
> org.apache.kafka.connect.mirror.formatters.CheckpointFormatter}}
> {{{}Checkpoint{consumerGroupId=edogroup, 
> topicPartition=source.vf-mirroring-test-edo-0, upstreamOffset=3, 
> {*}downstreamOffset=1{*}, metadata={}}}}
>  
> the downstreamOffset remains at 1, while, in a fresh cluster pair like with 
> the source topic created while MM2 is running, 
> I'd expect the downstreamOffset to match the upstreamOffset.
> Note that dumping the offset sync topic, shows matching initial offsets
> {{% bin/kafka-console-consumer.sh --bootstrap-server localhost:9192 --topic 
> mm2-offset-syncs.source.internal --from-beginning --formatter 
> org.apache.kafka.connect.mirror.formatters.OffsetSyncFormatter}}
> {{{}OffsetSync{topicPartition=vf-mirroring-test-edo-0, upstreamOffset=0, 
> downstreamOffset=0{}}}}
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to