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

2018-05-22 Thread Arjun Satish
After some discussions with Jason, we decided that the error handler should retry an operation only it throws a RetriableException. At the same time, any Exception in the Transformation and Converter step can be tolerated (by skipping the record). I updated the table in the Proposed Changes section

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

2018-05-21 Thread Arjun Satish
Hey Jason, This KIP does take serialization errors to be retriable. The typical use case is that Schema Registry can have a bad/unavailable schema, which can be corrected over time. But since the converters throw DataExceptions for all failures, it is hard to determine what caused these errors. H

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

2018-05-21 Thread Jason Gustafson
Thanks Arjun. I had one additional question. High level, I'm wondering if it makes sense to treat processing errors such as serialization failures the same as IO errors. In the former case, retrying typically doesn't help because the processing is deterministic. In the latter case, the failure may

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 wrote: > On Mon, May 21, 2018 at 12:39 PM Arjun Satish > wrote: > > > Thanks a lot, Ewen! I'll

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

2018-05-21 Thread Ewen Cheslack-Postava
On Mon, May 21, 2018 at 12:39 PM Arjun Satish wrote: > Thanks a lot, Ewen! I'll make sure the documentation is clear on the > differences between retries an tolerance. > > Do you think percentage would have the same problem as the one you brought > up? Also, if we say 10% tolerance, do we have to

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

2018-05-21 Thread Arjun Satish
Thanks a lot, Ewen! I'll make sure the documentation is clear on the differences between retries an tolerance. Do you think percentage would have the same problem as the one you brought up? Also, if we say 10% tolerance, do we have to wait for the duration to finish before failing the task, or sho

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

2018-05-21 Thread Ewen Cheslack-Postava
Arjun, Understood on retries vs tolerance -- though I suspect this will end up being a bit confusing to users as well. It's two levels of error handling which is what tripped me up. One last comment on KIP (which otherwise looks good): for the tolerance setting, do we want it to be an absolute va

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 it easier as a user t

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

2018-05-21 Thread Jason Gustafson
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 it easier as a user to work with timeouts since it's difficult to know how long a retry might take. Have you considered adding a timeout option which would

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

2018-05-18 Thread Arjun Satish
Super! Thanks, Magesh! On Fri, May 18, 2018 at 2:53 PM, Magesh Nandakumar wrote: > Arjun, > > Thanks for all the updates. I think it looks great and I don't have any > other concerns. > > Thanks > Magesh > > On Fri, May 18, 2018 at 2:36 PM, Konstantine Karantasis < > konstant...@confluent.io> wr

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

2018-05-18 Thread Magesh Nandakumar
Arjun, Thanks for all the updates. I think it looks great and I don't have any other concerns. Thanks Magesh On Fri, May 18, 2018 at 2:36 PM, Konstantine Karantasis < konstant...@confluent.io> wrote: > The updated version of the KIP that uses the dead-letter-queue only for > sink records and on

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

2018-05-18 Thread Arjun Satish
Thanks, Konstantine! On Fri, May 18, 2018 at 2:36 PM, Konstantine Karantasis < konstant...@confluent.io> wrote: > The updated version of the KIP that uses the dead-letter-queue only for > sink records and only to store the raw record data looks better and easier > to understand. > I think it's mo

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

2018-05-18 Thread Konstantine Karantasis
The updated version of the KIP that uses the dead-letter-queue only for sink records and only to store the raw record data looks better and easier to understand. 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:0

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

2018-05-18 Thread Arjun Satish
Ewen, Thanks a lot for your comments! 1. For errors.retry.delay.max.ms, yes we plan to use exponential backoffs with an fixed initial value. Updated the KIP to say this. 2. A failed operation will be retried (potentially multiple times). If all the retries fail, we declare this to be an error. T

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

2018-05-17 Thread Ewen Cheslack-Postava
A few more thoughts -- might not change things enough to affect a vote, but still some things to consider: * errors.retry.delay.max.ms -- this defines the max, but I'm not seeing where we define the actual behavior. Is this intentional, or should we just say that it is something like exponential,

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

2018-05-17 Thread Arjun Satish
Konstantine, Thanks for pointing out the typos. Fixed them. I had added the JSON schema which should now include key and header configs in there too. This should have been in the public interfaces section. Thanks very much, On Thu, May 17, 2018 at 9:13 AM, Konstantine Karantasis < konstant...@c

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

2018-05-17 Thread Konstantine Karantasis
Thanks Arjun for your quick response. Adding an example for the failure log improves things, but I think it'd be better to also add the schema definition of these Json entries. And I'll agree with Magesh that this format should be public API. Also, does the current example have a copy/paste typo?

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

2018-05-17 Thread Arjun Satish
On Wed, May 16, 2018 at 7:13 PM, Matt Farmer wrote: > Hey Arjun, > > I like deadletterqueue all lower case, so I'm +1 on that. > Super! updated the KIP. > > Yes, in the case we were seeing there were external system failures. > We had issues connecting to S3. While the connector does include >

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

2018-05-17 Thread Arjun Satish
Magesh, Updated the KIP. Thanks a lot! Best, On Wed, May 16, 2018 at 7:12 PM, Magesh Nandakumar wrote: > Arjun, > > Thanks for all the changes. Technically, the message format used for the > DLQ should be part of the public interface since users could consume it and > take actions. > > Thanks,

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

2018-05-16 Thread Matt Farmer
Hey Arjun, I like deadletterqueue all lower case, so I'm +1 on that. Yes, in the case we were seeing there were external system failures. We had issues connecting to S3. While the connector does include some retry functionality, however setting these values sufficiently high seemed to cause us to

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

2018-05-16 Thread Magesh Nandakumar
Arjun, Thanks for all the changes. Technically, the message format used for the DLQ should be part of the public interface since users could consume it and take actions. Thanks, Magesh On Wed, May 16, 2018 at 6:56 PM, Arjun Satish wrote: > Hi Konstantine, > > Thanks a lot for your feedback. I

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

2018-05-16 Thread Arjun Satish
Hi Konstantine, Thanks a lot for your feedback. I have made the necessary changes to the KIP. Best, On Wed, May 16, 2018 at 11:38 AM, Konstantine Karantasis < konstant...@confluent.io> wrote: > Arjun, it's exciting to see a KIP around better handling of bad-data and > errors in Kafka Connect. >

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

2018-05-16 Thread Konstantine Karantasis
Arjun, it's exciting to see a KIP around better handling of bad-data and errors in Kafka Connect. I have only a few comments below, which I hope you'll find helpful. 1. I think it'd be useful to describe a bit more in detail how someone can extract the raw data of a Kafka record that failed to ge

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

2018-05-15 Thread Arjun Satish
Magesh, Just to add to your point about retriable exceptions: the producer can throw retriable exceptions which we are handling it here: https://github.com/apache/kafka/blob/trunk/connect/ runtime/src/main/java/org/apache/kafka/connect/runtime/ WorkerSourceTask.java#L275 BTW, exceptions like Tim

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

2018-05-15 Thread Arjun Satish
Magesh, Thanks for the feedback! Really appreciate your comments. 1. I updated the KIP to state that only the configs of the failed operation will be emitted. Thank you! The purpose of bundling the configs of the failed operation along with the error context is to have a single place to find eve

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

2018-05-15 Thread Arjun Satish
Matt, Thanks so much for your comments. Really appreciate it! 1. Good point about the acronym. I can use deadletterqueue instead of dlq (using all lowercase to be consistent with the other configs in Kafka). What do you think? 2. Could you please tell us what errors caused these tasks to fail? W

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

2018-05-15 Thread Magesh Nandakumar
Hi Arjun, I think this a great KIP and would be a great addition to have in connect. Had a couple of minor questions: 1. What would be the value in logging the connector config using errors.log.include.configs for every message? 2. Not being picky on format here but it might be clearer if the beh

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

2018-05-14 Thread Matt Farmer
Hi Arjun, I'm following this very closely as better error handling in Connect is a high priority for MailChimp's Data Systems team. A few thoughts (in no particular order): For the dead letter queue configuration, could we use deadLetterQueue instead of dlq? Acronyms are notoriously hard to keep