Re: [DISCUSS] KIP-449: Add connector contexts to Connect worker logs

2019-04-15 Thread Randall Hauch
ere's room for inclusion in this KIP. > > Konstantine > > > > On Fri, Apr 12, 2019 at 3:24 PM Randall Hauch wrote: > > > Thanks for the review and feedback, Konstantine. > > > > 1. Great suggestion. I've updated the KIP to hopefully make it more clear >

Re: [DISCUSS] KIP-449: Add connector contexts to Connect worker logs

2019-04-12 Thread Randall Hauch
; to follow the KIP. Currently I see placeholder text below Contents and in > Rejected alternatives. > > And my least favorite section, but I can't help it :) > Typos/grammar: > to understand what (is) happening within the worker, > Kafka Connect use(s) ... > quotes maybe not need

Re: [DISCUSS] KIP-437: Custom replacement for MaskField SMT

2019-04-12 Thread Randall Hauch
Thanks for the proposal and the PR request, Valeria. I've been working with Valeria on this PR, so it's not surprising I'm in favor of this KIP and improvement so that users can easily mask values with specific replacement values with the same primitive type. This proposal has been open for over

[jira] [Resolved] (KAFKA-8058) ConnectRestExtensionContext does not return list of connectors from its ConnectClusterState

2019-04-07 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8058. -- Resolution: Fixed > ConnectRestExtensionContext does not return list of connectors from

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-04-04 Thread Randall Hauch
aul, the proposal looks great, thanks. > > > > Ryanne > > > > On Mon, Mar 25, 2019, 9:03 AM Randall Hauch wrote: > > > > > Paul, > > > > > > Thanks for updating the KIP with the proposal. I do think the KIP > should > > at > &g

[jira] [Resolved] (KAFKA-5141) WorkerTest.testCleanupTasksOnStop transient failure due to NPE

2019-04-03 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-5141. -- Resolution: Fixed > WorkerTest.testCleanupTasksOnStop transient failure due to

[jira] [Resolved] (KAFKA-8126) Flaky Test org.apache.kafka.connect.runtime.WorkerTest.testAddRemoveTask

2019-04-03 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8126. -- Resolution: Fixed > Flaky Test org.apache.kafka.connect.runtime.WorkerTest.testAddRemoveT

[jira] [Resolved] (KAFKA-8063) Flaky Test WorkerTest#testConverterOverrides

2019-04-03 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8063?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8063. -- Resolution: Fixed > Flaky Test WorkerTest#testConverterOverri

[DISCUSS] KIP-449: Add connector contexts to Connect worker logs

2019-04-02 Thread Randall Hauch
I've been working on https://github.com/apache/kafka/pull/5743 for a while, but there were a number of comment, suggestions, and mild concerns on the PR. One of those comments was that maybe changing the Connect log content in this way probably warrants a KIP. So here it is:

Re: [DISCUSS] KIP-440: Extend Connect Converter to support headers

2019-04-02 Thread Randall Hauch
Thanks for the submission, Yaroslav -- and for building on the suggestion of Jeremy C in https://issues.apache.org/jira/browse/KAFKA-7273. This is a nice and simple approach that is backward compatible. The KIP looks good so far, but I do have two specific suggestions to make things just a bit

[jira] [Resolved] (KAFKA-8014) Extend Connect integration tests to add and remove workers dynamically

2019-03-25 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8014?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8014. -- Resolution: Fixed > Extend Connect integration tests to add and remove workers dynamica

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-03-25 Thread Randall Hauch
ible to usefully override the default client id properties. > > > > I'm not sure how we would handle the dead-letter queue case though - > maybe > > we could automatically add a "dlq-" prefix to the producer client id? > > > > If there is agreement on this ch

[DISCUSSION] KIP-407

2019-03-15 Thread Randall Hauch
https://cwiki.apache.org/confluence/display/KAFKA/KIP-407%3A+Kafka+Connect+support+override+worker+kafka+api+configuration+with+connector+configuration+that+post+by+rest+api was created by the author and a discussion thread was never started. This appears to very similar to (or rather a subset

Re: [VOTE] KIP-415: Incremental Cooperative Rebalancing in Kafka Connect

2019-03-13 Thread Randall Hauch
Excellent work, Konstantine! +1 (binding) On Mon, Mar 11, 2019 at 8:05 PM Konstantine Karantasis < konstant...@confluent.io> wrote: > Thanks Jason! > That makes perfect sense. The change is reflected in the KIP now. > "compatible" will be the default mode for "connect.protocol" > > Cheers, >

Re: [VOTE] KIP-131 - Add access to OffsetStorageReader from SourceConnector

2019-02-25 Thread Randall Hauch
> >> > >> Are there any more votes ? Thanks > >> > >> Le 12 sept. 2017 20:18, "Gwen Shapira" a écrit : > >> > >>> Thanks for clarifying. > >>> > >>> +1 again :) > >>> > >>> > >>&

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-02-21 Thread Randall Hauch
t the best process > to > > > proceed. Will the change in default behaviour require a new KIP, given > it > > > will affect existing deployments? Would I be best to repurpose this > > > KIP-411, or am I best to create a new KIP? Thanks! > > > > > >

Re: [ANNOUNCE] New Committer: Randall Hauch

2019-02-21 Thread Randall Hauch
ng Wang wrote: > > > Hello all, > > > > > > The PMC of Apache Kafka is happy to announce another new committer > > joining > > > the project today: we have invited Randall Hauch as a project committer > > and > > > he has accepted. > &g

[jira] [Resolved] (KAFKA-7907) KafkaDistributed - Unknown datum class: class java.lang.Short

2019-02-07 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-7907. -- Resolution: Invalid Closing this issue, since it is not a problem in Kafka Connect but rather

Re: [jira] [Created] (KAFKA-7907) KafkaDistributed

2019-02-07 Thread Randall Hauch
This is a regression in Confluent's AvroConverter that shipped in Confluent Platform 5.1.0; earlier versions were not affected. Please see https://github.com/confluentinc/schema-registry/issues/968 for more details. It has already been fixed, and will be in the upcoming Confluent Platform 5.1.1

Re: [Discuss] Question on KIP-298: Error Handling in Kafka Connect

2019-02-06 Thread Randall Hauch
Hi, Pere. The primary reason that KIP-298 did not support a DLQ for source connectors was because we couldn't get around serialization problems. With source connectors, the converter (serializer) is the last element in the chain, and if there is a problem serializing a record then we could not

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2019-01-26 Thread Randall Hauch
further. Best regards, Randall On Tue, Jan 22, 2019 at 11:04 AM Ryanne Dolan wrote: > Thanks Randall, makes sense to me. > > I suggest we change the topic.creation.enabled property name though. Sounds > like it means topics are not created at all when disabled. > >

[jira] [Created] (KAFKA-7873) KafkaBasedLog's consumer should always seek to beginning when starting

2019-01-25 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7873: Summary: KafkaBasedLog's consumer should always seek to beginning when starting Key: KAFKA-7873 URL: https://issues.apache.org/jira/browse/KAFKA-7873 Project: Kafka

Re: [DISCUSS] KIP-415: Incremental Cooperative Rebalancing in Kafka Connect

2019-01-19 Thread Randall Hauch
Thanks for all this work, Konstantine. I have a question about when a member leaves. Here's the partial scenario, repeated from the KIP: Initial group and assignment: W1([AC0, AT1]), W2([AT2, BC0]), W3([BT1]) Config topic contains: AC0, AT1, AT2, BC0, BT1 W2 leaves Rebalance is triggered W1

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2019-01-19 Thread Randall Hauch
ehavior doesn't change from what > we have today. If a user wants to change from the default, they can opt-in > to one of the other policies or implement their own. > > Ryanne > > On Tue, Nov 27, 2018 at 6:31 PM Randall Hauch wrote: > > > Thanks for the feedback. Some tho

Re: [DISCUSS] KIP-411: Add option to make Kafka Connect task client ID values unique

2019-01-08 Thread Randall Hauch
Hi, Paul. I concur with the others, and I like the new approach that avoids a new configuration, especially because it does not change the behavior for anyone already using `producer.client.id` and/or `consumer.client.id`. I did leave a few comments on the PR. Perhaps the biggest one is whether

[jira] [Reopened] (KAFKA-5117) Kafka Connect REST endpoints reveal Password typed values

2019-01-08 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reopened KAFKA-5117: -- Assignee: Randall Hauch (was: Ewen Cheslack-Postava) I'm reopening this because

Re: [DISCUSS] KIP-404: Add Kafka Connect configuration parameter for disabling WADL output on OPTIONS request

2018-12-18 Thread Randall Hauch
more of a bug. Unless we think it's a good idea to support it going > > > > > forward, I'd suggest going with the rejected alternative of just > > > turning > > > > it > > > > > off. What do you think? > > > > > > > > > > Thanks, &g

Re: [DISCUSS] KIP-404: Add Kafka Connect configuration parameter for disabling WADL output on OPTIONS request

2018-12-13 Thread Randall Hauch
Thanks, Alex. The KIP looks good to me. Randall On Wed, Dec 12, 2018 at 10:08 PM Guozhang Wang wrote: > Alex, > > Thanks for putting up this KIP. The proposal lgtm. > > Guozhang > > On Wed, Dec 12, 2018 at 7:41 PM Oleksandr Diachenko > > wrote: > > > Hi all, > > > > I would like to start a

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-11-28 Thread Randall Hauch
On Tue, Nov 27, 2018 at 6:31 PM Randall Hauch wrote: > Thanks for the feedback. Some thoughts inline. > > On Tue, Nov 27, 2018 at 5:47 PM Ewen Cheslack-Postava > wrote: > >> re: AdminClient vs this proposal, one consideration is that AdminClient >> exposes a lot mor

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-11-27 Thread Randall Hauch
r` instead. Then it'd be useful outside of Connect, but is it strange that it's not in the Connect API packages? Randall > > -Ewen > > On Mon, Sep 24, 2018 at 11:56 AM Andrew Otto wrote: > > > FWIW, I’d find this feature useful. > > > > On Mon, Sep 24, 2018 a

[jira] [Created] (KAFKA-7509) Kafka Connect logs unnecessary warnings about unused configurations

2018-10-15 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7509: Summary: Kafka Connect logs unnecessary warnings about unused configurations Key: KAFKA-7509 URL: https://issues.apache.org/jira/browse/KAFKA-7509 Project: Kafka

[jira] [Created] (KAFKA-7472) Implement KIP-145 transformations

2018-10-02 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7472: Summary: Implement KIP-145 transformations Key: KAFKA-7472 URL: https://issues.apache.org/jira/browse/KAFKA-7472 Project: Kafka Issue Type: New Feature

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-09-24 Thread Randall Hauch
thing > we should support across all connectors, as if it is some established > pattern or universally useful. > > Ryanne > > On Mon, Sep 24, 2018, 10:14 AM Randall Hauch wrote: > > > Hi, Ryanne. My apologies for not responding earlier, as I was on a long > > holida

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-09-24 Thread Randall Hauch
us, the configuration > > will > > > end up lying and misleading, and there won't be any indication that the > > > configuration is lying. > > > - Connectors that want to control settings will end up naming topics > > > accordingly. For example, a connector that

[VOTE] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-09-07 Thread Randall Hauch
I believe the feedback on KIP-158 has been addressed. I'd like to start a vote. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics Discussion Thread:

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-09-04 Thread Randall Hauch
hink. I'd like to start voting soon, but because I made the above change I'll wait a few days. Best regards, Randall On Wed, Aug 29, 2018 at 9:41 PM Randall Hauch wrote: > Thanks, Magesh. > > All, I've made a few very minor changes to some JavaDocs and the > signatures of the name-val

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-08-29 Thread Randall Hauch
; that we can support any topic setting added in Kafka without any code > > changes in connect. Since these are for topics that will have the actual > > data stream, users might possibly need more flexibility in terms of how > the > > topics get created. > > > > Thanks &

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-08-28 Thread Randall Hauch
ttings that can be > configured. Is it limited to the setting exposed in the TopicSettings > interface? > > Thanks > Magesh > > On Tue, Aug 21, 2018 at 7:59 PM Randall Hauch wrote: > > > Okay, after much delay let's try this again for AK 2.1. Has anyone found > > any

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-08-21 Thread Randall Hauch
, Jan 26, 2018 at 4:20 PM Randall Hauch wrote: > The KIP deadline for 1.1 has already passed, but I'd like to restart this > discussion so that we make the next release. I've not yet addressed the > previous comment about *existing* topics, but I'll try to do that over the > next few week

Re: Plan for new Kafka Connect Transform

2018-07-23 Thread Randall Hauch
How about changing the ReplaceField SMT to be able to support nested fields? If we come up with a unified way to identify nested fields, then we could add support for nested fields to other SMTs, too. Best regards, Randall On Thu, Jul 19, 2018 at 2:18 PM, karri saisatish kumar reddy <

[jira] [Created] (KAFKA-7157) Connect TimestampConverter SMT doesn't handle null values

2018-07-12 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7157: Summary: Connect TimestampConverter SMT doesn't handle null values Key: KAFKA-7157 URL: https://issues.apache.org/jira/browse/KAFKA-7157 Project: Kafka

Re: [DISCUSS] KIP-316: Command-line overrides for ConnectDistributed worker properties

2018-06-20 Thread Randall Hauch
My only concern with this proposal is that it adds yet another way to specify configuration properties, making it a bit more difficult to track down how/whether a configuration property has been set. Configuring Kafka Connect is already too challenging, so we need to be very clear that this

Re: [VOTE] KIP-316: Command-line overrides for ConnectDistributed worker properties

2018-06-20 Thread Randall Hauch
IMO we should not request a vote without additional time for discussion. Best regards, Randall On Wed, Jun 20, 2018 at 7:29 AM, Jakub Scholz wrote: > +1 (non-binding) > > On Mon, Jun 18, 2018 at 8:42 PM Kevin Lafferty > wrote: > > > Hi all, > > > > I got a couple notes of interest on the

Re: [DISCUSS] KIP-318: Make Kafka Connect Source idempotent

2018-06-20 Thread Randall Hauch
Thanks for starting this conversation, Stephane. I have a few questions. The worker already accepts nearly all producer properties already, and all `producer.*` properties override any hard-coded properties defined in `Worker.java`. So isn't it currently possible for a user to define these

[jira] [Created] (KAFKA-7056) Connect's new numeric converters should be in a different package

2018-06-13 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7056: Summary: Connect's new numeric converters should be in a different package Key: KAFKA-7056 URL: https://issues.apache.org/jira/browse/KAFKA-7056 Project: Kafka

[jira] [Created] (KAFKA-7047) Connect isolation whitelist does not include SimpleHeaderConverter

2018-06-12 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7047: Summary: Connect isolation whitelist does not include SimpleHeaderConverter Key: KAFKA-7047 URL: https://issues.apache.org/jira/browse/KAFKA-7047 Project: Kafka

[jira] [Created] (KAFKA-7043) Connect isolation whitelist does not include new primitive converters (KIP-305)

2018-06-11 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7043: Summary: Connect isolation whitelist does not include new primitive converters (KIP-305) Key: KAFKA-7043 URL: https://issues.apache.org/jira/browse/KAFKA-7043

[jira] [Created] (KAFKA-7031) Kafka Connect API module depends on Jersey

2018-06-10 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7031: Summary: Kafka Connect API module depends on Jersey Key: KAFKA-7031 URL: https://issues.apache.org/jira/browse/KAFKA-7031 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-7009) Mute logger for reflections.org at the warn level in system tests

2018-06-06 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-7009: Summary: Mute logger for reflections.org at the warn level in system tests Key: KAFKA-7009 URL: https://issues.apache.org/jira/browse/KAFKA-7009 Project: Kafka

[jira] [Resolved] (KAFKA-6831) FileStreamSink is very slow

2018-05-29 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6831. -- Resolution: Invalid Fix Version/s: (was: 1.1.0) [~vrmprabhat], first of all

Re: [VOTE] KIP-305: Add Connect primitive number converters

2018-05-25 Thread Randall Hauch
wrote: > > +1 > > > > Thanks, > > Yeva > > > > > > On Tue, May 22, 2018 at 7:48 PM, Magesh Nandakumar <mage...@confluent.io > > wrote: > > > > > +1 (non-binding) > > > > > > Thanks > > > Magesh > > >

Re: [VOTE] KIP-305: Add Connect primitive number converters

2018-05-22 Thread Randall Hauch
(bump so a few new subscribers see this thread.) On Tue, May 22, 2018 at 4:39 PM, Randall Hauch <rha...@gmail.com> wrote: > +1 (non-binding) > > On Tue, May 22, 2018 at 4:05 PM, Matthias J. Sax <matth...@confluent.io> > wrote: > >> +1 (binding) >> &g

Re: [VOTE] KIP-305: Add Connect primitive number converters

2018-05-22 Thread Randall Hauch
ack-Postava < > e...@confluent.io> > > wrote: > > > >> +1 (binding) > >> > >> -Ewen > >> > >> On Tue, May 22, 2018 at 9:29 AM Ted Yu <yuzhih...@gmail.com> wrote: > >> > >>> +1 > >>> > &

[VOTE] KIP-305: Add Connect primitive number converters

2018-05-22 Thread Randall Hauch
I'd like to start a vote of a very straightforward proposal for Connect to add converters for the basic primitive number types: integer, short, long, double, and float that reuse Kafka's corresponding serdes. Here is the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-

Re: [DISCUSS] KIP-242: Mask password fields in Kafka Connect REST response

2018-05-21 Thread Randall Hauch
fashion. > > > > @Ewen - Regarding the concern of accessing information securely - has > > there been any consideration of adding authentication to the connect api? > > > > > On Jan 17, 2018, at 3:55 PM, Randall Hauch <rha...@gmail.com> wrote: > &g

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

2018-05-21 Thread Randall Hauch
Thanks, Arjun. +1 (non-binding) Regards, Randall On Mon, May 21, 2018 at 11:14 AM, Guozhang Wang wrote: > Thanks for the KIP. +1 (binding) > > > Guozhang > > On Fri, May 18, 2018 at 3:36 PM, Gwen Shapira wrote: > > > +1 > > > > Thank you! Error handling

[jira] [Created] (KAFKA-6926) Reduce NPath exceptions in Connect

2018-05-21 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6926: Summary: Reduce NPath exceptions in Connect Key: KAFKA-6926 URL: https://issues.apache.org/jira/browse/KAFKA-6926 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-305: Add Connect primitive number converters

2018-05-19 Thread Randall Hauch
g if they exist. > > -Ewen > >> On Fri, May 18, 2018 at 11:55 AM Randall Hauch <rha...@gmail.com> wrote: >> >> Thanks, Ewen. >> >> You make several good points, and I've updated the KIP to hopefully address >> your comments. I think the symmetr

Re: [DISCUSS] KIP-305: Add Connect primitive number converters

2018-05-18 Thread Randall Hauch
> > -Ewen > > On Thu, May 17, 2018 at 6:04 PM Magesh Nandakumar <mage...@confluent.io> > wrote: > > > Thanks Randall for the KIP. I think it will be super useful and looks > > pretty straightforward to me. > > > > Thanks > > Magesh > >

Re: [VOTE] KIP-297: Externalizing Secrets for Connect Configurations

2018-05-18 Thread Randall Hauch
Looks great. +1 (non-binding) Regards, Randall On Fri, May 18, 2018 at 10:23 AM, Rajini Sivaram wrote: > Thanks, Robert! Sounds good. And thanks for the KIP. > > +1 (binding) > > Regards, > > Rajini > > On Fri, May 18, 2018 at 4:17 PM, Robert Yokota

Re: [VOTE] KIP-285: Connect Rest Extension Plugin

2018-05-18 Thread Randall Hauch
+1 (non-binding) Regards, Randall On Fri, May 18, 2018 at 11:32 AM, Konstantine Karantasis < konstant...@confluent.io> wrote: > +1 (non-binding) > > - Konstantine > > On Thu, May 17, 2018 at 10:05 PM, Ewen Cheslack-Postava > > wrote: > > > +1 (binding) > > > > Thanks, > >

Re: [DISCUSS] KIP-301 Schema Inferencing in JsonConverters

2018-05-18 Thread Randall Hauch
There are lots of potential details/gotchas here, with no obvious defaults. Even if we did have optional configs for the type(s) to use for empty arrays and null values, how likely is it that those would apply for all occurrences? Two fields might have empty arrays, but the best schema for each

[DISCUSS] KIP-305: Add Connect primitive number converters

2018-05-17 Thread Randall Hauch
I'd like to start discussion of a very straightforward proposal for Connect to add converters for the basic primitive number types: integer, short, long, double, and float. Here is the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-305%3A+Add+Connect+primitive+number+converters As

[jira] [Created] (KAFKA-6913) Add primitive numeric converters to Connect

2018-05-17 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6913: Summary: Add primitive numeric converters to Connect Key: KAFKA-6913 URL: https://issues.apache.org/jira/browse/KAFKA-6913 Project: Kafka Issue Type

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-05-16 Thread Randall Hauch
gt; minor edits per your suggestions. Since there are no major outstanding > issues, i'm moving this to vote. > > Thanks > Magesh > >> On Wed, May 16, 2018 at 4:38 PM, Randall Hauch <rha...@gmail.com> wrote: >> >> A few very minor suggestions: >> &

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-05-16 Thread Randall Hauch
ncludes details on packages for the interfaces and > > the classes. Let me know your thoughts. > > > > Thanks > > Magesh > > > > On Fri, Apr 27, 2018 at 12:03 PM, Randall Hauch <rha...@gmail.com> > wrote: > > > >> Great work, Magesh. I like the ove

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-27 Thread Randall Hauch
a replica of the required > entities in the draft implementation. If you can take a look at the PR and > let me know your thoughts, I will update the KIP to reflect the same > > https://github.com/apache/kafka/pull/4931 > > On Tue, Apr 24, 2018 at 11:44 AM, Randall Hauch <rha...@gmail.com

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-24 Thread Randall Hauch
what kind of impact and choices we'll have to make? Best regards, Randall On Tue, Apr 24, 2018 at 12:48 PM, Randall Hauch <rha...@gmail.com> wrote: > Thanks for updating the KIP, Magesh. You've resolved all of my concerns, > though I have one more: we should specify the package names

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-24 Thread Randall Hauch
resources. > > The fact that we use Jersey shouldn't even be exposed in the interface. > > Hence it doesn't affect the public API by any means. > > > > I will update the KIP and let everyone know. > > > > Thanks > > Magesh > > > > On

[jira] [Resolved] (KAFKA-6797) Connect oracle database to kafka and stream data

2018-04-20 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6797?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6797. -- Resolution: Invalid Please use the Apache Kafka user discussion forum for questions. See https

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-19 Thread Randall Hauch
make it a > deterministic behavior and avoid any potential re-registrations. Let me > know your thoughts on these. > This sounds a good idea. Is it as flexible as the current proposal? If not, then I'd love to see how this affects the public APIs. > > Thanks > Magesh > > > On Wed,

Re: [DISCUSS] KIP-285: Connect Rest Extension Plugin

2018-04-11 Thread Randall Hauch
Very nice proposal, Magesh. I like the approach and the new concepts and interfaces, but I do have a few comments/suggestions about some specific details: 1. In the "Motivation" section, perhaps it makes sense to briefly describe one or two somewhat concrete examples of how this is useful.

[jira] [Created] (KAFKA-6755) MaskField SMT should optionally take a literal value to use instead of using null

2018-04-05 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6755: Summary: MaskField SMT should optionally take a literal value to use instead of using null Key: KAFKA-6755 URL: https://issues.apache.org/jira/browse/KAFKA-6755

Re: [DISCUSS] KIP-275 - Indicate "isClosing" in the SinkTaskContext

2018-04-03 Thread Randall Hauch
lso, adding a paragraph to the preCommit javadoc also seems like a > >> Very Very Good Idea™ so I'll make that update to the KIP as well. > >> > >> On Mon, Apr 2, 2018 at 10:48 AM, Randall Hauch <rha...@gmail.com> > wrote: > >> > >>> Thanks for

[jira] [Created] (KAFKA-6740) Plugins class' newConverter and newHeaderConverter methods are unclear

2018-04-02 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6740: Summary: Plugins class' newConverter and newHeaderConverter methods are unclear Key: KAFKA-6740 URL: https://issues.apache.org/jira/browse/KAFKA-6740 Project: Kafka

Re: Seeking Feedback on Kafka Connect Issues

2018-04-02 Thread Randall Hauch
Yes, Confluent would be interested in improvements to the S3 connector. Feel free to create an issue/PR in https://github.com/confluentinc/kafka-connect-storage-cloud/. I just created https://issues.apache.org/jira/browse/KAFKA-6738 to deal with the bad data handling issue, and we can use that to

[jira] [Created] (KAFKA-6738) Kafka Connect handling of bad data

2018-04-02 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6738: Summary: Kafka Connect handling of bad data Key: KAFKA-6738 URL: https://issues.apache.org/jira/browse/KAFKA-6738 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-275 - Indicate "isClosing" in the SinkTaskContext

2018-04-02 Thread Randall Hauch
Thanks for the KIP proposal, Matt. You mention in the "Rejected Alternatives" section that you considered changing the signature of the `preCommit` method but rejected it because it would break backward compatibility. Strictly speaking, it is possible to do this without breaking compatibility by

[jira] [Resolved] (KAFKA-6661) Sink connectors that explicitly 'resume' topic partitions can resume a paused task

2018-03-19 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6661?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6661. -- Resolution: Fixed > Sink connectors that explicitly 'resume' topic partitions can res

[jira] [Created] (KAFKA-6675) Connect workers should log plugin path and available plugins more clearly

2018-03-17 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6675: Summary: Connect workers should log plugin path and available plugins more clearly Key: KAFKA-6675 URL: https://issues.apache.org/jira/browse/KAFKA-6675 Project

[jira] [Created] (KAFKA-6661) Sink connectors that explicitly 'resume' topic partitions can resume a paused task

2018-03-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6661: Summary: Sink connectors that explicitly 'resume' topic partitions can resume a paused task Key: KAFKA-6661 URL: https://issues.apache.org/jira/browse/KAFKA-6661

[jira] [Created] (KAFKA-6605) Flatten SMT does not properly handle fields that are null

2018-03-01 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6605: Summary: Flatten SMT does not properly handle fields that are null Key: KAFKA-6605 URL: https://issues.apache.org/jira/browse/KAFKA-6605 Project: Kafka

[jira] [Created] (KAFKA-6586) Refactor Connect executables

2018-02-22 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6586: Summary: Refactor Connect executables Key: KAFKA-6586 URL: https://issues.apache.org/jira/browse/KAFKA-6586 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-6577) Connect standalone SASL file source and sink test fails without explanation

2018-02-21 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6577: Summary: Connect standalone SASL file source and sink test fails without explanation Key: KAFKA-6577 URL: https://issues.apache.org/jira/browse/KAFKA-6577 Project

[jira] [Created] (KAFKA-6578) Connect distributed and standalone worker 'main()' methods should catch and log all exceptions

2018-02-21 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6578: Summary: Connect distributed and standalone worker 'main()' methods should catch and log all exceptions Key: KAFKA-6578 URL: https://issues.apache.org/jira/browse/KAFKA-6578

[jira] [Resolved] (KAFKA-4632) Kafka Connect WorkerSinkTask.closePartitions doesn't handle WakeupException

2018-02-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4632?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-4632. -- Resolution: Fixed Fix Version/s: 0.10.0.1 0.10.1.0 I'm going to close

[jira] [Created] (KAFKA-6525) Connect should allow pluggable encryption for records

2018-02-02 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6525: Summary: Connect should allow pluggable encryption for records Key: KAFKA-6525 URL: https://issues.apache.org/jira/browse/KAFKA-6525 Project: Kafka Issue

[jira] [Created] (KAFKA-6513) New Connect header support doesn't define `converter.type` property correctly

2018-01-31 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6513: Summary: New Connect header support doesn't define `converter.type` property correctly Key: KAFKA-6513 URL: https://issues.apache.org/jira/browse/KAFKA-6513 Project

[jira] [Resolved] (KAFKA-6079) Idempotent production for source connectors

2018-01-29 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6079?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6079. -- Resolution: Won't Fix Marking this as WONTFIX since it is *already possible since AK 0.11

[jira] [Resolved] (KAFKA-6484) 'ConsumerGroupCommand' performance optimization for old consumer describe group

2018-01-29 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6484?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6484. -- Resolution: Duplicate > 'ConsumerGroupCommand' performance optimization for old consu

Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2018-01-26 Thread Randall Hauch
, Randall On Thu, Oct 5, 2017 at 12:13 AM, Randall Hauch <rha...@gmail.com> wrote: > Oops. Yes, I meant “replication factor”. > > > On Oct 4, 2017, at 7:18 PM, Ted Yu <yuzhih...@gmail.com> wrote: > > > > Randall: > > bq. AdminClient current

Re: [VOTE] KIP-212: Enforce set of legal characters for connector names

2018-01-23 Thread Randall Hauch
o> wrote: > > > +1 (binding) > > > > On Tue, Jan 23, 2018, 1:17 PM Randall Hauch <rha...@gmail.com> wrote: > > > > > +1 (non-binding) > > > > > > On Mon, Jan 22, 2018 at 6:35 PM, Sönke Liebau < > > > soenke.lie...@opencore.com.

Re: [VOTE] KIP-145: Expose Record Headers in Kafka Connect

2018-01-23 Thread Randall Hauch
> for the simpler API unless you can think of a good reason not to. > > -Jason > > On Tue, Jan 23, 2018 at 8:47 AM, Randall Hauch <rha...@gmail.com> wrote: > > > I mostly just followed the pattern of the Converter methods, which > > also take the individual components. Re

Re: [VOTE] KIP-212: Enforce set of legal characters for connector names

2018-01-23 Thread Randall Hauch
+1 (non-binding) On Mon, Jan 22, 2018 at 6:35 PM, Sönke Liebau < soenke.lie...@opencore.com.invalid> wrote: > All, > > this KIP has been discussed for quite some time now and I believe we > addressed all major concerns in the current revision, so I'd like to > start a vote. > > KIP can be found

Re: [VOTE] KIP-145: Expose Record Headers in Kafka Connect

2018-01-23 Thread Randall Hauch
an 21, 2018 at 7:26 PM, Ewen Cheslack-Postava < > > > e...@confluent.io> > > > > wrote: > > > > > > > > > +1 (binding) > > > > > > > > > > Thanks for the work on this -- not a small upgrade to the Connect > > APIs! >

[jira] [Resolved] (KAFKA-6466) Kafka connect task sometimes fails on start-up

2018-01-22 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6466. -- Resolution: Duplicate > Kafka connect task sometimes fails on start

Re: [DISCUSS] KIP-212: Enforce set of legal characters for connector names

2018-01-19 Thread Randall Hauch
t; >> > > > >> > > > >> > > > > >> > In general my default approach (without context of a specific > > system) > > > >> would > > > >> > be to accept anything that we can encode in UTF-8 and only apply &g

[VOTE] KIP-145: Expose Record Headers in Kafka Connect

2018-01-19 Thread Randall Hauch
Hi everyone, I'd like to start the voting on this KIP to add support for headers in Connect.: *https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect *

Re: [DISCUSS] KIP-208: Add SSL support to Kafka Connect REST interface

2018-01-18 Thread Randall Hauch
Jakub, have you had a chance to update the KIP with the latest changes? Would be great to start the vote today so that it's open long enough to adopt before the deadline on Tuesday. Let me know if I can help. On Wed, Jan 17, 2018 at 1:25 AM, Jakub Scholz wrote: > I have been

Re: [DISCUSS] KIP 145 - Expose Record Headers in Kafka Connect

2018-01-17 Thread Randall Hauch
://github.com/apache/kafka/pull/4319 Still looking for feedback on my previous email and these updates. Randall On Wed, Jan 17, 2018 at 10:57 AM, Randall Hauch <rha...@gmail.com> wrote: > Ewen and Michael, > > Thanks for the feedback. I've updated the KIP based upon your feedback. >

<    1   2   3   4   5   6   7   8   >