Re: [VOTE] 2.6.2 RC0

2021-03-22 Thread Ewen Cheslack-Postava
Hey Justin, Can you clarify a few things? - On the first point, the only thing I see is zstd, which we do not, in fact, ship the library itself, just the jni bindings. Was there anything else you saw? - On the second point, I checked the src download from the RC and afaict we use this header and

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

2020-05-19 Thread Ewen Cheslack-Postava
+1 (binding) This will be a nice improvement. From the discussion thread it's clear this is tricky to get right, nice work! On Tue, May 19, 2020 at 8:16 AM Andrew Schofield wrote: > +1 (non-binding) > > This is now looking very nice. > > Andrew Schofield > > On 19/05/2020, 16:11, "Randall

Re: [VOTE] KIP-531: Drop support for Scala 2.11 in Kafka 2.5

2019-12-17 Thread Ewen Cheslack-Postava
+1 (binding), though there is definitely still some compatibility risk -- regardless of updates, people are going to have apps that may need other updates but upgrading between scala versions is still too painful for too long to warrant the effort to update. But in that case they probably aren't

Re: [VOTE] KIP-507: Securing Internal Connect REST Endpoints

2019-09-24 Thread Ewen Cheslack-Postava
+1 (binding) A couple of comments on the details: - I think this might be the first time we're adding a new message type to the config topic. Might be worth noting that (somewhat unfortunately) we currently log unknown keys as an error. Kinda unclear whether it should be an error or warn (since

Re: [VOTE] 2.2.0 RC2

2019-03-21 Thread Ewen Cheslack-Postava
+1 -Ewen On Thu, Mar 21, 2019 at 10:33 AM Harsha wrote: > +1 (non-bidning) > - Download artifacts, setup 3 node cluster > - Ran producer/consumer clients > > Thanks, > Harsha > > On Thu, Mar 21, 2019, at 5:54 AM, Andrew Schofield wrote: > > +1 (non-binding) > > > > - Downloaded the artifacts

[jira] [Resolved] (KAFKA-7813) JmxTool throws NPE when --object-name is omitted

2019-03-17 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7813?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7813. -- Resolution: Fixed Fix Version/s: 2.3.0 Issue resolved by pull request

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

2019-03-14 Thread Ewen Cheslack-Postava
+1 (binding) -Ewen On Wed, Mar 13, 2019 at 2:04 PM Randall Hauch wrote: > 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

[jira] [Resolved] (KAFKA-7834) Extend collected logs in system test services to include heap dumps

2019-02-04 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7834?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7834. -- Resolution: Fixed Fix Version/s: (was: 1.1.2

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

2019-01-25 Thread Ewen Cheslack-Postava
I was going to make a related comment about connect.protocol. Even if we have the config, we should default it to compatible given v0 state is small and we believe v1 is better and people should migrate to it. While I like getting rid of configs, not sure whether we should remove it here. If we

Re: [VOTE] KIP-421: Support resolving externalized secrets in AbstractConfig

2019-01-24 Thread Ewen Cheslack-Postava
> It allows _all_ existing clients of the class, e.g. those in Apache Kafka or in applications written by other people that use the class, to get this functionality for free, i.e. without any code changes. (I realize this is probably where the 'unexpected effects' comes from). > Personally, I

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

2019-01-23 Thread Ewen Cheslack-Postava
+1 (binding) My only final comment would be that the topic.creation.enable setting could potentially be left out. We're still backwards compatible with what we promised before (or at least compatibility is debatable). You could enforce not being able to create topics with ACLs anyway, so avoiding

[jira] [Resolved] (KAFKA-7461) Connect Values converter should have coverage of logical types

2019-01-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7461. -- Resolution: Fixed Fix Version/s: 2.2.0 Issue resolved by pull request

[jira] [Resolved] (KAFKA-7503) Integration Test Framework for Connect

2019-01-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7503. -- Resolution: Fixed Fix Version/s: 2.0.2 2.1.1

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2019-01-04 Thread Ewen Cheslack-Postava
Hey Ryanne, Sorry, late to the game here. On the ACL management, can you explain how things are supposed to work when you need to migrate to the new cluster? Or is this purely for a mirroring but not DR and failover cases? In particular, the rules outlined state that only MM2 would be able to

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

2019-01-04 Thread Ewen Cheslack-Postava
Hi Paul, Thanks for the KIP. A few comments. To me, biggest question here is if we can fix this behavior without adding a config. In particular, today, we don't even set the client.id for the producer and consumer at all, right? The *only* way it is set is if you include an override in the

[jira] [Resolved] (KAFKA-7551) Refactor to create both producer & consumer in Worker

2018-11-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7551?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7551. -- Resolution: Fixed Merged [https://github.com/apache/kafka/pull/5842,] my bad

[jira] [Resolved] (KAFKA-7620) ConfigProvider is broken for KafkaConnect when TTL is not null

2018-11-27 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7620. -- Resolution: Fixed Fix Version/s: 2.0.2 2.1.1

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

2018-11-27 Thread Ewen Cheslack-Postava
re: AdminClient vs this proposal, one consideration is that AdminClient exposes a lot more surface area and probably a bunch of stuff we actually don't want Connectors to be able to do, such as deleting topics. You can always lock down by ACLs, but what the framework enables directly vs requiring

[jira] [Resolved] (KAFKA-7560) PushHttpMetricsReporter should not convert metric value to double

2018-11-07 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7560?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7560. -- Resolution: Fixed Fix Version/s: 2.1.0 2.2.0 Issue

Re: [VOTE] 2.0.1 RC0

2018-11-02 Thread Ewen Cheslack-Postava
+1 -Ewen On Thu, Nov 1, 2018 at 10:10 AM Manikumar wrote: > We were waiting for the system test results. There were few failures: > KAFKA-7579, KAFKA-7559, KAFKA-7561 > they are not blockers for 2.0.1 release. We need more votes from > PMC/committers :) > > Thanks Stanislav! for the system

[jira] [Resolved] (KAFKA-6490) JSON SerializationException Stops Connect

2018-10-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6490. -- Resolution: Fixed Fix Version/s: 2.0.0 Closing as this is effectively

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

2018-10-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-5117. -- Resolution: Duplicate Assignee: Ewen Cheslack-Postava Fix

[jira] [Resolved] (KAFKA-7476) SchemaProjector is not properly handling Date-based logical types

2018-10-04 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7476?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7476. -- Resolution: Fixed Fix Version/s: 2.1.0 0.10.2.3

Re: Anyone interested in helping out a little brother Apache project with their Kafka integration?

2018-09-30 Thread Ewen Cheslack-Postava
Chris, Can you point at the starting point for your Connector? Happy to do a quick review of code. I'll admit, I'm a bit confused by the PLC4X positioning -- it seems to claim to be universal IoT protocol, but then I see, e.g., nothing about MQTT on the front page, a protocol that Kafka Connect

Re: Apache Kafka project charter

2018-09-30 Thread Ewen Cheslack-Postava
Hey all, Sorry I haven't been closely following the threads on this, but I think I can provide a bit more color. Jakub, re: general policy, I'll take the blame that the relevant "rejected alternatives" section in the KIP

[jira] [Resolved] (KAFKA-7460) Connect Values converter uses incorrect date format string

2018-09-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7460?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7460. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.1

[jira] [Created] (KAFKA-7461) Connect Values converter should have coverage of logical types

2018-09-30 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-7461: Summary: Connect Values converter should have coverage of logical types Key: KAFKA-7461 URL: https://issues.apache.org/jira/browse/KAFKA-7461 Project

[jira] [Created] (KAFKA-7460) Connect Values converter uses incorrect date format string

2018-09-30 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-7460: Summary: Connect Values converter uses incorrect date format string Key: KAFKA-7460 URL: https://issues.apache.org/jira/browse/KAFKA-7460 Project

[jira] [Resolved] (KAFKA-7434) DeadLetterQueueReporter throws NPE if transform throws NPE

2018-09-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7434?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7434. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.1 Issue

[jira] [Resolved] (KAFKA-4932) Add UUID Serde

2018-09-09 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4932?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4932. -- Resolution: Fixed Issue resolved by pull request 4438 [https://github.com

[jira] [Resolved] (KAFKA-7353) Connect logs 'this' for anonymous inner classes

2018-09-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7353?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7353. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.1

[jira] [Resolved] (KAFKA-7242) Externalized secrets are revealed in task configuration

2018-08-28 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7242?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7242. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.1 Issue

Re: [DISCUSS] add connect-related packages to "What is considered a "major change" that needs a KIP?"

2018-08-07 Thread Ewen Cheslack-Postava
First, I agree, updating that list would be a good idea. It's likely it will always be a little divergent from any new additions -- the last update was probably when the KIP page was originally created, before either Connect or Streams existed. However, note that we also document the exact set of

[jira] [Resolved] (KAFKA-7225) Kafka Connect ConfigProvider not invoked before validation

2018-08-07 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7225?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7225. -- Resolution: Fixed > Kafka Connect ConfigProvider not invoked before validat

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

2018-08-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7228?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7228. -- Resolution: Fixed > DeadLetterQueue throws a NullPointerExcept

Re: [Vote] KIP-321: Update TopologyDescription to better represent Source and Sink Nodes

2018-07-31 Thread Ewen Cheslack-Postava
Generally +1 (binding) It would be helpful to just provide the full, updated interfaces in the KIP and mark things as new with comments if needed. I had to go back and read the discussion thread to make sure I was understanding the intent correctly. Damian -- if we make that Optional, shouldn't

[jira] [Resolved] (KAFKA-7068) ConfigTransformer doesn't handle null values

2018-06-17 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7068?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7068. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request

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

2018-06-16 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7047. -- Resolution: Fixed Reviewer: Ewen Cheslack-Postava Fix Version/s

[jira] [Resolved] (KAFKA-7039) DelegatingClassLoader creates plugin instance even if its not Versioned

2018-06-16 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7039. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request

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

2018-06-15 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7056. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request

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

2018-06-12 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7009?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7009. -- Resolution: Fixed Fix Version/s: 2.1.0 0.11.0.3

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

2018-06-12 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7031?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7031. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request

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

2018-06-12 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7043. -- Resolution: Fixed Fix Version/s: 2.1.0 Issue resolved by pull request

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

2018-06-11 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7003?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-7003. -- Resolution: Fixed Fix Version/s: 2.1.0 2.0.0 Issue

[jira] [Resolved] (KAFKA-6997) Kafka run class doesn't exclude test-sources jar

2018-06-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6997?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6997. -- Resolution: Fixed Issue resolved by pull request 5139 [https://github.com

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

2018-06-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6981?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6981. -- Resolution: Fixed Issue resolved by pull request 5125 [https://github.com

Re: [VOTE] KIP-176: Remove deprecated new-consumer option for tools

2018-05-25 Thread Ewen Cheslack-Postava
+1 (binding) Just follow up on the existing version of the KIP, so nothing new here. Possibly a bit disruptive given how quick the 1.0 -> 2.0 jump happened, but it's the right time to remove it. -Ewen On Thu, May 24, 2018 at 8:13 AM Viktor Somogyi wrote: > +1

Re: [EXTERNAL] Kafka Connect: New Kafka Source Connector

2018-05-22 Thread Ewen Cheslack-Postava
Sorry for the delay, didn't see this until now. I've given you edit permissions on the wiki. -Ewen On Tue, May 15, 2018 at 2:21 PM McCaig, Rhys wrote: > Hi Team, > > Would someone be able to provide me with Confluence permission in order to > write a KIP for the below

[jira] [Resolved] (KAFKA-5807) Check Connector.config() and Transformation.config() returns a valid ConfigDef

2018-05-22 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5807?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-5807. -- Resolution: Fixed Fix Version/s: 2.0.0 Issue resolved by pull request

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

2018-05-22 Thread Ewen Cheslack-Postava
+1 (binding) -Ewen On Tue, May 22, 2018 at 9:29 AM Ted Yu wrote: > +1 > > On Tue, May 22, 2018 at 9:19 AM, Randall Hauch wrote: > > > I'd like to start a vote of a very straightforward proposal for Connect > to > > add converters for the basic primitive

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

2018-05-21 Thread Ewen Cheslack-Postava
On Sat, May 19, 2018 at 6:20 AM Randall Hauch <rha...@gmail.com> wrote: > Considering this KIP is straightforward, what do you think about kicking > off a vote? Or does it need more discussion time? > > Regards, > Randall > > > On May 18, 2018, at 4:30 PM, Ewen Chesl

Re: [DISCUSS] KIP-304: Connect runtime mode improvements for container platforms

2018-05-21 Thread Ewen Cheslack-Postava
Hey all, I think think this is a great discussion, and is helping to clarify the real pain points as well as explore a few more options than just what was initially proposed. Stephane, I think why you're ending up in "grand redesign" state is because you're highlighting (and the KIP's motivation

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

2018-05-21 Thread Ewen Cheslack-Postava
e limited enough to just be an enum. Not sure if anyone has use cases for larger positive values. -Ewen > > Best, > > > On Mon, May 21, 2018 at 11:28 AM, Ewen Cheslack-Postava <e...@confluent.io > > > wrote: > > > Arjun, > > > > Understood on retries

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

2018-05-21 Thread Ewen Cheslack-Postava
+1 binding. I had one last comment in the DISCUSS thread, but not really a blocker. -Ewen On Mon, May 21, 2018 at 9:48 AM Matthias J. Sax wrote: > +1 (binding) > > > > On 5/21/18 9:30 AM, Randall Hauch wrote: > > Thanks, Arjun. +1 (non-binding) > > > > Regards, > >

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

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

2018-05-18 Thread Ewen Cheslack-Postava
rdes is useful, so > I've kept all 5 converters in the KIP. > > Interestingly, perhaps the short and int converters (with the reduced > ranges) are not necessarily that useful for keys either. > > Regards, > > Randall > > On Thu, May 17, 2018 at 10:08 PM, Ewen Cheslack-Po

[jira] [Resolved] (KAFKA-6566) SourceTask#stop() not called after exception raised in poll()

2018-05-18 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6566. -- Resolution: Fixed > SourceTask#stop() not called after exception raised in p

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

2018-05-18 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: [VOTE] KIP-285: Connect Rest Extension Plugin

2018-05-17 Thread Ewen Cheslack-Postava
+1 (binding) Thanks, Ewen On Thu, May 17, 2018 at 12:16 PM Ted Yu wrote: > +1 > Original message From: Gwen Shapira > Date: 5/17/18 12:02 PM (GMT-08:00) To: dev > Subject: Re: [VOTE] KIP-285: Connect Rest

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

2018-05-17 Thread Ewen Cheslack-Postava
Yup, thanks for the changes. The 'health' package in particular feels like a nice fit given the way we expect it to be used. -Ewen On Wed, May 16, 2018 at 7:02 PM Randall Hauch wrote: > Looks good to me. Thanks for quickly making the changes! Great work! > > Best regards, > >

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

2018-05-17 Thread Ewen Cheslack-Postava
Thanks for addressing this Robert, it's a pretty common user need. First, +1 (binding) generally. Two very minor comments that I think could be clarified but wouldn't affect votes: * Let's list in the KIP what package the ConfigProvider, ConfigChangeCallback, ConfigData and ConfigTransformer

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

2018-05-17 Thread Ewen Cheslack-Postava
Just a couple of minor points that don't really affect the implementation: * For nulls, let's just mention the underlying serializers already support this. I'm actually not sure why they should/need to, but given they do, let's just defer to that implementation. * I'm not sure where Float and

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

2018-05-16 Thread Ewen Cheslack-Postava
Hey, Sorry for the late follow up. I just had a couple of minor questions about details: * Some of the public API being added is under a runtime package. But that would be new for public API -- currently only things under the runtime package use that package name. I think changing the package

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

2018-04-29 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-5141. -- Resolution: Fixed Assignee: Ewen Cheslack-Postava Not sure of the fix

Re: [DISCUSS] KIP-276: Add StreamsConfig prefix for different consumers

2018-04-11 Thread Ewen Cheslack-Postava
for > > different types of consumers, and they would go ahead and learn about the > > other three prefixes and set them there. > > > > > > I agree that four prefixes would be more confusing, but if we think use > > case 1)'s popularity is much larger tha

Re: [DISCUSS] KIP-276: Add StreamsConfig prefix for different consumers

2018-04-05 Thread Ewen Cheslack-Postava
I think this model is more confusing than it needs to be. We end up with 4 prefixes despite only have 3 types of consumers. We have prefixes for: "base", "main", "global", and "restore". However, we only instantiate consumers of type "main", "global", and "restore". Until now, we've only had two

[jira] [Resolved] (KAFKA-6728) Kafka Connect Header Null Pointer Exception

2018-04-03 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6728. -- Resolution: Fixed Fix Version/s: 1.1.1 1.2.0 Issue

Re: [VOTE] KIP-272: Add API version tag to broker's RequestsPerSec metric

2018-03-30 Thread Ewen Cheslack-Postava
+1 (binding) The incompatibility is unfortunate, but seems unlikely to cause a problem in practice. Let's just make sure there's a note in the upgrade notes about the incompatibility when we have a PR for this. -Ewen On Fri, Mar 30, 2018 at 10:22 AM, Jun Rao wrote: > Hi,

Re: [DISCUSSION] KIP-266: Add TimeoutException to KafkaConsumer#position()

2018-03-23 Thread Ewen Cheslack-Postava
Regarding the flexibility question, has someone tried to dig up the discussion of the new consumer APIs when they were being written? I vaguely recall these exact questions about using APIs vs configs and flexibility vs bloating the API surface area having already been discussed. (Not that we

Re: Kafka Connect task re-balance repeatedly

2018-03-22 Thread Ewen Cheslack-Postava
The log is showing that the Connect worker is trying to make sure it has read the entire log and gets to offset 119, but some other worker says it has read to offset 169. The two are in inconsistent states, so the one that seems to be behind will not start work with potentially outdated

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

2018-03-19 Thread Ewen Cheslack-Postava
this is opt-in, and that by default > > nothing will change? > > > > Randall > > > > On Tue, Jan 16, 2018 at 11:18 PM, Ewen Cheslack-Postava < > e...@confluent.io> > > wrote: > > > >> Vincent, > >> > >> I think with the ad

Re: Seeking Feedback on Kafka Connect Issues

2018-03-19 Thread Ewen Cheslack-Postava
Responses inline. On Mon, Mar 19, 2018 at 3:02 PM, Matt Farmer wrote: > Hi everyone, > > We’ve been experimenting recently with some limited use of Kafka Connect > and are hoping to expand to wider use cases soon. However, we had some > internal issues that gave us a well-timed

[jira] [Created] (KAFKA-6676) System tests do not handle ZK chroot properly with SCRAM

2018-03-17 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-6676: Summary: System tests do not handle ZK chroot properly with SCRAM Key: KAFKA-6676 URL: https://issues.apache.org/jira/browse/KAFKA-6676 Project: Kafka

Re: [VOTE] KIP-186: Increase offsets retention default to 7 days

2018-03-09 Thread Ewen Cheslack-Postava
> > avoid a lot of pain. Thanks for reviving it Ewen. > > > > > > -Jay > > > > > > On Mon, Mar 5, 2018 at 11:35 AM, Ewen Cheslack-Postava < > > e...@confluent.io> > > > wrote: > > > > > >> I'd like to kick off voting f

[jira] [Resolved] (KAFKA-5999) Offset Fetch Request

2018-03-06 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5999?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-5999. -- Resolution: Invalid Assignee: Ewen Cheslack-Postava Closing as Invalid

Re: Exposing additional metadata in Kafka Connect schema parameters

2018-03-06 Thread Ewen Cheslack-Postava
I can only speak to my view on it, but I view logical types as truly generic. The point is that they can be handled as the underlying type safely, processed as such as needed, but if the logical type is properly preserved they can properly translate through as the more "complicated" type. To me,

[ANNOUNCE] Apache Kafka 1.0.1 Released

2018-03-06 Thread Ewen Cheslack-Postava
, Ewen Cheslack-Postava, Filipe Agapito, fredfp, Guozhang Wang, huxihx, Ismael Juma, Jason Gustafson, Jeremy Custenborder, Jiangjie (Becket) Qin, Joel Hamill, Konstantine Karantasis, lisa2lisa, Logan Buckley, Manjula K, Matthias J. Sax, Nick Chiu, parafiend, Rajini Sivaram, Randall Hauch, Robert

[jira] [Resolved] (KAFKA-5471) Original Kafka paper link broken

2018-03-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-5471. -- Resolution: Fixed Assignee: Ewen Cheslack-Postava Updated the link

[VOTE] KIP-186: Increase offsets retention default to 7 days

2018-03-05 Thread Ewen Cheslack-Postava
I'd like to kick off voting for KIP-186: https://cwiki.apache.org/confluence/display/KAFKA/KIP-186%3A+Increase+offsets+retention+default+to+7+days This is the trivial fix that people in the DISCUSS thread were in favor of. There are some ideas for further refinements, but I think we can follow up

Re: [DISCUSS] KIP-186: Increase offsets retention default to 7 days

2018-03-05 Thread Ewen Cheslack-Postava
gt; from such groups. > >> > > > >> > > >> > I agree with Jason here, but maybe itself deserves a separate KIP > >> > discussion. > >> > > >> > > >> > > > >> > > -Jason > >> > > &

[jira] [Resolved] (KAFKA-4854) Producer RecordBatch executes callbacks with `null` provided for metadata if an exception is encountered

2018-03-02 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4854. -- Resolution: Not A Bug Assignee: Ewen Cheslack-Postava This behavior

Re: [VOTE] 1.0.1 RC2

2018-03-02 Thread Ewen Cheslack-Postava
at 8:44 PM, Jakub Scholz <ja...@scholz.cz> wrote: > > > > > +1 (non-binding) ... I used the Scala 2.12 binaries and run my tests > with > > > producers / consumers. > > > > > > On Thu, Feb 22, 2018 at 1:06 AM, Ewen Chesla

Re: Kafka 0.11.0.1 and filebeat 6.1 compatibility

2018-02-27 Thread Ewen Cheslack-Postava
filebeat is implemented using sarama last I checked, so presumably they are on a version that doesn't know about Kafka 0.11.0.1 and therefore it doesn't know which API versions to use. Not sure if they support leaving it blank or exactly how the sarama config works, but as far as I know sarama

Re: Question about developer documentation

2018-02-27 Thread Ewen Cheslack-Postava
The web page is more about general project info and might be of interest to people beyond just developers. But I agree the wiki landing page could use some updating. Even more than just the developer section as we're missing several releases, the oldest ones are listed at the top, etc. -Ewen On

[jira] [Resolved] (KAFKA-6236) stream not picking data from topic - after rebalancing

2018-02-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6236. -- Resolution: Cannot Reproduce Unresponsive, so we can't track this down. Please

[jira] [Resolved] (KAFKA-6239) Consume group hung into rebalancing state, now stream not able to poll data

2018-02-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6239. -- Resolution: Duplicate > Consume group hung into rebalancing state, now str

[jira] [Resolved] (KAFKA-6439) "com.streamsets.pipeline.api.StageException: KAFKA_50 - Error writing data to the Kafka broker: java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.N

2018-02-23 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6439. -- Resolution: Not A Bug > "com.streamsets.pipeline.api.StageException:

[jira] [Created] (KAFKA-6580) Connect bin scripts have incorrect usage

2018-02-21 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-6580: Summary: Connect bin scripts have incorrect usage Key: KAFKA-6580 URL: https://issues.apache.org/jira/browse/KAFKA-6580 Project: Kafka Issue

[VOTE] 1.0.1 RC2

2018-02-21 Thread Ewen Cheslack-Postava
/kafka/tree/1.0.1-rc2 * Documentation: http://kafka.apache.org/10/documentation.html * Protocol: http://kafka.apache.org/10/protocol.html /** Thanks, Ewen Cheslack-Postava

Re: [VOTE] 1.0.1 RC1

2018-02-20 Thread Ewen Cheslack-Postava
t; >> > > Satish. > >> > > > >> > > > >> > > On Tue, Feb 13, 2018 at 11:30 PM, Damian Guy <damian@gmail.com> > >> > wrote: > >> > > > >> > > > +1 > >> > > > > >&

[jira] [Resolved] (KAFKA-6503) Connect: Plugin scan is very slow

2018-02-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6503. -- Resolution: Fixed Fix Version/s: 1.2.0 Issue resolved by pull request

Re: [VOTE] 1.0.1 RC1

2018-02-12 Thread Ewen Cheslack-Postava
And of course I'm +1 since I've already done normal release validation before posting this. -Ewen On Mon, Feb 12, 2018 at 10:15 AM, Ewen Cheslack-Postava <e...@confluent.io> wrote: > Hello Kafka users, developers and client-developers, > > This is the second candidate for re

[VOTE] 1.0.1 RC1

2018-02-12 Thread Ewen Cheslack-Postava
/documentation.html * Protocol: http://kafka.apache.org/10/protocol.html Thanks, Ewen Cheslack-Postava

Re: [VOTE] 1.0.1 RC0

2018-02-09 Thread Ewen Cheslack-Postava
om source and running the quickstart were successful on Ubuntu > and Windows 10. > > Thanks for running the release. > --Vahid > > > > From: Ewen Cheslack-Postava <e...@confluent.io> > To: dev@kafka.apache.org, us...@kafka.apache.org, > kafka-clie...

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

2018-02-09 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6513?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-6513. -- Resolution: Fixed Fix Version/s: 1.2.0 Issue resolved by pull request

Documentation build system

2018-02-06 Thread Ewen Cheslack-Postava
Hi all, I just wrote a note in https://issues.apache.org/jira/browse/KAFKA-2967 with a proposal for changing how docs are written. I want to move on this soon if possible and normally would just leave the discussion to the JIRA, but as I think this is something everyone has an opinion on and

Re: [VOTE] 1.0.1 RC0

2018-02-05 Thread Ewen Cheslack-Postava
. This requires "closing" two separate staging repos to get everything released. If things look good now, I can update the release script/instructions to make this clearer. -Ewen On Mon, Feb 5, 2018 at 9:59 PM, Ewen Cheslack-Postava <e...@confluent.io> wrote: > Not sure, seem

Re: [VOTE] 1.0.1 RC0

2018-02-05 Thread Ewen Cheslack-Postava
.org/content/groups/staging/org/ > apache/kafka/kafka-clients/ > > I don't see 1.0.1 version. > > Cheers > > On Mon, Feb 5, 2018 at 7:48 PM, Ewen Cheslack-Postava <e...@confluent.io> > wrote: > > > Hello Kafka users, developers and client-developers, >

[VOTE] 1.0.1 RC0

2018-02-05 Thread Ewen Cheslack-Postava
Hello Kafka users, developers and client-developers, Sorry for a bit of delay, but I've now prepared the first candidate for release of Apache Kafka 1.0.1. This is a bugfix release for the 1.0 branch that was first released with 1.0.0 about 3 months ago. We've fixed 46 significant issues since

[jira] [Created] (KAFKA-6536) Streams quickstart pom.xml is missing versions for a bunch of plugins

2018-02-05 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-6536: Summary: Streams quickstart pom.xml is missing versions for a bunch of plugins Key: KAFKA-6536 URL: https://issues.apache.org/jira/browse/KAFKA-6536

  1   2   3   4   5   6   7   8   9   10   >