Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-08-22 Thread Arjun Satish
Hello Artem, Thanks for the KIP. I have the same question as Roger on concurrent writes, and an additional one on consumer behavior. Typically, transactions will timeout if not committed within some time interval. With the proposed changes in this KIP, consumers cannot consume past the ongoing

Re: [ANNOUNCE] New Committer: Chris Egerton

2022-07-26 Thread Arjun Satish
Many congrats Chris! On Tue, Jul 26, 2022 at 4:02 AM Bruno Cadonna wrote: > Congratulations Chris! Well deserved! > > Best, > Bruno > > On 26.07.22 05:24, Kumud Kumar Srivatsava Tirupati wrote: > > Congratulations Chris! > > > > On Tue, 26 Jul, 2022, 7:11 AM deng ziming, > wrote: > > > >>

Re: KIP-769: Connect API to retrieve connector configuration definitions

2021-12-04 Thread Arjun Satish
hey folks, Thanks a lot for the KIP and the discussions. Here are a couple of thoughts (apologies if I missed some point in this thread). 1. it seems like we are changing the behavior of /connector-plugins by now returning the list of transforms, converters as well? This would break backward

Re: [VOTE] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-16 Thread Arjun Satish
+1 (non-binding). Thanks for the KIP, Knowles! and appreciate the follow-ups! On Thu, Nov 11, 2021 at 2:55 PM John Roesler wrote: > Thanks, Knowles! > > I'm +1 (binding) > > -John > > On Wed, 2021-11-10 at 12:42 -0500, Christopher Shannon > wrote: > > +1 (non-binding). This looks good to me and

Re: [DISCUSS] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-16 Thread Arjun Satish
One more nit: the RetryWithToleranceOperator class is not a public interface. So we do not have to call the changes in them out in the Public Interfaces section. On Tue, Nov 16, 2021 at 10:42 AM Arjun Satish wrote: > Chris' point about upgrades is valid. An existing configuration will

Re: [DISCUSS] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-16 Thread Arjun Satish
; > > katchiso...@gmail.com > > > > > > > wrote: > > > > > > > Third time's the charm. > > > > > > > > I've added a getter for the RetryWithToleranceOperator to get the > > > > ToleranceType. I've updated

Re: [DISCUSS] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-01 Thread Arjun Satish
nds of the connector developer. I suppose that is for the best, in a > vacuum only the worker should have a say in how it handles message > production. > > Additional thoughts and feedback are welcome. > > Knowles > > On Thu, Oct 28, 2021 at 10:54 AM Arjun Satish > wr

Re: [DISCUSS] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-10-28 Thread Arjun Satish
ritten, but > looking at SourceTask more closely, in commitRecord(SourceRecord, > RecordMetadata), the RecordMetadata is set to null in the event of a > filtered transformation so the framework is already doing this in a certain > regard. > > Knowles > > On Thu, Oct 28, 2021 at

Re: [DISCUSS] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-10-28 Thread Arjun Satish
need some kind of callback from inside the connector with the > Source Record to successfully ack back to my source system. > > I have updated the KIP regarding the callback being executed in a different > thread than poll(). > > Knowles > > On Thu, Oct 28, 2021 at 2:02 AM Ar

Re: [DISCUSS] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-10-28 Thread Arjun Satish
Hi Knowles, Thanks for the KIP! Could you please call out some use-cases on what the source connectors would do when they hit such exceptions? I'm wondering if we would need to do anything other than skipping such records, writing some log messages, and/or writing some error context to a DLQ?

[jira] [Created] (KAFKA-10387) Cannot include SMT configs with source connector that include topic.creation.* properties

2020-08-11 Thread Arjun Satish (Jira)
Arjun Satish created KAFKA-10387: Summary: Cannot include SMT configs with source connector that include topic.creation.* properties Key: KAFKA-10387 URL: https://issues.apache.org/jira/browse/KAFKA-10387

[jira] [Created] (KAFKA-10340) Source connectors should report error when trying to producer records to non-existent topics instead of hanging forever

2020-08-03 Thread Arjun Satish (Jira)
Arjun Satish created KAFKA-10340: Summary: Source connectors should report error when trying to producer records to non-existent topics instead of hanging forever Key: KAFKA-10340 URL: https://issues.apache.org

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-19 Thread Arjun Satish
ehose", the task needs to pause all partitions? On Tue, May 19, 2020 at 9:26 AM Arjun Satish wrote: > Can we get a couple of examples that shows utility of waiting on the > Future<>? Also, in preCommit() we would report back on the incomplete > offsets. So that feedback mech

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-19 Thread Arjun Satish
wrote: > Thanks, Aakash, for updating the KIP. > > On Tue, May 19, 2020 at 2:18 AM Arjun Satish > wrote: > > > Hi Randall, > > > > Thanks for the explanation! Excellent point about guaranteeing offsets in > > the async case. > > > > If we c

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-19 Thread Arjun Satish
; > > > reporting > > > > > > of errant records instead of accepting a single record at a time; > > the > > > > > task > > > > > > would track errant records as it processes them in "put" and > report > > >

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-17 Thread Arjun Satish
Thanks for all the feedback, folks. re: having a callback as a parameter, I agree that at this point, it might not add much value to the proposal. re: synchronous vs asynchronous, is the motivation performance/higher throughput? Taking a step back, calling report(..) in the new interface does a

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-16 Thread Arjun Satish
or it should expect? Is there any case when the reporter will be null > or be a no-op, and if so what should the sink task do? Should it simply > wrap and throw a ConnectException? And if there is a reporter, won't > Connect treat this sink record as "processed" with respect to the exist

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-16 Thread Arjun Satish
; > of > > > > > those changes on the KIP. Aakash, can you please update the KIP > > quickly > > > > so > > > > > we can make sure the other parts are the KIP are acceptable? > > > > > > > > > > Best regards, > > > > > >

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-16 Thread Arjun Satish
ility section in this KIP as well. > > > > > > Regards, > > > Konstantine > > > > > > On Sat, May 16, 2020 at 10:13 AM Aakash Shah > wrote: > > > > > > > +1 > > > > > > > > On Sat, May 16, 2020 at 9:55 AM Konstan

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-15 Thread Arjun Satish
; soon > > due to the AK 2.6 deadlines; I will then shortly update the KIP and > start a > > vote. > > > > Thanks, > > Aakash > > > > On Fri, May 15, 2020 at 2:24 PM Randall Hauch wrote: > > > >> On Fri, May 15, 2020 at 3:13 PM Arj

Re: [DISCUSS] KIP-610: Error Reporting in Sink Connectors

2020-05-15 Thread Arjun Satish
Couple of thoughts: 1. If we add new parameters to put(..), and new connectors implement only this method, it makes them backward incompatible with older workers. I think newer connectors may only choose to only implement the latest method, and we are passing the compatibility problems back to

Re: [VOTE] KIP-495: Dynamically Adjust Log Levels in Connect

2019-09-17 Thread Arjun Satish
gt; Konstantine > > > > > > > > > > > > On Wed, Aug 14, 2019 at 12:13 AM Cyrus Vafadari > > > > > wrote: > > > > > > > >> I am excited to see this implemented +1 nonbinding > > > >> > > >

Re: [VOTE] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-09-17 Thread Arjun Satish
Thanks for the KIP, Almog. +1 (non-binding) On Mon, Sep 16, 2019 at 4:45 PM Randall Hauch wrote: > Thanks for the nice improvement, Almog! > > +1 (binding) > > Randall > > On Thu, Aug 15, 2019 at 11:59 AM Konstantine Karantasis < > konstant...@confluent.io> wrote: > > > Thanks Almog! > >

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-09-17 Thread Arjun Satish
his looks good! > > Best regards, > > Randall > > On Mon, Sep 16, 2019 at 4:15 AM Arjun Satish > wrote: > > > Good catch, Randall. Yes, we should be able to set the level of any > logger > > given its name. If this is an ancestor, then the levels of all child >

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-09-16 Thread Arjun Satish
n Thu, Sep 12, 2019 at 9:58 AM Gwen Shapira wrote: > > > > > The new REST API for logger management looks great to me. > > > > > > > > > On Thu, Sep 12, 2019 at 8:36 AM Arjun Satish > > > wrote: > > > > > > > > Bumping this thread. > &

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-09-12 Thread Arjun Satish
Bumping this thread. If there are no further comments, please add your votes here: https://www.mail-archive.com/dev@kafka.apache.org/msg100313.html Thanks in advance, Arjun On Fri, Sep 6, 2019 at 4:22 PM Arjun Satish wrote: > Thanks a lot, Jason! Answers inline. I'll also modify the

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-09-06 Thread Arjun Satish
not be persistent and only apply to the worker that received the request. > Thanks, > Jason > > On Fri, Aug 30, 2019 at 1:25 AM Arjun Satish > wrote: > > > OK. I didn't realize the plan was to deprecate and remove the JMX > endpoint. > > KIP-412 says brokers will contin

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-08-30 Thread Arjun Satish
of > > an admin endpoint now so that we're not left with odd compatibility > baggage > > in the future. > > Hi Jason, > > I agree... I think Connect needs a REST admin API. There will probably be > a lot of other stuff that we'll want to add to it. > > best, > Colin > > >

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-08-23 Thread Arjun Satish
rld that wants to use JMX if they don't have to? I > thought > > one of the major motivations of KIP-412 was how much of a pain JMX is. > > > > Thanks, > > Jason > > > > On Mon, Aug 19, 2019 at 5:28 PM Arjun Satish > > wrote: > > > >> Thanks

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-08-19 Thread Arjun Satish
he restrictions around log4j is that this is > information is significant and IMO needs to be included in the KIP. > > Speaking of its relevance to KIP-412, I think a reference would be nice > too. > > Konstantine > > > > On Thu, Aug 15, 2019 at 4:00 PM Arjun Satish &

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-08-15 Thread Arjun Satish
> wrote: > > > >> This looks like a useful feature, the strategy makes sense, and the KIP > is > >> thorough and nicely written. Thanks! > >> > >> Cyrus > >> > >> On Thu, Aug 1, 2019, 12:40 PM Chris Egerton > wrote: &g

[VOTE] KIP-495: Dynamically Adjust Log Levels in Connect

2019-08-13 Thread Arjun Satish
Hey everyone, I'd like to start a vote for KIP-495 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-495%3A+Dynamically+Adjust+Log+Levels+in+Connect). This change will make Connect easier to debug in production environment. Based on the discussion, I updated the KIP to reflect how Connect

Re: [DISCUSS] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-08-08 Thread Arjun Satish
I will add that documentation to the serialization config. > > > > Note that there would not be an issue on the _serialization_ side of > > things as Jackson respects BigDecimal. > > > > Almog > > > > On Tue, Aug 6, 2019 at 11:23 PM Arjun Satish > > w

Re: [DISCUSS] KIP-481: SerDe Improvements for Connect Decimal type in JSON

2019-08-07 Thread Arjun Satish
hey Almog, nice work! couple of thoughts (hope I'm not late since you started the voting thread already): can you please add some examples to show the changes that you are proposing. makes me think that for a given decimal number, we will have two encodings: “asHex” and “asNumber”. should we

Re: [DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-08-01 Thread Arjun Satish
use that new utility. Is the > "Example Usage" section (which involves invoking the utility with a > namespace of "kafka.connect") actually meant to be part of the proposed > changes to public interface? > > Cheers, > > Chris > > On Mon, Jul 22, 2019 at

[DISCUSS] KIP-495: Dynamically Adjust Log Levels in Connect

2019-07-23 Thread Arjun Satish
Hi everyone. I'd like to propose the following KIP to implement changing log levels on the fly in Connect workers: https://cwiki.apache.org/confluence/display/KAFKA/KIP-495%3A+Dynamically+Adjust+Log+Levels+in+Connect Would like to hear your thoughts on this. Thanks very much, Arjun

Re: [VOTE] KIP-411: Make default Kafka Connect worker task client IDs distinct

2019-05-13 Thread Arjun Satish
. Thanks very much and apologies for the confusion! Best, On Mon, May 6, 2019 at 10:33 AM Paul Davidson wrote: > Thanks Arjun. I've updated the KIP using your suggestion - just a few > slight changes. > > On Fri, May 3, 2019 at 4:48 PM Arjun Satish > wrote: > > > Maybe w

Re: [VOTE] KIP-411: Make default Kafka Connect worker task client IDs distinct

2019-05-03 Thread Arjun Satish
> > Thanks, > > Paul > > On Thu, May 2, 2019 at 5:36 PM Arjun Satish > wrote: > > > Paul, > > > > You might want to make a note on the KIP regarding the impact on quotas. > > > > Thanks, > > > > On Thu, May 2, 2019 at 9:48 AM Paul D

Re: [VOTE] KIP-411: Make default Kafka Connect worker task client IDs distinct

2019-05-02 Thread Arjun Satish
yanne, Arjun, Magesh) > > Regards, > > Paul > > On Wed, May 1, 2019 at 10:07 PM Arjun Satish > wrote: > > > Good point, Gwen. We always set a non empty value for client id: > > > > > https://github.com/apache/kafka/blob/2.2.0/clients/src/main/java/

Re: [VOTE] KIP-411: Make default Kafka Connect worker task client IDs distinct

2019-05-01 Thread Arjun Satish
t to enforce? > So setting the client name will only change something if there's already a > quota for that client? > > On the other hand, I fully support switching to "easy-to-wildcard" template > for the client id. > > On Wed, May 1, 2019 at 8:50 PM Arjun Satish

Re: [VOTE] KIP-411: Make default Kafka Connect worker task client IDs distinct

2019-05-01 Thread Arjun Satish
I just realized that setting the client.id on the will now trigger any quota restrictions ( https://kafka.apache.org/documentation/#design_quotasconfig) on the broker. It seems like this PR will enforce quota policies that will either require admins to set limits for each task (since the chosen

Re: [VOTE] KIP-411: Make default Kafka Connect worker task client IDs distinct

2019-04-29 Thread Arjun Satish
Thanks, Paul! This is very useful. +1 (non-binding) Best, Arjun On Fri, Apr 12, 2019 at 4:13 PM Ryanne Dolan wrote: > +1 (non binding) > > Thanks > Ryanne > > On Fri, Apr 12, 2019, 11:11 AM Paul Davidson > wrote: > > > Just a reminder that KIP-411 is open for voting. No votes received yet! >

[jira] [Resolved] (KAFKA-7877) Connect DLQ not used in SinkTask put()

2019-04-28 Thread Arjun Satish (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7877?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Arjun Satish resolved KAFKA-7877. - Resolution: Fixed Updated KIP. > Connect DLQ not used in SinkTask

[jira] [Resolved] (KAFKA-7999) Flaky Test ExampleConnectIntegrationTest#testProduceConsumeConnector

2019-02-26 Thread Arjun Satish (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7999?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Arjun Satish resolved KAFKA-7999. - Resolution: Fixed PR: https://github.com/apache/kafka/pull/6326 > Flaky T

[jira] [Created] (KAFKA-7909) Coordinator changes cause Connect integration test to fail

2019-02-08 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7909: --- Summary: Coordinator changes cause Connect integration test to fail Key: KAFKA-7909 URL: https://issues.apache.org/jira/browse/KAFKA-7909 Project: Kafka Issue

[jira] [Created] (KAFKA-7772) Dynamically adjust log level in Connect workers

2018-12-27 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7772: --- Summary: Dynamically adjust log level in Connect workers Key: KAFKA-7772 URL: https://issues.apache.org/jira/browse/KAFKA-7772 Project: Kafka Issue Type

[jira] [Created] (KAFKA-7503) Integration Tests for Connect

2018-10-13 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7503: --- Summary: Integration Tests for Connect Key: KAFKA-7503 URL: https://issues.apache.org/jira/browse/KAFKA-7503 Project: Kafka Issue Type: Task

[jira] [Created] (KAFKA-7228) DeadLetterQueue throws a NullPointerException

2018-08-01 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7228: --- Summary: DeadLetterQueue throws a NullPointerException Key: KAFKA-7228 URL: https://issues.apache.org/jira/browse/KAFKA-7228 Project: Kafka Issue Type: Task

[jira] [Created] (KAFKA-7003) Add headers with error context in messages written to the Connect DeadLetterQueue topic

2018-06-05 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7003: --- Summary: Add headers with error context in messages written to the Connect DeadLetterQueue topic Key: KAFKA-7003 URL: https://issues.apache.org/jira/browse/KAFKA-7003

[jira] [Created] (KAFKA-7002) Allow replication factor to be set via a configuration property for the Connect DLQ topic

2018-06-05 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7002: --- Summary: Allow replication factor to be set via a configuration property for the Connect DLQ topic Key: KAFKA-7002 URL: https://issues.apache.org/jira/browse/KAFKA-7002

[jira] [Created] (KAFKA-7001) Rename `errors.allowed.max` in Connect to `errors.tolerance`

2018-06-05 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-7001: --- Summary: Rename `errors.allowed.max` in Connect to `errors.tolerance` Key: KAFKA-7001 URL: https://issues.apache.org/jira/browse/KAFKA-7001 Project: Kafka

[jira] [Created] (KAFKA-6981) Missing Connector Config (errors.deadletterqueue.topic.name) kills Connect Clusters

2018-06-01 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-6981: --- Summary: Missing Connector Config (errors.deadletterqueue.topic.name) kills Connect Clusters Key: KAFKA-6981 URL: https://issues.apache.org/jira/browse/KAFKA-6981

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-25 Thread Arjun Satish
to the KIP in the section here, and let me know what you think: https://cwiki.apache.org/confluence/display/KAFKA/KIP- 298%3A+Error+Handling+in+Connect#KIP-298:ErrorHandlinginConnect- DeadLetterQueue(forSinkConnectorsonly) Thanks very much, On Tue, May 22, 2018 at 2:37 PM, Arjun Satish <arjun.

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-22 Thread Arjun Satish
;ja...@confluent.io> wrote: > +1. Thanks for the KIP! > > On Mon, May 21, 2018 at 1:34 PM, Arjun Satish <arjun.sat...@gmail.com> > wrote: > > > All, > > > > Thanks so much for your feedback on this KIP. I've made some small > > modifications today. I'll wa

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-22 Thread Arjun Satish
section to reflect this: https://cwiki.apache.org/confluence/display/KAFKA/KIP-298%3A+Error+Handling+in+Connect#KIP-298:ErrorHandlinginConnect-ProposedChanges Thanks, On Mon, May 21, 2018 at 3:12 PM, Arjun Satish <arjun.sat...@gmail.com> wrote: > Hey Jason, > > This KIP does take seria

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
r, I would probably want the option to skip over processing > failures, but retry indefinitely if the downstream system is unavailable. > Is that use case supported? > > Thanks, > Jason > > > > On Mon, May 21, 2018 at 12:39 PM, Arjun Satish <arjun.sat...@gmail.com> > wrote

Re: [VOTE] KIP-298: Error Handling in Connect kafka

2018-05-21 Thread Arjun Satish
Thanks for the KIP. +1 (binding) > > >> > > >> > > >> Guozhang > > >> > > >> On Fri, May 18, 2018 at 3:36 PM, Gwen Shapira <g...@confluent.io> > > wrote: > > >> > > >>> +1 > > >>> > &

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
OK. Let's simplify tolerance to simply have NONE or ALL values. For extensions, we can open a KIP and implement in later versions. Thanks a lot! On Mon, May 21, 2018 at 1:18 PM, Ewen Cheslack-Postava <e...@confluent.io> wrote: > On Mon, May 21, 2018 at 12:39 PM Arjun Satish &

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
onnector to get it past one bad message, then reverting back > to -1 or 0. > > -Ewen > > On Mon, May 21, 2018 at 11:01 AM Arjun Satish <arjun.sat...@gmail.com> > wrote: > > > Hey Jason, > > > > Thanks for your comments. Please find answers inline: >

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-21 Thread Arjun Satish
Hey Jason, Thanks for your comments. Please find answers inline: On Mon, May 21, 2018 at 9:52 AM, Jason Gustafson wrote: > Hi Arjun, > > Thanks for the KIP. Just a few comments/questions: > > 1. The proposal allows users to configure the number of retries. I usually > find

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-18 Thread Arjun Satish
moving to the right direction. > > > > No further comments from my side. > > > > Thanks Arjun! > > > > - Konstantine > > > > On Fri, May 18, 2018 at 1:07 AM, Arjun Satish <arjun.sat...@gmail.com> > > wrote: > > > > > Ewen, > &

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-18 Thread Arjun Satish
> I think it's moving to the right direction. > > No further comments from my side. > > Thanks Arjun! > > - Konstantine > > On Fri, May 18, 2018 at 1:07 AM, Arjun Satish <arjun.sat...@gmail.com> > wrote: > > > Ewen, > > > > Thanks a lot for you

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-18 Thread Arjun Satish
data, we potentially lose raw data & schema info because we're rendering it > as JSON. Not sure that's a good idea... > > I think that last item might be the biggest concern to me -- DLQ formats > and control over content & reprocessing seems a bit unclear to me here, so &g

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-17 Thread Arjun Satish
a source connector, the process is similar, but care needs > to be taken while writing back to the source." and sounds like it's > suggested that Connect will write records back to the source, which can't > be correct. > > Finally, a nit: " adds store the row information "...

[VOTE] KIP-298: Error Handling in Connect kafka

2018-05-17 Thread Arjun Satish
All, Many thanks for all the feedback on KIP-298. Highly appreciate the time and effort you all put into it. I've updated the KIP accordingly, and would like to start to start a vote on it. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP- 298%3A+Error+Handling+in+Connect JIRA:

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-17 Thread Arjun Satish
can easily creep in, and can be notoriously hard to detect and clean up. > Thanks, > Matt > > On Tue, May 15, 2018 at 8:46 PM, Arjun Satish <arjun.sat...@gmail.com> > wrote: > > > Matt, > > > > Thanks so much for your comments. Really appreciate i

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-17 Thread Arjun Satish
sers could consume it and > take actions. > > Thanks, > Magesh > > On Wed, May 16, 2018 at 6:56 PM, Arjun Satish <arjun.sat...@gmail.com> > wrote: > > > Hi Konstantine, > > > > Thanks a lot for your feedback. I have made the necessary changes to the > >

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-16 Thread Arjun Satish
g with state, thread, jira, etc). > Now it's a bit hidden in the text and it's not clear that the KIP includes > a link to a PR. > > Thanks for working on this missing but important functionality. > > - Konstantine > > > On Tue, May 15, 2018 at 10:41 PM, Arjun Satish

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-15 Thread Arjun Satish
TimeoutExceptions (which extend RetriableExceptions) are bubbled back to the application, and need to be handled as per application requirements. Best, On Tue, May 15, 2018 at 8:30 PM, Arjun Satish <arjun.sat...@gmail.com> wrote: > Magesh, > > Thanks for the feedback! Really

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-15 Thread Arjun Satish
ctors, would I have to set it up > for each of them? I would think most people might want the behavior applied > to all the connectors. > > Let me know your thoughts :). > > Thanks > Magesh > > On Tue, May 8, 2018 at 11:59 PM, Arjun Satish <arjun.sat...@gmail.com> > wrote:

Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-15 Thread Arjun Satish
that temporarily whitelisted that Exception as > retry-worthy > and continued attempting to make progress while the other team worked > on mitigating the problem. > > Thanks for the KIP! > > On Wed, May 9, 2018 at 2:59 AM, Arjun Satish <arjun.sat...@gmail.com> > wrote:

[DISCUSS] KIP-298: Error Handling in Connect

2018-05-09 Thread Arjun Satish
All, I'd like to start a discussion on adding ways to handle and report record processing errors in Connect. Please find a KIP here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-298%3A+Error+Handling+in+Connect Any feedback will be highly appreciated. Thanks very much, Arjun

Re: Permission to edit KIP page

2018-05-07 Thread Arjun Satish
Thank you! On Mon, May 7, 2018 at 5:41 PM, Matthias J. Sax <matth...@confluent.io> wrote: > Done :) > > On 5/7/18 2:30 PM, Arjun Satish wrote: > > Could someone please grant me the permissions to submit a KIP? My > username > > on apache.org is wicknicks. > > > > Thanks very much, > > Arjun > > > >

Permission to edit KIP page

2018-05-07 Thread Arjun Satish
Could someone please grant me the permissions to submit a KIP? My username on apache.org is wicknicks. Thanks very much, Arjun

[jira] [Created] (KAFKA-6833) KafkaProducer throws "Invalid partition given with record" exception

2018-04-27 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-6833: --- Summary: KafkaProducer throws "Invalid partition given with record" exception Key: KAFKA-6833 URL: https://issues.apache.org/jira/browse/KAFKA-6833 Proj

[jira] [Created] (KAFKA-6511) Connect header parser incorrectly parses arrays

2018-01-31 Thread Arjun Satish (JIRA)
Arjun Satish created KAFKA-6511: --- Summary: Connect header parser incorrectly parses arrays Key: KAFKA-6511 URL: https://issues.apache.org/jira/browse/KAFKA-6511 Project: Kafka Issue Type: Bug

Add to contributors list

2017-11-10 Thread Arjun Satish
Hi, Could you please add me to the contributors list for Apache Kafka? My Apache username is wicknicks, and the profile page is located here ( https://issues.apache.org/jira/secure/ViewProfile.jspa?name=wicknicks). Thanks very much, Arjun