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

2020-05-15 Thread Konstantine Karantasis
I was on the fence between the various overloading methods myself, liking `start(...)` the least. Initially, I thought we were interested in offering the ability to call the reporter out of band, outside `put`. But after your replies I understand you don't think that's the case, and I also agree t

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

2020-05-15 Thread John Roesler
Thanks for the KIP, Randall! I've just taken a look, and I'm +1 (binding) as well. Thanks, -John On Fri, May 15, 2020, at 11:50, Bill Bejeck wrote: > Thanks for clear KIP Randall, it clearly demonstrates the need for this > change. > > +1 (binding) > > -Bill > > On Tue, May 12, 2020 at 2:02 P

Re: [VOTE] KIP-596 Safely abort Producer transactions during application shutdown

2020-05-15 Thread Matthias J. Sax
+1 (binding) -Matthias On 5/15/20 8:20 AM, Boyang Chen wrote: > Thanks for the KIP Xiang, +1 (non-binding) from me! > > On Fri, May 15, 2020 at 2:05 AM 张祥 wrote: > >> Hi all, >> >> I'd like to start a vote for KIP-586. You can find the link for this KIP >> here: >> >> https://cwiki.apache.org/

Re: [DISCUSS] KIP-608: Add a new method to AuthorizerServerInfo Interface

2020-05-15 Thread Jeff Huang
Hi David, Thanks for your suggestions. I updated KIP based on your comments 1,2 and 4. For comment 3), I think the approach which Authorizer extend Monitorable would be simple for broker calling monitor function without checking whether Object implemented Monitorable function or not and make it

[VOTE] KIP-572: Improve timeouts and retires in Kafka Streams

2020-05-15 Thread Matthias J. Sax
Hi, I would like to start the vote on KIP-572: https://cwiki.apache.org/confluence/display/KAFKA/KIP-572%3A+Improve+timeouts+and+retries+in+Kafka+Streams -Matthias signature.asc Description: OpenPGP digital signature

[jira] [Resolved] (KAFKA-9521) Flaky Test DeleteConsumerGroupsTest#testDeleteCmdEmptyGroup

2020-05-15 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-9521. Resolution: Duplicate > Flaky Test DeleteConsumerGroupsTest#testDeleteCmdEmptyGroup

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

2020-05-15 Thread Bill Bejeck
Thanks for clear KIP Randall, it clearly demonstrates the need for this change. +1 (binding) -Bill On Tue, May 12, 2020 at 2:02 PM Randall Hauch wrote: > FYI: I made a few clarifications to more carefully describe what happens > with extra topic settings that the broker doesn't accept. > > Fir

[jira] [Resolved] (KAFKA-8174) Can't call arbitrary SimpleBenchmarks tests from streams_simple_benchmark_test.py

2020-05-15 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8174?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-8174. Resolution: Won't Do > Can't call arbitrary SimpleBenchmarks tests from > streams_

[jira] [Resolved] (KAFKA-9001) Flaky Test KStreamAggregationIntegrationTest#shouldReduceSessionWindows

2020-05-15 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-9001. Resolution: Cannot Reproduce > Flaky Test KStreamAggregationIntegrationTest#shouldR

[jira] [Resolved] (KAFKA-9132) Refactor StreamThread to take advantage of new ConsumerRebalanceListener exception handling

2020-05-15 Thread Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sophie Blee-Goldman resolved KAFKA-9132. Fix Version/s: 2.6.0 Resolution: Fixed > Refactor StreamThread to take advan

Re: Consumer hang-up in case of unclean leader election

2020-05-15 Thread Sophie Blee-Goldman
Hey Dmitry, Can you open a ticket at https://issues.apache.org/jira/issues/ and include all this information so we can track and look into it? Thanks! Sophie On Fri, May 15, 2020 at 2:26 AM Dmitry Sorokin wrote: > According to documentation, in case if `auto.offset.reset` is set > to none or n

Re: [VOTE] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-15 Thread Sophie Blee-Goldman
Thanks Bruno! +1 (non-binding) Sophie On Fri, May 15, 2020 at 8:15 AM Bill Bejeck wrote: > Thanks for the KIP! > > +1 (binding) > > -Bill > > On Fri, May 15, 2020 at 11:12 AM Guozhang Wang wrote: > > > +1. > > > > Thanks! > > > > On Fri, May 15, 2020 at 1:36 AM Bruno Cadonna > wrote: > > > >

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

2020-05-15 Thread Randall Hauch
Konstantine said: > I notice Randall also used BiFunction in his example, I wonder if it's for > similar reasons. > Nope. Just a typo on my part. There appear to be three outstanding questions. First, Konstantine suggested calling this "failedRecordReporter". I think this is minor, but using th

Re: [VOTE] KIP-596 Safely abort Producer transactions during application shutdown

2020-05-15 Thread Boyang Chen
Thanks for the KIP Xiang, +1 (non-binding) from me! On Fri, May 15, 2020 at 2:05 AM 张祥 wrote: > Hi all, > > I'd like to start a vote for KIP-586. You can find the link for this KIP > here: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+

Re: [VOTE] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-15 Thread Bill Bejeck
Thanks for the KIP! +1 (binding) -Bill On Fri, May 15, 2020 at 11:12 AM Guozhang Wang wrote: > +1. > > Thanks! > > On Fri, May 15, 2020 at 1:36 AM Bruno Cadonna wrote: > > > Hi all, > > > > I'd like to call for votes on > > > > KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafk

Re: [VOTE] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-15 Thread Guozhang Wang
+1. Thanks! On Fri, May 15, 2020 at 1:36 AM Bruno Cadonna wrote: > Hi all, > > I'd like to call for votes on > > KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams > > The KIP can be found here > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+

Re: [DISCUSS] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-15 Thread Bill Bejeck
Thanks for the KIP, Bruno. Having sensible, easy to access RocksDB memory reporting will be a welcomed addition. FWIW I also agree to have the metrics reported on a store level. I'm glad you changed the KIP to that effect. -Bill On Wed, May 13, 2020 at 6:24 PM Guozhang Wang wrote: > Hi Bruno

Re: [DISCUSS] KIP-608: Add a new method to AuthorizerServerInfo Interface

2020-05-15 Thread David Jacot
Hey Jeff, Thanks for the updated KIP. The interface looks good to me. I've got couple of comments: 1. You say: "Other broker or client plugins could potentially implement the interface and get the broker's or client's Metrics instance to add additional metrics from sub-components." Does it imply

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

2020-05-15 Thread Bill Bejeck
Thanks for the KIP. This is a great improvement, +1(binding) -Bill On Fri, May 15, 2020 at 8:02 AM Andrew Schofield wrote: > +1 (non-binding). Looks like a good addition. > > Andrew Schofield > > On 14/05/2020, 06:19, "Konstantine Karantasis" > wrote: > > I think this improvement makes t

[jira] [Created] (KAFKA-10004) using kafka-configs.sh --describe for brokers will have error when querying default broker

2020-05-15 Thread Luke Chen (Jira)
Luke Chen created KAFKA-10004: - Summary: using kafka-configs.sh --describe for brokers will have error when querying default broker Key: KAFKA-10004 URL: https://issues.apache.org/jira/browse/KAFKA-10004

Re: [DISCUSS] KIP-554: Add Broker-side SCRAM Config API

2020-05-15 Thread Rajini Sivaram
Hi Colin, We have used different approaches for kafka-configs using ZooKeeper and using brokers until now. This is based on the fact that whatever you can access using kafka-configs with ZooKeeper, you can also access directly using ZooKeeper shell. For example, you can retrieve any config stored

Re: [VOTE] KIP 585: Filter and conditional SMTs

2020-05-15 Thread Andrew Schofield
+1 (non-binding) Thanks for the KIP. This will be very useful. Andrew Schofield On 13/05/2020, 10:14, "Tom Bentley" wrote: Hi, I'd like to start a vote on KIP-585: Filter and conditional SMTs https://cwiki.apache.org/confluence/display/KAFKA/KIP-585%3A+Filter+and+Conditional+SM

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

2020-05-15 Thread Andrew Schofield
+1 (non-binding). Looks like a good addition. Andrew Schofield On 14/05/2020, 06:19, "Konstantine Karantasis" wrote: I think this improvement makes total sense. It's interesting that it didn't accompany the initial version of this transformation. +1 (binding) Konstantine

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

2020-05-15 Thread Andrew Schofield
Hi, Randall's suggestion is really good. I think it gives the flexibility required and also keeps the interface the right way round. Thanks, Andrew Schofield On 15/05/2020, 02:07, "Aakash Shah" wrote: > Hi Randall, > > Thanks for the feedback. > > 1. This is a great suggestion, but I find t

Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-15 Thread Rajini Sivaram
Hi Cheng, Let me rephrase my question. Let's say we didn't have the case of leastLoadedNode. We are only talking about connections to a specific node (i.e. leader or controller). We have a lot of these and I want to understand the benefits of the proposed timeout in this case alone. We currently h

Re: [DISCUSS] KIP-612: Ability to Limit Connection Creation Rate on Brokers

2020-05-15 Thread Rajini Sivaram
Hi Anna, Thanks for the KIP, looks good overall. A couple of comments about per-IP connection quotas: 1) Should we consider making per-IP quota similar to other quotas? Configured as a dynamic quota for entity type IP, with per-IP limit as well as defaults? Perhaps that would fit better rather th

Consumer hang-up in case of unclean leader election

2020-05-15 Thread Dmitry Sorokin
According to documentation, in case if `auto.offset.reset` is set to none or not set, the exception is thrown to a client code, allowing to handle it in a way that client want. In case if one will take a closer look on this mechanism, it will turn out that it is not working. Starting from kafka 2.

[VOTE] KIP-596 Safely abort Producer transactions during application shutdown

2020-05-15 Thread 张祥
Hi all, I'd like to start a vote for KIP-586. You can find the link for this KIP here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+application+shutdown Thanks.

[VOTE] KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams

2020-05-15 Thread Bruno Cadonna
Hi all, I'd like to call for votes on KIP-607: Add Metrics to Record the Memory Used by RocksDB to Kafka Streams The KIP can be found here https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+to+Kafka+Streams+to+Record+the+Memory+Used+by+RocksDB The discussion thread can be

Re: [VOTE] KIP-221: Enhance KStream with Connecting Topic Creation and Repartition Hint

2020-05-15 Thread Levani Kokhreidze
Hi Matthias, Thanks for your thoughts and sorry for taking so long to respond. For me that makes total sense, so +1 from my side. I took the liberty of creating a ticket for it: https://issues.apache.org/jira/browse/KAFKA-10003 Regards, Levan

[jira] [Created] (KAFKA-10003) Deprecate KStream#through in favor of KStream#repartition

2020-05-15 Thread Levani Kokhreidze (Jira)
Levani Kokhreidze created KAFKA-10003: - Summary: Deprecate KStream#through in favor of KStream#repartition Key: KAFKA-10003 URL: https://issues.apache.org/jira/browse/KAFKA-10003 Project: Kafka

Re: [DISCUSS] KIP-554: Add Broker-side SCRAM Config API

2020-05-15 Thread Tom Bentley
Hi Colin, The AdminClient should do the hashing, right? I don't see any advantage to > doing it externally. I'm happy so long as the AdminClient interface doesn't require users to do the hashing themselves. I do think we should support setting the salt explicitly, but really only > for testing

[jira] [Resolved] (KAFKA-5601) Refactor ReassignPartitionsCommand to use AdminClient

2020-05-15 Thread Tom Bentley (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-5601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tom Bentley resolved KAFKA-5601. Resolution: Fixed Addressed by KIP-455 > Refactor ReassignPartitionsCommand to use AdminClient > -

[jira] [Resolved] (KAFKA-6379) Work for KIP-240

2020-05-15 Thread Tom Bentley (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tom Bentley resolved KAFKA-6379. Resolution: Fixed Addressed by KIP-455 > Work for KIP-240 > > > K

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

2020-05-15 Thread Konstantine Karantasis
Thanks for KIP Aakash. This proposal will address a significant gap in error handling for sink connectors, so I'd also like to see it implemented. Interesting discussion so far and I agree with a lot of what's been said. First of all I agree with Andrew. When I first read the KIP I also felt this

[jira] [Resolved] (KAFKA-6039) Improve TaskAssignor to be more load balanced

2020-05-15 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6039?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-6039. -- Resolution: Duplicate > Improve TaskAssignor to be more load balanced > ---

[jira] [Resolved] (KAFKA-5578) Streams Task Assignor should consider the staleness of state directories when allocating tasks

2020-05-15 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-5578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-5578. -- Resolution: Duplicate > Streams Task Assignor should consider the staleness of state directorie

[jira] [Created] (KAFKA-10002) Improve performances of StopReplicaRequest with large number of partitions to be deleted

2020-05-15 Thread David Jacot (Jira)
David Jacot created KAFKA-10002: --- Summary: Improve performances of StopReplicaRequest with large number of partitions to be deleted Key: KAFKA-10002 URL: https://issues.apache.org/jira/browse/KAFKA-10002

<    1   2