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

2020-05-14 Thread Christopher Egerton
't necessarily be a producer involved? Yep, exactly :) Cheers, Chris On Thu, May 14, 2020 at 4:34 PM Christopher Egerton wrote: > Hi Randall, > > I think I prefer the method originally specified in the KIP. A separate > method can come with a contract about if/when it's called

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

2020-05-14 Thread Christopher Egerton
Hi Randall, I think I prefer the method originally specified in the KIP. A separate method can come with a contract about if/when it's called so that tasks can assume that it's only invoked once over their lifetime, and allows connector developers to separate the logic for storing (and possibly do

Re: [DISCUSS] KIP-587 Suppress detailed responses for handled exceptions in security-sensitive environments

2020-05-13 Thread Christopher Egerton
sted > in removing stack traces from all responses, which will expand the scope of > this KIP a bit. I'm going to update the wiki entry, and then would it be > reasonable to call for a vote? > > Thanks! > Connor > > On 4/17/20, 3:53 PM, "Christopher Egerton" wr

Re: [DISCUSS] KIP-611: Improved Handling of Abandoned Connectors and Tasks

2020-05-08 Thread Christopher Egerton
change the calculus > in the second rejected alternative about keeping both the task- and > connector-based timeout property? I actually think it might. > 8. How much value does the sequence number in the logging context bring? Is > this really worth the risk given the backward

[DISCUSS] KIP-611: Improved Handling of Abandoned Connectors and Tasks

2020-05-08 Thread Christopher Egerton
Hi all, I've introduced a KIP to improve how the Connect framework handles abandoned connectors and tasks: https://cwiki.apache.org/confluence/display/KAFKA/KIP-611%3A+Improved+Handling+of+Abandoned+Connectors+and+Tasks The improvements currently being proposed include new JMX metrics, a slight s

Re: [Discuss] KIP-582 Add a "continue" option for Kafka Connect error handling

2020-05-07 Thread Christopher Egerton
tten learning curve of consumer tools, but also enabling extern tools > for > analysis. > > Best, > Zihan > > On 2020/05/03 17:36:34, Christopher Egerton wrote: > > Hi Zihan, > > > > I guess I'm still unclear on exactly what form this analysis might take. &

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

2020-05-07 Thread Christopher Egerton
Hi Aakash, Thanks for the KIP! Given the myriad of different error-handling mechanisms available in sink connectors today, I think it'd be a great improvement to add this kind of support to the framework and not only take some of the development burden off of connector writers but also some of the

Re: [VOTE] KIP-605 - Expand Connect Worker Internal Topic Settings

2020-05-07 Thread Christopher Egerton
LGTM, +1 (non-binding) Thanks Randall! On Thu, May 7, 2020 at 11:01 AM Randall Hauch wrote: > I'd like to open the vote for KIP-605: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-605%3A+Expand+Connect+Worker+Internal+Topic+Settings > > This is relatively straightforward, and the dis

Re: [DISCUSS] KIP-605 Expand Connect Worker Internal Topic Settings

2020-05-04 Thread Christopher Egerton
lternative describing why I think failing is not desirable. > > 4. Added a sentence as suggested. > > Thanks! > > On Sun, May 3, 2020 at 12:55 PM Christopher Egerton > wrote: > > > Hi Randall, > > > > Thanks for the KIP! I have a few questions and suggestions b

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-04 Thread Christopher Egerton
+1 (non-binding) On Mon, May 4, 2020 at 5:02 AM Edoardo Comar wrote: > +1 (non-binding) > Thanks Mickael > > -- > > Edoardo Comar > > Event Streams for IBM Cloud > IBM UK Ltd, Hursley Park, SO21 2JN > > > > > From: Mickael Maison > To: dev

Re: [DISCUSS] KIP-585: Conditional SMT

2020-05-03 Thread Christopher Egerton
ivably implement on their own, and that the KIP > > introduces the > > > most likely ones as supplied implementations, much as there > are SMT > > > implementations such as HoistField. I don't really see users > > implementing > >

Re: [Discuss] KIP-581: Value of optional null field which has default value

2020-05-03 Thread Christopher Egerton
only implement the > serialization now. This is because the original serde implement is not > symmetrical, the convertToConnect is a static method and can’t access the > field in JsonConverter > instance, maybe I should do some refactoring to implement the > deserialization. > >

Re: [DISCUSS] KIP-605 Expand Connect Worker Internal Topic Settings

2020-05-03 Thread Christopher Egerton
Hi Randall, Thanks for the KIP! I have a few questions and suggestions but no major objections. 1. The motivation is pretty clear for altering the various "*.storage.replication.factor" properties to allow -1 as a value now. Are there expected use cases for allowing modification of other properti

Re: [Discuss] KIP-582 Add a "continue" option for Kafka Connect error handling

2020-05-03 Thread Christopher Egerton
Task.putBrokenRecord() instead of DLQ. > > Best, > Zihan > > On 2020/04/25 20:05:37, Christopher Egerton wrote: > > Hi Zihan, > > > > Thanks for the changes and the clarifications! I agree that the > complexity > > of maintaining a second topic and a second

Re: [DISCUSS] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-03 Thread Christopher Egerton
ark it as > deprecated and make it extend the new interface. I've updated to the > KIP. > > > I've also opened a draft PR: https://github.com/apache/kafka/pull/8604 > > On Fri, Apr 24, 2020 at 12:17 AM Christopher Egerton > wrote: > > > > Hi Mickael,

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-25 Thread Christopher Egerton
n FALSE if the predicate matches. > > I feel that a DSL for SMTs is overkill. If you need something that > complex, it's > perhaps too complex for a transformation chain and it's really a streaming > application. > > Andrew Schofield > IBM Event Streams > >

Re: [Discuss] KIP-582 Add a "continue" option for Kafka Connect error handling

2020-04-25 Thread Christopher Egerton
to the KIP. Compared with this alternative, the point of this > proposal is to save the effort to maintain twice as many connectors as > necessary. > > Thanks again. Looking forward to the discussion! > > Sorry if you see this email twice, the previous one didn't sh

Re: [DISCUSS] KIP-597: MirrorMaker2 internal topics Formatters

2020-04-23 Thread Christopher Egerton
Hi Mickael, The KIP looks great and the additional formatters seem like excellent tools for debugging and diving into the internals of Mirror Maker 2.0. I've got a few thoughts on the proposed changes: 1. Would it be possible to provide examples for what the actual output might look like when the

Re: [DISCUSS] KIP-587 Suppress detailed responses for handled exceptions in security-sensitive environments

2020-04-17 Thread Christopher Egerton
cussion! > > > > RE: the 200 response you highlighted, great catch! I'll work with my > > customer and get back to you on their audit team's intention! I'm > > fairly certain I know the answer, but I need to be sure before I > speak >

Re: [Discuss] KIP-581: Value of optional null field which has default value

2020-04-14 Thread Christopher Egerton
Hi Cheng, Thanks for the KIP! I really appreciate the care that was taken to ensure backwards compatibility for existing users, and the minimal changes to public interface that are suggested to address this. I have two quick requests for clarification: 1) Where is the proposed "accept.optional.n

Re: [Discuss] KIP-582 Add a "continue" option for Kafka Connect error handling

2020-04-13 Thread Christopher Egerton
HI Zihan, Thanks for the KIP! I have some questions that I'm hoping we can address to help better understand the motivation for this proposal. 1. In the "Motivation" section it's written that "If users want to store their broken records, they have to config a broken record queue, which is too muc

Re: [DISCUSS] KIP-587 Suppress detailed responses for handled exceptions in security-sensitive environments

2020-04-08 Thread Christopher Egerton
r. > > > > > > 3. I agree with your suggestion here. Absolutely ready to take > the > > > community feedback on what makes sense here. > > > > > > 4. I should note that while I emphasized uncau

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-08 Thread Christopher Egerton
t;>> > >>> Would love to see this feature in one or another way in Connect. > >>> > >>> Best, > >>> > >>> --Gunnar > >>> > >>> > >>> > >>> Am Do., 2. Apr. 2020 um 18:48 Uhr

Re: [DISCUSS] KIP-587 Suppress detailed responses for handled exceptions in security-sensitive environments

2020-04-02 Thread Christopher Egerton
Hi Connor, Great stuff! I generally like being able to see the stack trace of an exception directly via the REST API but can definitely understand the security concerns here. I've got a few questions/remarks about the KIP and would be interested in your thoughts: 1. The KIP mentions a SUPRESSED_E

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-01 Thread Christopher Egerton
Hi Tom, This looks great and I'd love to see the out-of-the-box SMTs become even more powerful with the improvements you've proposed! Got a few remarks and would be interested in your thoughts: 1. Instead of the new "ConfigDef config(Map props)" method, what would you think about adopting a simil

Re: Working on a contribution for suppressing exceptions from KafkaConnect

2020-03-31 Thread Christopher Egerton
Hi Connor, Thanks for the contribution! It looks like the feature you've implemented changes public interface, which means that a KIP would be required in order to merge them into Kafka. You can find more information about what kinds of changes require KIPs, what a KIP should consist of, and the p

Re: [VOTE] 2.5.0 RC2

2020-03-30 Thread Christopher Egerton
uded in the next RC > > -David > > On Thu, Mar 26, 2020 at 6:01 PM Christopher Egerton > wrote: > > > Hi all, > > > > I'd like to request that > https://issues.apache.org/jira/browse/KAFKA-9771 > > be > > treated as a release blocker for 2

Re: [VOTE] 2.5.0 RC2

2020-03-26 Thread Christopher Egerton
Hi all, I'd like to request that https://issues.apache.org/jira/browse/KAFKA-9771 be treated as a release blocker for 2.5. This is a regression caused by the recent bump in Jetty version that causes inter-worker communication to fail for Connect clusters that use SSL and a keystore that contains

Re: [DISCUSS] KIP-494 Connect REST Endpoint for Transformations (SMTs) and other Plugins

2020-02-12 Thread Christopher Egerton
Hi Cyrus, Thanks for the KIP! One quick question--I see the use case for being able to list per-connector plugins (SMTs, converters, and header converters) via REST API, but I'm not so sure about worker plugins (REST extensions and config providers). Since those are configured at startup for the

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Christopher Egerton
lue for the newly-proposed parameter be? If resets are performed by default, the first scenario I outlined would become possible; if not, then the second would become possible. I'd lean towards performing them by default but would be interested in others' thoughts. (If the proposed &qu

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Christopher Egerton
Hi Konstantine, Thanks for the KIP! There's been a lot of productive discussion so far so I'll try to keep my remarks brief. 1. As far as resetting the active topics for a connector goes, it's noted in the KIP that this can be done for a deleted connector. Can this also be done for connectors tha

Re: [VOTE] 2.4.0 RC2

2019-12-02 Thread Christopher Egerton
Hi all, I'd like to submit https://issues.apache.org/jira/browse/KAFKA-9258 for blocker consideration. This issue affects the Connect framework and causes a 500 error to be thrown from the REST API when restarting a task that has failed during startup, even after the connector has been reconfigure