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
wro
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
> accord
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
"Viktor Somogyi-Vass"
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
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`). Obviousl
@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
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 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
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 sour
Hi huqedi,
Thanks for taking a look at this,
The *poll* is executed by the task anyway (it's an essential part of the
logic) and this KIP only aims to piggyback that request - it will not make
any kind of additional requests, thus not adding any performance overhead.
Thanks for mentioning the issu
Hi Elkhan,
I have also done work similar to the partition replication lag mentioned in
this kip. After going online, I discovered a problem: when `MirrorSourceTask`
executes `poll`, it obtains the LEO of the source topic through the consumer.
This logic may take seconds to jitter. , I think this
11 matches
Mail list logo