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 everything relevant to the failure. This way, we can only look at the error logs to find the most common pieces to "failure" puzzles: the operation, the config and the input record. Ideally, a programmer should be able to take these pieces and reproduce the error locally. 2. Added a table to describe this in the KIP. 3. Raw bytes will be base64 encoded before being logged. Updated the KIP to state this. Thank you! 4. I'll add an example log4j config to show we can take logs from a class and redirect it to a different location. Made a note in the PR for this. 5. When we talk about logging messages, this could mean instances of SinkRecords or SourceRecords. When we disable logging of messages, these records would be replaced by a "null". If you think it makes sense, instead of completely dropping the object, we could drop only the key and value objects from ConnectRecord? That way some context will still be retained. 6. Yes, for now I think it is good to have explicit config in Connectors which dictates the error handling behavior. If this becomes an inconvenience, we can think of having a cluster global default, or better defaults in the configs. Best, On Tue, May 15, 2018 at 1:07 PM, Magesh Nandakumar <mage...@confluent.io> wrote: > 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 behavior > is called out for each stage separately and what the connector developers > need to do ( may be a tabular format). Also, I think all retriable > exception when talking to Broker are never propagated to the Connect > Framework since the producer is configured to try indefinitely > 3. If a message fails in serialization, would the raw bytes be available to > the dlq or the error log > 4. Its not necessary to mention in KIP, but it might be better to separate > the error records to a separate log file as part of the default log4j > properties > 5. If we disable message logging, would there be any other metadata > available like offset that helps reference the record? > 6. If I need error handler for all my connectors, 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: > > > 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 > > >