Thanks Yash,

Yes , I think we can use @JsonInclude(JsonInclude.Include.NON_NULL) to
exclude “initial_offsets_response” from the create response if offset is
not specified.

I’ll close the voting this week , if there are no further comments.

Thanks for voting, everyone!


Ashwin

On Tue, Mar 5, 2024 at 11:20 PM Yash Mayya <yash.ma...@gmail.com> wrote:

> Hi Chris,
>
> I followed up with Ashwin offline and I believe he wanted to take a closer
> look at the `ConnectorInfoWithInitialOffsetsResponse` stuff he mentioned in
> the previous email and whether or not that'll be required (alternatively
> using some Jackson JSON tricks). However, that's an implementation detail
> and shouldn't hold up the KIP. Bikeshedding a little on the
> "initial_offsets_response" field - I'm wondering if something like
> "offsets_status" might be more appropriate, what do you think? I don't
> think the current name is terrible though, so I'm +1 (binding) if everyone
> else agrees that it's suitable.
>
> Thanks,
> Yash
>
> On Tue, Mar 5, 2024 at 9:51 PM Chris Egerton <chr...@aiven.io.invalid>
> wrote:
>
> > Hi all,
> >
> > Wanted to bump this and see if it looks good enough for a third vote.
> Yash,
> > any thoughts?
> >
> > Cheers,
> >
> > Chris
> >
> > On Mon, Jan 29, 2024 at 2:55 AM Ashwin <apan...@confluent.io.invalid>
> > wrote:
> >
> > > Thanks for reviewing this KIP,  Yash.
> > >
> > > Could you please elaborate on the cleanup steps? For instance, if we
> > > > encounter an error after wiping existing offsets but before writing
> the
> > > new
> > > > offsets, there's not really any good way to "revert" the wiped
> offsets.
> > > > It's definitely extremely unlikely that a user would expect the
> > previous
> > > > offsets for a connector to still be present (by creating a new
> > connector
> > > > with the same name but without initial offsets for instance) after
> > such a
> > > > failed operation, but it would still be good to call this out
> > > explicitly. I
> > > > presume that we'd want to wipe the newly written initial offsets if
> we
> > > fail
> > > > while writing the connector's config however?
> > >
> > >
> > > Agree - I have clarified the cleanup here -
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-995%3A+Allow+users+to+specify+initial+offsets+while+creating+connectors#KIP995:Allowuserstospecifyinitialoffsetswhilecreatingconnectors-ProposedChanges
> > > .
> > >
> > > The `PATCH /connectors/{connector}/offsets` and `DELETE
> > > > /connectors/{connector}/offsets` endpoints have two possible success
> > > > messages in the response depending on whether or not the connector
> > plugin
> > > > has implemented the `alterOffsets` connector method. Since we're
> > > proposing
> > > > to utilize the same offset validation during connector creation if
> > > initial
> > > > offsets are specified, I think it would be valuable to surface
> similar
> > > > information to users here as well
> > >
> > >
> > > Thanks for pointing this out. I have updated the response to include a
> > new
> > > field “initial_offsets_response” which will contain the response based
> on
> > > whether the connector implements alterOffsets or not. This also means
> > that
> > > if initial_offsets is set in the ConnectorCreate request, we will
> return
> > a
> > > new REST entity (ConnectorInfoWithInitialOffsetsResponse ?) which will
> > be a
> > > child class of ConnectorInfo.
> > >
> > > (
> > >
> > >
> >
> https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java#L28-L28
> > > )
> > >
> > > Thanks,
> > > Ashwin
> > >
> > > On Wed, Jan 17, 2024 at 4:48 PM Yash Mayya <yash.ma...@gmail.com>
> wrote:
> > >
> > > > Hi Ashwin,
> > > >
> > > > Thanks for the KIP.
> > > >
> > > > > If Connect runtime encounters an error in any of these steps,
> > > > > it will cleanup (if required) and return an error response
> > > >
> > > > Could you please elaborate on the cleanup steps? For instance, if we
> > > > encounter an error after wiping existing offsets but before writing
> the
> > > new
> > > > offsets, there's not really any good way to "revert" the wiped
> offsets.
> > > > It's definitely extremely unlikely that a user would expect the
> > previous
> > > > offsets for a connector to still be present (by creating a new
> > connector
> > > > with the same name but without initial offsets for instance) after
> > such a
> > > > failed operation, but it would still be good to call this out
> > > explicitly. I
> > > > presume that we'd want to wipe the newly written initial offsets if
> we
> > > fail
> > > > while writing the connector's config however?
> > > >
> > > > > Validate the offset using the same checks performed while
> > > > > altering connector offsets (PATCH /$connector/offsets ) as
> > > > > specified in KIP-875
> > > >
> > > > The `PATCH /connectors/{connector}/offsets` and `DELETE
> > > > /connectors/{connector}/offsets` endpoints have two possible success
> > > > messages in the response depending on whether or not the connector
> > plugin
> > > > has implemented the `alterOffsets` connector method. Since we're
> > > proposing
> > > > to utilize the same offset validation during connector creation if
> > > initial
> > > > offsets are specified, I think it would be valuable to surface
> similar
> > > > information to users here as well. Thoughts?
> > > >
> > > > Thanks,
> > > > Yash
> > > >
> > > > On Wed, Jan 17, 2024 at 3:31 PM Ashwin <apan...@confluent.io.invalid
> >
> > > > wrote:
> > > >
> > > > > Hi All ,
> > > > >
> > > > > Can I please get one more binding vote, so that the KIP is
> approved ?
> > > > > Thanks for the votes Chris and Mickael !
> > > > >
> > > > >
> > > > > - Ashwin
> > > > >
> > > > >
> > > > > On Thu, Jan 11, 2024 at 3:55 PM Mickael Maison <
> > > mickael.mai...@gmail.com
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Hi Ashwin,
> > > > > >
> > > > > > +1 (binding), thanks for the KIP
> > > > > >
> > > > > > Mickael
> > > > > >
> > > > > > On Tue, Jan 9, 2024 at 4:54 PM Chris Egerton
> > <chr...@aiven.io.invalid
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > Thanks for the KIP! +1 (binding)
> > > > > > >
> > > > > > > On Mon, Jan 8, 2024 at 9:35 AM Ashwin
> > <apan...@confluent.io.invalid
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hi All,
> > > > > > > >
> > > > > > > > I would like to start  a vote on KIP-995.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-995%3A+Allow+users+to+specify+initial+offsets+while+creating+connectors
> > > > > > > >
> > > > > > > > Discussion thread -
> > > > > > > >
> > https://lists.apache.org/thread/msorbr63scglf4484yq764v7klsj7c4j
> > > > > > > >
> > > > > > > > Thanks!
> > > > > > > >
> > > > > > > > Ashwin
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to