Hi Viktor and huqedi,
Apologies for the late reply - yes, I believe a periodic update of the LEO
is the better approach here.
I will update the KIP accordingly and ping back in this thread once done.

Thanks a lot for the input!
Best, Elkhan


On Wed, 13 Sept 2023 at 15:34, Viktor Somogyi-Vass
<viktor.somo...@cloudera.com.invalid> wrote:

> Elkhan, do you think making yours similar would make sense?
>
> On Wed, Sep 6, 2023 at 4:12 AM hudeqi <16120...@bjtu.edu.cn> wrote:
>
> > Hey, Viktor.
> > As far as my implementation is concerned, the default setting is 30s, but
> > I added it to `MirrorConnectorConfig`, which can be adjusted freely
> > according to the load of the source cluster and the number of tasks.
> >
> > best,
> > hudeqi
> >
> > &quot;Viktor Somogyi-Vass&quot; &lt;viktor.somo...@cloudera.com.INVALID
> > &gt;写道:
> > > Hey Elkhan and hudeqi,
> > >
> > > I'm reading your debate around the implementation. I also think a
> > > scheduled task would be better in overall accuracy and performance
> > > (compared to calling endOffsets with every poll).
> > > Hudeqi, do you have any experience of what works best for you in terms
> of
> > > time intervals? I would think refreshing the metric every 5-10sec would
> > be
> > > overall good and sufficient for the users (as short intervals can be
> > quite
> > > noisy anyways).
> > >
> > > Best,
> > > Viktor
> > >
> > > On Mon, Sep 4, 2023 at 11:41 AM hudeqi <16120...@bjtu.edu.cn> wrote:
> > >
> > > > My approach is to create another thread to regularly request and
> update
> > > > the end offset of each partition for the `keySet` in the collection
> > > > `lastReplicatedSourceOffsets` mentioned by your kip (if there is no
> > update
> > > > for a long time, it will be removed from
> > `lastReplicatedSourceOffsets`).
> > > > Obviously, such processing makes the calculation of the partition
> > offset
> > > > lag less real-time and accurate.
> > > > But this also meets our needs, because we need the partition offset
> > lag to
> > > > analyze the replication performance of the task and which task may
> have
> > > > performance problems; and if you monitor the overall offset lag of
> the
> > > > topic, then using the
> > > > "kafka_consumer_consumer_fetch_manager_metrics_records_lag" metric
> > will be
> > > > more real-time and accurate.
> > > > This is my suggestion. I hope to be able to throw bricks and start
> > jade,
> > > > we can come up with a better solution.
> > > >
> > > > best,
> > > > hudeqi
> > > >
> > > > &quot;Elxan Eminov&quot; &lt;elxanemino...@gmail.com&gt;写道:
> > > > > @huqedi replying to your comment on the PR (
> > > > > https://github.com/apache/kafka/pull/14077#discussion_r1314592488
> ),
> > > > quote:
> > > > >
> > > > > "I guess we have a disagreement about lag? My understanding of lag
> > is:
> > > > the
> > > > > real LEO of the source cluster partition minus the LEO that has
> been
> > > > > written to the target cluster. It seems that your definition of lag
> > is:
> > > > the
> > > > > lag between the mirror task getting data from consumption and
> > writing it
> > > > to
> > > > > the target cluster?"
> > > > >
> > > > > Yes, this is the case. I've missed the fact that the consumer
> itself
> > > > might
> > > > > be lagging behind the actual data in the partition.
> > > > > I believe your definition of the lag is more precise, but:
> > > > > Implementing it this way will come at the cost of an extra
> > listOffsets
> > > > > request, introducing the overhead that you mentioned in your
> initial
> > > > > comment.
> > > > >
> > > > > If you have enough insights about this, what would you say is the
> > chances
> > > > > of the task consumer lagging behind the LEO of the partition?
> > > > > Are they big enough to justify the extra call to listOffsets?
> > > > > @Viktor,  any thoughts?
> > > > >
> > > > > Thanks,
> > > > > Elkhan
> > > > >
> > > > > On Mon, 4 Sept 2023 at 09:36, Elxan Eminov <
> elxanemino...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > I already have the PR for this so if it will make it easier to
> > discuss,
> > > > > > feel free to take a look:
> > https://github.com/apache/kafka/pull/14077
> > > > > >
> > > > > > 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
> > > > > >>
> > > > > >>
> > > > > >> &gt; -----原始邮件-----
> > > > > >> &gt; 发件人: "Elxan Eminov" <elxanemino...@gmail.com>
> > > > > >> &gt; 发送时间: 2023-09-04 14:52:08 (星期一)
> > > > > >> &gt; 收件人: dev@kafka.apache.org
> > > > > >> &gt; 抄送:
> > > > > >> &gt; 主题: Re: [DISCUSS] KIP-971 Expose replication-offset-lag
> > > > MirrorMaker2
> > > > > >> metric
> > > > > >> &gt;
> > > > > >> </elxanemino...@gmail.com>
> > > > > >
> > > > > >
> > > >
> >
>

Reply via email to