Re: [VOTE] KIP-995: Allow users to specify initial offsets while creating connectors

2024-03-11 Thread Ashwin
Thanks Folks, Renamed the field to `offsets_status`. And now that we have 3 binding votes, I will update the KIP status to `accepted` . Thanks again for all the valuable feedback. Ashwin On Wed, Mar 6, 2024 at 2:42 PM Chris Egerton wrote: > Hi Yash, > > Thanks for the follow-u

Re: [VOTE] KIP-995: Allow users to specify initial offsets while creating connectors

2024-03-05 Thread Ashwin
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

Re: [VOTE] KIP-995: Allow users to specify initial offsets while creating connectors

2024-01-28 Thread Ashwin
oWithInitialOffsetsResponse ?) 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 wrote: &

Re: [VOTE] KIP-995: Allow users to specify initial offsets while creating connectors

2024-01-17 Thread Ashwin
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 wrote: > Hi Ashwin, > > +1 (binding), thanks for the KIP > > Mickael > > On Tue, Jan 9,

[VOTE] KIP-995: Allow users to specify initial offsets while creating connectors

2024-01-08 Thread Ashwin
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

Re: [DISCUSS] KIP-995: Allow users to specify initial offsets while creating connectors

2024-01-03 Thread Ashwin
Thanks Chris, It makes sense to call out the cleaning up of existing offsets in the KIP. I have made the change and will initiate the voting in a day. Regards, Ashwin On Tue, Dec 12, 2023 at 6:57 PM Chris Egerton wrote: > Hi Ashwin, > > LGTM! One small adjustment I'd suggest but we d

Re: [DISCUSS] KIP-995: Allow users to specify initial offsets while creating connectors

2023-12-11 Thread Ashwin
Thanks for pointing this out Chris. I have updated the KIP with the correct sequence of steps. Thanks, Ashwin On Wed, Dec 6, 2023 at 11:48 PM Chris Egerton wrote: > Hi Ashwin, > > Regarding point 4--I think this is still a bit unwise. When workers pick up > a new connector

Re: [DISCUSS] KIP-995: Allow users to specify initial offsets while creating connectors

2023-12-06 Thread Ashwin
the necessary changes. Thanks again, Ashwin On Mon, Dec 4, 2023 at 9:05 PM Chris Egerton wrote: > Hi Ashwin, > > Thanks for the KIP! This would be a nice simplification to the process for > migrating connectors enabled by KIP-980, and would also add global support > for a fe

[jira] [Created] (KAFKA-15976) KIP-995: Allow users to specify initial offsets while creating connectors

2023-12-06 Thread Ashwin Pankaj (Jira)
Ashwin Pankaj created KAFKA-15976: - Summary: KIP-995: Allow users to specify initial offsets while creating connectors Key: KAFKA-15976 URL: https://issues.apache.org/jira/browse/KAFKA-15976 Project

[DISCUSS] KIP-995: Allow users to specify initial offsets while creating connectors

2023-11-29 Thread Ashwin
IP-875%3A+First-class+offsets+support+in+Kafka+Connect#KIP875:FirstclassoffsetssupportinKafkaConnect-Alteringoffsets(request)> and KIP-980 <https://cwiki.apache.org/confluence/display/KAFKA/KIP-980%3A+Allow+creating+connectors+in+a+stopped+state>. Thanks, Ashwin

Re: [DISCUSS] KIP-1006: Remove SecurityManager Support

2023-11-20 Thread Ashwin
” option of Javac such that the resulting JARs will not load in JVMs which are lesser than or equal to that version ? Thanks, Ashwin On Tue, Nov 21, 2023 at 6:18 AM Greg Harris wrote: > Hi all, > > I'd like to invite you all to discuss removing SecurityManager support > from Kafka.

Re: [DISCUSS] KIP-967: Support custom SSL configuration for Kafka Connect RestServer

2023-10-09 Thread Ashwin
t creates the SslEngineFactory. Yes, I think this class should be moved to something like `server-common` module - but would like any of the committers to comment on this. Thanks, Ashwin On Fri, Sep 29, 2023 at 9:22 PM Taras Ledkov wrote: > Hi Ashwin, > > Thanks a lot for your re

Re: [DISCUSS] KIP-967: Support custom SSL configuration for Kafka Connect RestServer

2023-09-28 Thread Ashwin
/apache/kafka/pull/14203/, why did you have to remove the code which sets sslEngineFactoryConfig in instantiateSslEngineFactory ? Thanks, Ashwin On Tue, Sep 26, 2023 at 6:39 PM Taras Ledkov wrote: > Hi Kafka Team. > > Ping... >

Re: [ANNOUNCE] New committer: Yash Mayya

2023-09-21 Thread Ashwin
Awesome ! Congratulations Yash !! On Thu, Sep 21, 2023 at 9:25 PM Edoardo Comar wrote: > Congratulations Yash > > On Thu, 21 Sept 2023 at 16:28, Bruno Cadonna wrote: > > > > Hi all, > > > > The PMC of Apache Kafka is pleased to announce a new Kafka committer > > Yash Mayya. > > > > Yash's

Re: [DISCUSS] KIP-980: Allow creating connectors in a stopped state

2023-09-20 Thread Ashwin
? 3. In the case of a downgrade, how will Connect worker handle the optional “state” field in config topic ? Thanks, Ashwin On Sun, Sep 17, 2023 at 11:09 PM Yash Mayya wrote: > Hi all, > > I'd like to begin discussion on a KIP to allow creating connectors in a > stopped state

Re: KIP-976: Cluster-wide dynamic log adjustment for Kafka Connect

2023-09-06 Thread Ashwin
t know that system tests are more "heavy weight" than integration tests - so thanks for letting me know Yash & Chris. Cheers, Ashwin On Tue, Sep 5, 2023 at 8:51 PM Chris Egerton wrote: > Hi all, > > Thank you so much for the generous review comments! Happy to see interest > i

Re: KIP-976: Cluster-wide dynamic log adjustment for Kafka Connect

2023-09-03 Thread Ashwin
nse contains the last modified time. Thanks, Ashwin On Sat, Sep 2, 2023 at 5:12 AM Chris Egerton wrote: > Hi all, > > Can't imagine a worse time to publish a new KIP (it's late on a Friday and > we're in the middle of the 3.6.0 release), but I wanted to put forth > KIP-976 for dis

[jira] [Resolved] (KAFKA-14598) Fix flaky ConnectRestApiTest

2023-08-04 Thread Ashwin Pankaj (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashwin Pankaj resolved KAFKA-14598. --- Resolution: Fixed Did not observe this recently > Fix flaky ConnectRestApiT

Re: [VOTE] KIP-882: Kafka Connect REST API timeout improvements

2023-03-01 Thread Ashwin
the idea of having smaller timeout values and the unit as part of the value eg: '2m', '10s' or '500ms'. Is it too late to incorporate this change? Thanks, Ashwin On Wed, Mar 1, 2023 at 6:32 PM Yash Mayya wrote: > Hi all, > > I'd like to call for a vote on the (hopefully) straightforwar

[jira] [Created] (KAFKA-14598) Fix flaky ConnectRestApiTest

2023-01-06 Thread Ashwin Pankaj (Jira)
Ashwin Pankaj created KAFKA-14598: - Summary: Fix flaky ConnectRestApiTest Key: KAFKA-14598 URL: https://issues.apache.org/jira/browse/KAFKA-14598 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2022-11-22 Thread Ashwin
uot; } }, "required": [ "name", "schema_version", "iam_role" ] } The connector which supports Fictional config schema 2.0 will validate the access key and secret key. Whereas a connector which supports config with schema version 3.0 wil

Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2022-11-21 Thread Ashwin
e "boolean migrate(Version fromVersion)" to the connector interface. Thanks, Ashwin On Mon, Nov 21, 2022 at 2:27 PM Snehashis wrote: > Hi all, > > I'd like to start a discussion thread on KIP-891: Running multiple versions > of a connector. > > The KIP aims to a

Re: [DISCUSS] KIP-875: First-class offsets support in Kafka Connect

2022-10-13 Thread Ashwin
like shift-by , to-date-time etc. using a REST API like DELETE /connectors/{connector}/offsets ? 3. Today PAUSE operation on a connector invokes its stop method - will there be a change here to reduce confusion with the new proposed STOPPED state ? Thanks, Ashwin On Fri, Oct 14, 2022 at 2:22 AM

Re: [DISCUSS] KIP-850: REST API for filtering Connector plugins by type

2022-06-24 Thread Ashwin
No worries Chris - I understand what you are getting at. I will pursue updating the current REST API. Cheers, Ashwin On Fri, Jun 24, 2022 at 8:57 AM Chris Egerton wrote: > Hi Ashwin, > > I just want to be clear that I do think there might be some value to being > able to addr

[jira] [Resolved] (KAFKA-4279) REST endpoint to list converter plugins

2022-06-23 Thread Ashwin Pankaj (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4279?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashwin Pankaj resolved KAFKA-4279. -- Resolution: Won't Do Obsolete - https://lists.apache.org/thread

Re: [DISCUSS] KIP-850: REST API for filtering Connector plugins by type

2022-06-23 Thread Ashwin
Hi Chris, Sure, if JIRA is obsolete and does not add value, I am fine with abandoning this change. I will close the JIRA as well. Thanks for helping out! Ashwin On Fri, Jun 24, 2022 at 4:23 AM Chris Egerton wrote: > Hi Ashwin, > > Thanks for the clarification, which is certain

Re: [DISCUSS] KIP-850: REST API for filtering Connector plugins by type

2022-06-22 Thread Ashwin
her for HeaderConverter or Converter plugin types. Thanks, Ashwin On Thu, Jun 23, 2022 at 12:03 AM Chris Egerton wrote: > Hi Ashwin, > > Can you provide a use case for this new behavior? It seems pretty > straightforward to get this via client-side filtering, I'm a little > skep

[DISCUSS] KIP-850: REST API for filtering Connector plugins by type

2022-06-22 Thread Ashwin
/jira/browse/KAFKA-4279 was for an endpoint to list converter plugins, I propose that we implement a generic "filter" API which will list connector plugins matched by type (one of SOURCE,SINK,CONVERTER,HEADER_CONVERTER). Thanks, Ashwin

Requesting permissions to contribute to Apache Kafka

2022-06-20 Thread Ashwin
Hello, Please approve my request for permissions to contribute. wiki id : ashwin pankaj jira id: ashwinpankaj Thanks, Ashwin

Re: [VOTE] KIP-162: Enable topic deletion by default

2017-06-06 Thread Ashwin Sinha
Hi, > >> > > >> > The discussion has been quite positive, so I posted a JIRA, a PR and > >> > updated the KIP with the latest decisions. > >> > > >> > Lets officially vote on the KIP: > >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > >> > 162+-+Enable+topic+deletion+by+default > >> > > >> > JIRA is here: https://issues.apache.org/jira/browse/KAFKA-5384 > >> > > >> > Gwen > >> > > >> > -- Thanks and Regards, Ashwin

[jira] [Commented] (KAFKA-1716) hang during shutdown of ZookeeperConsumerConnector

2015-03-18 Thread Ashwin Jayaprakash (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14367361#comment-14367361 ] Ashwin Jayaprakash commented on KAFKA-1716: --- Sorry for the delay, I will get

[jira] [Updated] (KAFKA-1716) hang during shutdown of ZookeeperConsumerConnector

2015-03-12 Thread Ashwin Jayaprakash (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashwin Jayaprakash updated KAFKA-1716: -- Attachment: after-shutdown.log before-shutdown.log [~becket_qin] Here

[jira] [Commented] (KAFKA-1716) hang during shutdown of ZookeeperConsumerConnector

2015-03-10 Thread Ashwin Jayaprakash (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14356326#comment-14356326 ] Ashwin Jayaprakash commented on KAFKA-1716: --- I'm beginning to wonder

[jira] [Updated] (KAFKA-1716) hang during shutdown of ZookeeperConsumerConnector

2015-03-10 Thread Ashwin Jayaprakash (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashwin Jayaprakash updated KAFKA-1716: -- Attachment: kafka-shutdown-stuck.log hang during shutdown