Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-05-26 Thread Randall Hauch
Sorry, I missed Matthias' +1 binding. I'll move the KIP back to "Adopted" and add it to the AK 2.6.0. Apologies for the noise. On Tue, May 26, 2020 at 12:14 PM Randall Hauch wrote: > Just a quick note: I've changed > https://cwiki.apache.org/confluence/display/KAFKA/KIP

Re: [VOTE] KIP-557: Add emit on change support for Kafka Streams

2020-05-26 Thread Randall Hauch
Just a quick note: I've changed https://cwiki.apache.org/confluence/display/KAFKA/KIP-557%3A+Add+emit+on+change+support+for+Kafka+Streams to denote that this KIP is still in voting, as it has only received 2 binding votes. I will also remove the KIP from the AK 2.6.0 release, since the KIP freeze

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

2020-05-24 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6755. -- Fix Version/s: 2.6.0 Reviewer: Randall Hauch Resolution: Fixed [KIP-437|https

[jira] [Resolved] (KAFKA-9767) Basic auth extension should have logging

2020-05-24 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9767?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9767. -- Fix Version/s: 2.6.0 Reviewer: Randall Hauch Resolution: Fixed Merged

[jira] [Resolved] (KAFKA-9944) Allow HTTP Response Headers to be Configured for Kafka Connect

2020-05-24 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9944?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9944. -- Reviewer: Randall Hauch Resolution: Fixed KIP-577 has been approved, and I merged the PR

[jira] [Resolved] (KAFKA-9780) Deprecate commit records without record metadata

2020-05-21 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9780?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9780. -- Reviewer: Randall Hauch Resolution: Fixed Merged to `trunk` after [KIP-586|https

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

2020-05-21 Thread Randall Hauch
get right, nice work! > > > > On Tue, May 19, 2020 at 8:16 AM Andrew Schofield < > > andrew_schofi...@live.com> > > wrote: > > > > > +1 (non-binding) > > > > > > This is now looking very nice. > > > > > > Andrew Schofield

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-05-20 Thread Randall Hauch
t thought > it better to ask than to just add it myself). > > Thanks, > > Tom > > On Tue, May 5, 2020 at 6:54 PM Randall Hauch wrote: > > > Greetings! > > > > I'd like to volunteer to be release manager for the next time-based > feature > > release which

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

2020-05-19 Thread Randall Hauch
Thank you, Aakash, for putting together this KIP and shepherding the discussion. Also, many thanks to all those that participated in the very active discussion. I'm actually very happy with the current proposal, am confident that it is a valuable improvement to the Connect framework, and know that

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

2020-05-19 Thread Randall Hauch
Thanks, Aakash, for updating the KIP. On Tue, May 19, 2020 at 2:18 AM Arjun Satish wrote: > Hi Randall, > > Thanks for the explanation! Excellent point about guaranteeing offsets in > the async case. > > If we can guarantee that the offsets will be advanced only after the bad > records are

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

2020-05-18 Thread Randall Hauch
Hi, Chris, Aakash, and others: First of all, apologies for the extremely long email. Secondly, thanks for the input on this KIP. The timing is unfortunately, but I do believe we're agreed on most points. Chris asked earlier: > I'm still unclear on how futures are going to provide any benefit to

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

2020-05-17 Thread Randall Hauch
All good points regarding `Future` instead of `Future`, so +1 to that change. A few more nits. The following sentences should be removed because they actually describe a change from the current DLQ functionality that already sets `max.in.flight.requests.per.connection=1` by default: "In order to

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

2020-05-17 Thread Randall Hauch
ecord and would then only send the 999 after that's happened. With an asynchronous `record(...)` method, the `put(...)` method could report the first record, send the 999 records, and then wait for the futures returned by the report method. > > On Sun, May 17, 2020, 9:27 AM Randall Hauch wrote:

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

2020-05-17 Thread Randall Hauch
for an asynchronous error > reporter if there were requests for it. But I agree with what you're > saying, if we can design it asynchronously, then it opens up the > possibilities to implement it in multiple ways in the future. > > Thanks, > Aakash > > On Sun, May 17, 20

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

2020-05-17 Thread Randall Hauch
On Sun, May 17, 2020 at 11:01 AM Magesh kumar Nandakumar < mage...@confluent.io> wrote: > Thanks Randall. The suggestion i made also has a problem when reporter > isn't enabled where it could potentially write records after error records > to sink before failing. > > The other concern i had with

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

2020-05-17 Thread Randall Hauch
consistencies. > > > > Overall, I think it should be safe to have new interfaces, and have them > > loaded safely, with correct checks. > > > > BTW, one more opportunity we have is that the connector can check if > these > > new methods are present or not. For example

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

2020-05-17 Thread Randall Hauch
//cwiki.apache.org/confluence/display/KAFKA/KIP-610%3A+Error+Reporting+in+Sink+Connectors > > > > > > Please let me know what you think. > > > > Thanks, > > Aakash > > > > On Sat, May 16, 2020 at 12:34 PM Randall Hauch wrote: > > > >

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

2020-05-16 Thread Randall Hauch
take a > look at it. > > Thanks, > > On Sat, May 16, 2020 at 3:39 PM Randall Hauch wrote: > > > Have you tried this? IIUC the problem is with the new type, and any class > > that uses ‘ErrantRecordReporter’ with an import would fail to be loaded > by >

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

2020-05-16 Thread Randall Hauch
the reporter in Kafka > Connect > > > versions that support this feature makes the most sense. The burden for > > > connector developers that want to use this reporter _and_ make > connectors > > > compatible with old and new workers is minimal. > > > > >

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

2020-05-16 Thread Randall Hauch
being said, I think the value added > > in > > > > > > clarity of contract of when the error reporter will be invoked > and > > > > > overall > > > > > > aesthetic while maintaining backwards compatibility outweighs the > > > >

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

2020-05-15 Thread Randall Hauch
; > Best, > > PS: Apologies if the language is incorrect or some points are unclear. > > On Fri, May 15, 2020 at 12:02 PM Randall Hauch wrote: > > > On Fri, May 15, 2020 at 1:45 PM Konstantine Karantasis < > > konstant...@confluent.io> wrote: > > > &

[jira] [Resolved] (KAFKA-9537) Abstract transformations in configurations cause unfriendly error message.

2020-05-15 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9537?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9537. -- Fix Version/s: 2.5.1 2.4.2 2.6.0 Reviewer

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

2020-05-15 Thread Randall Hauch
ous functionality available for the developer, then I am fine > with > > that as well. > > > > Lastly, I am on board with changing the name to failedRecordReporter, > > > > Please let me know your thoughts. > > > > Thanks, > > Aakash > > >

Re: [VOTE] KIP-586: Deprecate commit records without record metadata

2020-05-15 Thread Randall Hauch
This KIP is now accepted. The vote has been open for >72 hours (actually ~4 weeks), and we have three binding +1s (Konstantine, Gwen, me), one non-binding +1 (Mario), and no -1 votes. Thanks, everyone! On Thu, May 14, 2020 at 6:42 PM Gwen Shapira wrote: > +1 (binding) > > Makes a lot of sense

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

2020-05-15 Thread Randall Hauch
gt; > On 14/05/2020, 06:19, "Konstantine Karantasis" < > konstant...@confluent.io> > > wrote: > > > > I think this improvement makes total sense. It's interesting that it > > didn't > > accompany the initial version of this transformati

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

2020-05-15 Thread Randall Hauch
ssary and straightforward KIPs. > > > > Thanks for the nice write-up Randall. > > > > > > > > +1 (binding) > > > > > > > > Konstantine > > > > > > > > On Mon, May 11, 2020 at 8:00 AM Randall Hauch > wrote: > > > > > > &g

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

2020-05-15 Thread Randall Hauch
; thanks. > > > > 5. This KIP will use existing metrics as well introduce new metrics. I > will > > update this section to fully specify the metrics. > > > > Please let me know what you think. > > > > Thanks, > > Aakash > > > > On

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

2020-05-14 Thread Randall Hauch
Hi, Aakash. Thanks for the KIP. Connect does need an improved ability for sink connectors to report individual records as being problematic, and this integrates nicely with the existing DLQ feature. I also appreciate the desire to maintain compatibility so that connectors can take advantage of

Re: [VOTE] KIP-606: Add Metadata Context to MetricsReporter

2020-05-14 Thread Randall Hauch
+1 (binding) Thanks for the proposal, Xavier. On Wed, May 13, 2020 at 4:10 PM Gwen Shapira wrote: > +1 (binding) > Thanks for the proposal, Xavier. > > On Wed, May 13, 2020 at 11:54 AM Xavier Léauté wrote: > > > Hi everyone, > > > > Folks seem happy with the state of the KIP, so I'd like to

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

2020-05-12 Thread Randall Hauch
+1 (binding) > > Konstantine > > On Mon, May 11, 2020 at 8:00 AM Randall Hauch wrote: > > > Ping for reviewers. > > > > I guess I never voted, so +1 (binding). > > > > On Thu, May 7, 2020 at 4:13 PM Christopher Egerton > > wrote: > > > >

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

2020-05-11 Thread Randall Hauch
Ping for reviewers. I guess I never voted, so +1 (binding). On Thu, May 7, 2020 at 4:13 PM Christopher Egerton wrote: > 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

Re: [DISCUSS] KIP-606: Add Metadata Context to MetricsReporter

2020-05-09 Thread Randall Hauch
Thanks, Xavier. Looks great. On Fri, May 8, 2020 at 7:31 PM Xavier Léauté wrote: > > This does seem very useful. A minor request would be to mention the new > > configs for Connect, Streams and clients, specifically that because they > > are optional they will not hinder upgrades, and because

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

2020-05-08 Thread Randall Hauch
Thanks for the KIP, Chris. This will be a nice improvement on top of the work you're already doing in https://issues.apache.org/jira/browse/KAFKA-9374 Overall I like the direction this is going, but I do have some comments/questions/suggestions, all of which are pretty minor except for the last

Re: [DISCUSS] KIP-606: Add Metadata Context to MetricsReporter

2020-05-07 Thread Randall Hauch
Thanks, Xavier. This does seem very useful. A minor request would be to mention the new configs for Connect, Streams and clients, specifically that because they are optional they will not hinder upgrades, and because they are namespaced are unlikely to clash or conflict with other configs from

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

2020-05-07 Thread Randall Hauch
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 discussion thread had just a few suggestions that have already been incorporated into the KIP. Best

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

2020-05-06 Thread Randall Hauch
Thanks for starting the vote, Yu. +1 (binding) Randall On Sat, Dec 21, 2019 at 1:22 AM Yu Watanabe wrote: > Thank for the KIP. > I really want this for my project. > > +1 (non-binding) >

Re: [VOTE] KIP-586: Deprecate commit records without record metadata

2020-05-06 Thread Randall Hauch
Thanks for putting this KIP together, Mario. +1 (binding) Randall On Mon, Apr 27, 2020 at 2:05 PM Mario Molina wrote: > Hi all, > > I'd like to start a vote for KIP-586. You can find the link for this KIP > here: > >

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

2020-05-06 Thread Randall Hauch
e instead of failing > on startup. +1 non-binding when the vote thread comes > > Cheers, > > Chris > > On Mon, May 4, 2020 at 12:51 PM Randall Hauch wrote: > > > Thanks, Chris. > > > > 1. Added a use case that describes why you might want to use -1 for > > r

Re: [VOTE] KIP-577: Allow HTTP Response Headers Configured for Kafka Connect

2020-05-06 Thread Randall Hauch
Thanks for putting this together. +1 (binding) On Fri, Apr 17, 2020 at 2:02 PM Aneel Nazareth wrote: > Thanks Jeff, this seems like it addresses a user need. > > +1 (non-binding) > > On Fri, Apr 17, 2020 at 1:28 PM Zhiguo Huang > wrote: > > > > Thanks to everyone for their input. I've

[DISCUSS] Apache Kafka 2.6.0 release

2020-05-05 Thread Randall Hauch
or actively in discussion (though I'm happy to adjust as necessary). To stay on our time-based cadence, the KIP freeze is on May 20 with a target release date of June 24. Let me know if there are any objections. Thanks, Randall Hauch

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

2020-05-04 Thread Randall Hauch
nd > converters, whose properties are namespaced (presumably to avoid collisions > like this). Might be worth it to note this in a small paragraph or even > just a single sentence. > > Cheers, > > Chris > > On Thu, Apr 30, 2020 at 4:32 PM Ryanne Dolan > wrote: >

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

2020-05-04 Thread Randall Hauch
ractice for connectors that can afford to run in old and new workers (some > might not be able to work meaningfully without this KIP of course). > > Konstantine > > On Thu, Mar 5, 2020 at 11:08 AM Randall Hauch wrote: > > > Status: this KIP has 2 binding +1 votes, but has no

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

2020-04-30 Thread Randall Hauch
Hello! I'd like to use this thread to discuss KIP-605, which expands some of the properties that the Connect distributed worker uses when creating internal topics: https://cwiki.apache.org/confluence/display/KAFKA/KIP-605%3A+Expand+Connect+Worker+Internal+Topic+Settings Best regards, Randall

[jira] [Created] (KAFKA-9931) Kafka Connect should accept '-1' as a valid replication factor

2020-04-28 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9931: Summary: Kafka Connect should accept '-1' as a valid replication factor Key: KAFKA-9931 URL: https://issues.apache.org/jira/browse/KAFKA-9931 Project: Kafka

[jira] [Resolved] (KAFKA-9883) Connect request to restart task can result in IllegalArgumentError: "uriTemplate" parameter is null

2020-04-23 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9883?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9883. -- Resolution: Fixed > Connect request to restart task can result in IllegalArgumentEr

[jira] [Created] (KAFKA-9883) Connect request to restart task can result in IllegalArgumentError: "uriTemplate" parameter is null

2020-04-16 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9883: Summary: Connect request to restart task can result in IllegalArgumentError: "uriTemplate" parameter is null Key: KAFKA-9883 URL: https://issues.apache.org/jira/browse/

Re: [ANNOUNCE] Apache Kafka 2.5.0

2020-04-15 Thread Randall Hauch
; Barrett, > > > > Boyang Chen, Brian Bushree, Brian Byrne, Bruno Cadonna, Bryan Ji, > > > Chia-Ping > > > > Tsai, Chris Egerton, Chris Pettitt, Chris Stromberger, Colin P. > Mccabe, > > > > Colin Patrick McCabe, commandini, Cyrus Vafadari, Dae-Ho Kim, Davi

Re: [DISCUSS] KIP-586: Deprecate commit records without record metadata

2020-04-03 Thread Randall Hauch
Hi, Mario. Thanks for creating this small but useful KIP! The only suggestion I have is to include the JavaDoc for the method we want to deprecate so that we can include the line: * @deprecated Use {@link #commitRecord(SourceRecord, RecordMetadata)} This will make it more clear that there are

[jira] [Resolved] (KAFKA-9763) Recent changes to Connect's InsertField will fail to inject field on key of tombstone record

2020-03-25 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9763. -- Resolution: Duplicate Duplicate of KAFKA-9707, so closing this issue. > Recent chan

[jira] [Created] (KAFKA-9763) Recent changes to Connect's InsertField will fail to inject field on key of tombstone record

2020-03-25 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9763: Summary: Recent changes to Connect's InsertField will fail to inject field on key of tombstone record Key: KAFKA-9763 URL: https://issues.apache.org/jira/browse/KAFKA-9763

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

2020-03-20 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reopened KAFKA-7509: -- > Kafka Connect logs unnecessary warnings about unused configurati

Re: [DISCUSS] KIP-577: Allow HTTP Response Headers Configured for Kafka Connect

2020-03-18 Thread Randall Hauch
Thanks for the proposal, Jeff. I can see how this proposal will add value. I have a few comments, most of which are asking for more detail in the KIP. 1. I do think the KIP needs to be much more explicit about what the new configuration properties will be and to use them consistently. For

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

2020-03-05 Thread Randall Hauch
On Mon, Feb 25, 2019 at 9:36 AM Randall Hauch wrote: > +1 (binding) > > On Mon, Feb 25, 2019 at 4:32 AM Florian Hussonnois > wrote: > >> Hi Kafka Team, >> >> I'd like to bring this thread back at the top of the email stack to get a >> chance to see this KI

[jira] [Resolved] (KAFKA-9601) Workers log raw connector configs, including values

2020-02-26 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9601. -- Reviewer: Randall Hauch Resolution: Fixed Thanks for the fix, [~ChrisEgerton]! Merged

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-02-26 Thread Randall Hauch
ogs seems fairly > severe. I think should include this. Let's proceed with cherry-picking to > 2.5. > > -David > > On Wed, Feb 26, 2020 at 2:25 PM Randall Hauch wrote: > > > Hi, David. > > > > If we're still not quite ready for an RC, I'd like to squeeze in

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-02-26 Thread Randall Hauch
pproval > > > > > for this bugfix :) > > > > > > > > > > -David > > > > > > > > > > On Fri, Feb 14, 2020 at 2:21 PM Konstantine Karantasis < > > > > > konstant...@confluent.io> wrote: > > > > >

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-02-14 Thread Randall Hauch
Hi, David. I just filed https://issues.apache.org/jira/browse/KAFKA-9556 that identifies two pretty minor issues with the new KIP-558 that adds new Connect REST API endpoints to get the list of topics used by a connector. The impact is high: the feature cannot be fully disabled, and Connect does

[jira] [Created] (KAFKA-9556) KIP-558 cannot be fully disabled and when enabled topic reset not working on connector deletion

2020-02-14 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9556: Summary: KIP-558 cannot be fully disabled and when enabled topic reset not working on connector deletion Key: KAFKA-9556 URL: https://issues.apache.org/jira/browse/KAFKA-9556

[jira] [Resolved] (KAFKA-9204) ReplaceField transformation fails when encountering tombstone event

2020-02-12 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9204?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9204. -- Fix Version/s: 2.4.1 2.3.2 2.5.0 2.2.3

[jira] [Resolved] (KAFKA-9192) NullPointerException if field in schema not present in value

2020-02-12 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9192?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9192. -- Fix Version/s: 2.4.1 2.3.2 2.5.0 2.2.3

[jira] [Resolved] (KAFKA-7052) ExtractField SMT throws NPE - needs clearer error message

2020-02-11 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-7052. -- Fix Version/s: 2.4.1 2.3.2 2.5.0 2.2.3

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

2020-02-05 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7509?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-7509. -- Resolution: Won't Fix > Kafka Connect logs unnecessary warnings about unused configurati

[jira] [Resolved] (KAFKA-9074) Connect's Values class does not parse time or timestamp values from string literals

2020-02-04 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9074?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9074. -- Fix Version/s: 2.4.1 2.3.2 2.5.0 Reviewer: Jason

[jira] [Resolved] (KAFKA-9462) Correct exception message in DistributedHerder

2020-01-24 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9462. -- Reviewer: Randall Hauch Resolution: Fixed Thanks, [~yuzhih...@gmail.com]. Merged

[jira] [Resolved] (KAFKA-9024) org.apache.kafka.connect.transforms.ValueToKey throws NPE

2020-01-21 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9024?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9024. -- Reviewer: Randall Hauch Resolution: Fixed Merged to `trunk` and backported to the `2.4

[jira] [Resolved] (KAFKA-9083) Various parsing issues in Values class

2020-01-21 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9083?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9083. -- Reviewer: Randall Hauch Resolution: Fixed Merged to the `trunk`, `2.4`, `2.3`, and `2.2

Re: [VOTE] KIP-558: Add Connect REST API endpoints to view the topics used by connectors in Kafka Connect

2020-01-21 Thread Randall Hauch
Thanks again for the KIP and this improvement for Connect. +1 (binding) Randall On Tue, Jan 21, 2020 at 10:45 AM Tom Bentley wrote: > +1 (non-binding). Thanks for the KIP Konstantine. > > On Sat, Jan 18, 2020 at 2:18 AM Konstantine Karantasis < > konstant...@confluent.io> wrote: > > > Hi all,

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

2020-01-21 Thread Randall Hauch
ation of the > >> > StatusBackingStore interface. This is intentional because this > >> > implementation influences and informs the semantics of topic tracking. > >> I'd > >> > prefer not to make the language too abstract here. A KIP is not > exactly

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

2020-01-15 Thread Randall Hauch
On Wed, Jan 15, 2020 at 4:36 PM Randall Hauch wrote: > On Wed, Jan 15, 2020 at 2:05 PM Konstantine Karantasis < > konstant...@confluent.io> wrote: > >> >> 9. I assumed that partitioning is implied by default, because there's no >> requirement for complete

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

2020-01-15 Thread Randall Hauch
tric for this? I don't think it would > > satisfy > > > all the use cases you have in mind, but you could mention it in the > > > rejected alternatives. > > > > > > b) If the topic name contains the string "-connector" then the key > format >

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

2020-01-15 Thread Randall Hauch
on as a worker detects the > addition > > of a topic to a connector's set of active topics, the worker will cease > to > > post update messages to the status.storage.topic for that connector. ". > I'm > > sure I've overlooking something but why is this necessa

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

2020-01-14 Thread Randall Hauch
Oh, one more thing: 9. There's no mention of how the status topic is partitioned, or how partitioning will be used by the new topic records. The KIP should probably outline this for clarity and completeness. Best regards, Randall On Tue, Jan 14, 2020 at 5:25 PM Randall Hauch wrote: > Tha

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

2020-01-14 Thread Randall Hauch
Thanks, Konstantine. Overall, this KIP looks interesting and really useful, and for the most part is spot on. I do have a number of questions/comments about specifics: 1. The topic records have a value that includes the connector name, task number that last reported the topic is used, and

Re: [VOTE] On the new KIP-158: Kafka Connect allows source connectors to set topic settings when creating new topics

2020-01-14 Thread Randall Hauch
Thanks for the updated KIP, Konstantine. I have a few minor nits, but all are around the implementation details. +1 (binding) Best regards, Randall On Mon, Jan 13, 2020 at 10:16 AM Konstantine Karantasis < konstant...@confluent.io> wrote: > Hi everyone. > > I hope y'all had a nice break. The

CVE-2019-12399: Apache Kafka Connect REST API may expose plaintext secrets in tasks endpoint

2020-01-13 Thread Randall Hauch
CVE-2019-12399: Apache Kafka Connect REST API may expose plaintext secrets in tasks endpoint Severity: Medium Vendor: The Apache Software Foundation Versions Affected: Apache Kafka 2.0.0, 2.0.1, 2.1.0, 2.1.1, 2.2.0, 2.2.1, 2.3.0 Description: When Connect workers in Apache Kafka 2.0.0, 2.0.1,

Re: [RESULTS] [VOTE] Release Kafka version 2.4.0

2019-12-16 Thread Randall Hauch
Nice job, everyone! And thanks to Manikumar for being the release manager. On Sat, Dec 14, 2019 at 11:00 AM Adam Bellemare wrote: > Yes, thank you very much Manikumar! > > > On Dec 14, 2019, at 1:02 AM, Ismael Juma wrote: > > > > Thanks for driving the release Manikumar! > > > > Ismael > > >

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

2019-12-12 Thread Randall Hauch
> wrote: > I've taken a second look to KIP-158 after syncing with Randall Hauch, who > was the original author of the proposal, and I have updated the KIP in > place. > > The main new features of this updated KIP-158 is the introduction of groups > of configs that can be compo

[jira] [Resolved] (KAFKA-7489) ConnectDistributedTest is always running broker with dev version

2019-12-06 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-7489. -- Reviewer: John Roesler Resolution: Fixed Merged this to the `2.3`, `2.2`., and `2.1

[jira] [Resolved] (KAFKA-9184) Redundant task creation and periodic rebalances after zombie worker rejoins the group

2019-12-04 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9184?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9184. -- Reviewer: Randall Hauch Resolution: Fixed > Redundant task creation and perio

Re: [VOTE] 2.4.0 RC2

2019-12-04 Thread Randall Hauch
Ah, I merged https://issues.apache.org/jira/browse/KAFKA-9258 last night and forgot to send an email to this thread. Thanks! Randall On Wed, Dec 4, 2019 at 10:42 AM Eric Lalonde wrote: > > I have also shared a pull request for the changes I have made to the test > > script > > > >

[jira] [Resolved] (KAFKA-9258) Connect ConnectorStatusMetricsGroup sometimes NPE

2019-12-03 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9258. -- Reviewer: Randall Hauch Resolution: Fixed Merged into `trunk` and backported to the `2.4

[ANNOUNCE] Apache Kafka 2.2.2

2019-12-01 Thread Randall Hauch
to this release! A. Sophie Blee-Goldman, Matthias J. Sax, Bill Bejeck, Jason Gustafson, Chris Egerton, Boyang Chen, Alex Diachenko, cpettitt-confluent, Magesh Nandakumar, Randall Hauch, Ismael Juma, John Roesler, Konstantine Karantasis, Mickael Maison, Nacho Muñoz Gómez, Nigel Liang, Paul, Rajini Sivaram, Robert

[jira] [Resolved] (KAFKA-9051) Source task source offset reads can block graceful shutdown

2019-11-22 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9051. -- Resolution: Fixed > Source task source offset reads can block graceful shutd

[jira] [Resolved] (KAFKA-9223) RebalanceSourceConnectorsIntegrationTest disrupting builds with System::exit

2019-11-22 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9223?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9223. -- Fix Version/s: 2.3.2 Resolution: Fixed Thanks, @C0urante. This PR changes one

[jira] [Created] (KAFKA-9216) Enforce connect internal topic configuration at startup

2019-11-20 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9216: Summary: Enforce connect internal topic configuration at startup Key: KAFKA-9216 URL: https://issues.apache.org/jira/browse/KAFKA-9216 Project: Kafka Issue

Re: Subject: [VOTE] 2.2.2 RC2

2019-11-20 Thread Randall Hauch
inding) > > > > Thanks Randall. Verified signatures and tests. > > > > On Fri, Oct 25, 2019 at 7:10 AM Randall Hauch wrote: > > > > > > Hello all, we identified around three dozen bug fixes, including an > > update > > > of a third party depende

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-11-13 Thread Randall Hauch
s some credence that it's not an obvious blocker. If you think > it's a blocker, you can suggest so and provide a bit more detail on the > impact and why it's gone unnoticed until now. > > Ismael > > On Wed, Nov 13, 2019 at 10:21 AM Randall Hauch wrote: > > >

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-11-13 Thread Randall Hauch
> > > >>>> >>>> +serialization+and+deserialization> > > > > >>>> >>>> > > > > >>>> >>>> - KIP-455: Create an Administrative API for Replica > > Rea

Re: Subject: [VOTE] 2.2.2 RC2

2019-11-05 Thread Randall Hauch
; > On Fri, Oct 25, 2019 at 3:10 PM Randall Hauch wrote: > > > > Hello all, we identified around three dozen bug fixes, including an > update > > of a third party dependency, and wanted to release a patch release for > the > > Apache Kafka 2.2.0 release. > > &g

Subject: [VOTE] 2.2.2 RC2

2019-10-25 Thread Randall Hauch
216/ /** Thanks, Randall Hauch

[jira] [Created] (KAFKA-9074) Connect's Values class does not parse time or timestamp values from string literals

2019-10-21 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9074: Summary: Connect's Values class does not parse time or timestamp values from string literals Key: KAFKA-9074 URL: https://issues.apache.org/jira/browse/KAFKA-9074

[jira] [Resolved] (KAFKA-6890) Add connector level configurability for producer/consumer client configs

2019-10-16 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-6890. -- Resolution: Duplicate This has been implemented via [KIP-458|https://cwiki.apache.org

[jira] [Created] (KAFKA-9057) Backport KAFKA-8819 and KAFKA-8340 to 1.1 and 1.0 branches

2019-10-16 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-9057: Summary: Backport KAFKA-8819 and KAFKA-8340 to 1.1 and 1.0 branches Key: KAFKA-9057 URL: https://issues.apache.org/jira/browse/KAFKA-9057 Project: Kafka

[jira] [Resolved] (KAFKA-8819) Plugin path for converters not working as intended

2019-10-16 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8819?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8819. -- Reviewer: Randall Hauch Assignee: Greg Harris (was: Magesh kumar Nandakumar

[jira] [Resolved] (KAFKA-8340) ServiceLoader fails when used from isolated plugin path directory

2019-10-16 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8340?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8340. -- Reviewer: Randall Hauch Resolution: Fixed > ServiceLoader fails when used from isola

[jira] [Resolved] (KAFKA-8947) Connect framework incorrectly instantiates TaskStates for REST extensions

2019-10-16 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8947. -- Resolution: Fixed > Connect framework incorrectly instantiates TaskStates for REST extensi

[jira] [Resolved] (KAFKA-8947) Connect framework incorrectly instantiates TaskStates for REST extensions

2019-10-15 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8947. -- Resolution: Fixed > Connect framework incorrectly instantiates TaskStates for REST extensi

[jira] [Reopened] (KAFKA-8947) Connect framework incorrectly instantiates TaskStates for REST extensions

2019-10-15 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reopened KAFKA-8947: -- > Connect framework incorrectly instantiates TaskStates for REST extensi

[jira] [Resolved] (KAFKA-8945) Incorrect null check in the constructor for ConnectorHealth and AbstractState

2019-10-15 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-8945. -- Resolution: Fixed > Incorrect null check in the constructor for ConnectorHea

[jira] [Resolved] (KAFKA-9014) AssertionError thrown by SourceRecordWriteCounter when SourceTask.poll returns an empty list

2019-10-15 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9014?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9014. -- Reviewer: Randall Hauch Resolution: Fixed Merged onto the `trunk`, `2.3`, `2.2`, `2.1

<    1   2   3   4   5   6   7   8   >