You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Sagar <sa...@gmail.com> on 2023/09/06 10:26:51 UTC

Re: [DISCUSS] KIP-910: Update Source offsets for Source Connectors without producing records

Hey All,

I had an offline discussion with Yash on this and while so far there didn't
seem to be a pressing need to introduce the delete offsets mechanism via
the updateOffsets method, Yash had brought up an interesting point. Point
being that if we don't introduce the deletion of offsets mechanism in this
KIP but do it in a Future version, then connector developers and users
would get different behaviour on tombstone offsets based on the runtime
version being run. This could lead to confusion.

Considering this, I have updated the KIP to also allow deleting offsets via
tombstone records. Thanks Yash for closing out on this one!

Hopefully all open questions have now been addressed.

Thanks!
Sagar.

On Tue, Aug 29, 2023 at 3:33 PM Yash Mayya <ya...@gmail.com> wrote:

> Hi Sagar,
>
> > The size of offsets topic can be controlled by
> > setting appropriate topic retention values and
> > that is a standard practice in Kafka
>
> Kafka Connect enforces the `cleanup.policy` configuration for the offsets
> topic to be "compact" only (references - [1], [2]), so the topic retention
> related configurations won't be relevant right?
>
> > Deleting offsets is not something which should
> > be done very frequently and should be handled
> > with care
>
> > Agreed this involves some toil but it's not something
> > that should be done on a very regular basis.
>
> I'm not sure I follow how we came to this conclusion, could you please
> expand on the pitfalls of allowing connector plugins to wipe the offsets
> for source partitions that they no longer care about?
>
> > The usecases you highlighted are edge cases at
> > best. As I have been saying, if it is needed we can
> > always add it in the future but that doesn't look like
> > a problem we need to solve upfront.
>
> I agree that these cases might not be too common, but I'm just trying to
> understand the reasoning behind preventing this use case since null offsets
> don't require any separate handling from the Connect runtime's point of
> view (and wouldn't need any additional implementation work in this KIP).
>
> Thanks,
> Yash
>
> [1] - https://kafka.apache.org/documentation/#connect_running
> [2] -
>
> https://github.com/apache/kafka/blob/0912ca27e2a229d2ebe02f4d1dabc40ed5fab0bb/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaTopicBasedBackingStore.java#L47
>
> On Mon, Aug 28, 2023 at 1:38 PM Sagar <sa...@gmail.com> wrote:
>
> > Hey Yash,
> >
> > Thanks for your further comments. Here are my responses:
> >
> > 1) Deleting offsets via updateOffsets.
> >
> > Hmm, I am not sure this is really necessary to be part of the KIP at this
> > point, and we can always add it later on if needed. I say this for the
> > following reasons:
> >
> >
> >    - The size of offsets topic can be controlled by setting appropriate
> >    topic retention values and that is a standard practice in Kafka. Sure
> > it's
> >    not always possible to get the right values but as I said it is a
> > standard
> >    practice. For Connect specifically, there is also a KIP (KIP-943
> >    <
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255073470
> > >)
> >    which is trying to solve the problem of a large connect-offsets topic.
> > So,
> >    if that is really the motivation, then these are being addressed
> > separately
> >    anyways.
> >    - Deleting offsets is not something which should be done very
> frequently
> >    and should be handled with care. That is why KIP-875's mechanism to
> have
> >    users/ cluster admin do this externally is the right thing to do.
> Agreed
> >    this involves some toil but it's not something that should be done on
> a
> >    very regular basis.
> >    - There is no stopping connector implementations to send tombstone
> >    records as offsets but in practice how many connectors actually do it?
> >    Maybe 1 or 2 from what we discussed.
> >    - The usecases you highlighted are edge cases at best. As I have been
> >    saying, if it is needed we can always add it in the future but that
> > doesn't
> >    look like a problem we need to solve upfront.
> >
> > Due to these reasons, I don't think this is a point that we need to
> stress
> > so much upon. I say this because offsets topic's purging/clean up can be
> > handled either via standard Kafka techniques (point #1 above) or via
> > Connect runtime techniques (Pt #2  above). IMO the problem we are trying
> to
> > solve via this KIP has been solved by connectors using techniques which
> > have been termed as having higher maintenance cost or a high cognitive
> load
> > (i.e separate topic) and that needs to be addressed upfront. And since
> you
> > yourself termed it as a nice to have feature, we can leave it to that and
> > take it up as Future Work. Hope that's ok with you and other community
> > members.
> >
> > 2) Purpose of offsets parameter in updateOffsets
> >
> > The main purpose is to provide the task with the visibility into what
> > partitions are getting their offsets committed. It is not necessary that
> a
> > task might choose to update offsets everytime it sees that a given source
> > partition is missing from the about to be committed offsets. Maybe it
> > chooses to wait for some X iterations or X amount of time and send out an
> > updated offset for a partition only when such thresholds are breached.
> Even
> > here we could argue that since it's sending the partition/offsets it can
> do
> > the tracking on it's own, but IMO that is too much work given that the
> > information is already available via offsets to be committed.
> >
> > Thanks!
> > Sagar.
> >
>