The offset lag is the difference between the last end offset of the source partition (LEO) and the last replicated source offset (LRO). The offset of the last `ConsumerRecord` for a partition obtained in poll represent the LEO in the source cluster, and LRO is obtained at producer callback where we have the source offset of the record being committed available to us: https://kafka.apache.org/26/javadoc/org/apache/kafka/connect/source/SourceTask.html#commitRecord-org.apache.kafka.connect.source.SourceRecord-org.apache.kafka.clients.producer.RecordMetadata-
Does this make sense? Thanks! Elkhan On Mon, 4 Sept 2023 at 09:17, hudeqi <16120...@bjtu.edu.cn> wrote: > But does the offset of the last `ConsumerRecord` obtained in poll not only > represent the offset of this record in the source cluster? It seems that it > cannot represent the LEO of the source cluster for this partition. I > understand that the offset lag introduced here should be the LEO of the > source cluster minus the offset of the last record to be polled? > > best, > hudeqi > > > > -----原始邮件----- > > 发件人: "Elxan Eminov" <elxanemino...@gmail.com> > > 发送时间: 2023-09-04 14:52:08 (星期一) > > 收件人: dev@kafka.apache.org > > 抄送: > > 主题: Re: [DISCUSS] KIP-971 Expose replication-offset-lag MirrorMaker2 > metric > > > </elxanemino...@gmail.com>