Re: [VOTE] KIP-671: Add method to Shutdown entire Streams Application

2020-09-30 Thread Bruno Cadonna
Thanks a lot Walker! +1 (non-binding) Best, Bruno On 30.09.20 03:10, Matthias J. Sax wrote: Thanks Walker. The proposed API changes LGTM. +1 (binding) One minor nit: you should also mention the global-thread that also needs to be shutdown if requested by the user. Minor side question:

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-30 Thread Bruno Cadonna
, this sounds good to me. -John On Tue, Sep 29, 2020, at 03:13, Bruno Cadonna wrote: Hi all, I did two minor modifications to the KIP. - I removed the rather strict guarantee "Dead stream threads are removed from a Kafka Streams client at latest after the next call to KafkaStreams#addStreamT

[DISCUSS] KIP-674: API to Aggregate Metrics in Kafka Streams

2020-09-30 Thread Bruno Cadonna
Hi, I would like to propose the following KIP to add an API to the Kafka Streams client to aggregate metrics. https://cwiki.apache.org/confluence/display/KAFKA/KIP-674%3A+API+to+Aggregate+Metrics+in+Kafka+Streams Best, Bruno

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-29 Thread Bruno Cadonna
though it will work for many cases. So I would say it's best effort and I will mention it in the kip. As for when to use it I think we can discuss if this will be sufficient when we come to it, as long as we document its capabilities. I hope this answers your question, Walker On T

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-29 Thread Bruno Cadonna
come to it, as long as we document its capabilities. I hope this answers your question, Walker On Tue, Sep 22, 2020 at 12:33 AM Bruno Cadonna wrote: Walker, I am sorry, but I still have a comment on the KIP although you have already started voting. What happens when a consumer of the group

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-29 Thread Bruno Cadonna
g and keep the votes so far. Best, Bruno On 22.09.20 01:19, John Roesler wrote: I’m +1 also. Thanks, Bruno! -John On Mon, Sep 21, 2020, at 17:08, Guozhang Wang wrote: Thanks Bruno. I'm +1 on the KIP. On Mon, Sep 21, 2020 at 2:49 AM Bruno Cadonna wrote: Hi, I would like to restart

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-22 Thread Bruno Cadonna
update the Kip to remove the exception. I would like to add that in the case of trying to shutdown from the uncaught exception handler that we need at least one StreamThread to be alive. So having our own handler instead of using the default one after the thread has died would let us always

Re: [VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-21 Thread Bruno Cadonna
J. Sax wrote: +1 (binding) On 9/3/20 6:16 AM, Bruno Cadonna wrote: Hi, I would like to start the voting on KIP-663 that proposes to add methods to the Kafka Streams client to add and remove stream threads during execution. https://cwiki.apache.org/confluence/display/KAFKA/KIP-663%3A+API

[jira] [Created] (KAFKA-10500) Add API to Start and Stop Stream Threads

2020-09-18 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10500: - Summary: Add API to Start and Stop Stream Threads Key: KAFKA-10500 URL: https://issues.apache.org/jira/browse/KAFKA-10500 Project: Kafka Issue Type: Sub

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-17 Thread Bruno Cadonna
Hence, I backpedal and propose to filter out dead stream threads from localThreadsMetadata(). WDYT? On Wed, Sep 16, 2020 at 2:07 AM Bruno Cadonna wrote: Hi again, I just realized that if we filter out DEAD stream threads in localThreadsMetadata(), users cannot log the metadata of dying stre

Re: [DISCUSS] KIP-671: Shutdown Streams Application when appropriate exception is thrown

2020-09-16 Thread Bruno Cadonna
Hi Walker, Thank you for the KIP! I like the motivation of the KIP and the method to request a shutdown of all Kafka Streams clients of Kafka Streams application. I think we really need such functionality to react on errors. However, I am not convinced that throwing an exception to shutdown

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-16 Thread Bruno Cadonna
. Best, Bruno On 16.09.20 09:23, Bruno Cadonna wrote: Hi Guozhang, Good point! I would propose to filter out DEAD stream threads in localThreadsMetadata() to get consistent results that do not depend on timing. I will update the KIP accordingly. Best, Bruno On 16.09.20 06:02, Guozhang Wang

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-16 Thread Bruno Cadonna
. As for localThreadsMetadata() itself, I'd like to clarify if it would return any still-bookkept threads, or would it specifically filter out those DEAD threads even if they are not yet removed. Otherwise, the KIP LGTM. Guozhang On Tue, Sep 15, 2020 at 2:58 AM Bruno Cadonna wrote: Hi Guozhang

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-15 Thread Bruno Cadonna
ead threads. But the only use case I can imagine would probably be better served by a callback that gets invoked when the thread dies, which we already have. On Tue, Sep 8, 2020 at 11:46 PM Bruno Cadonna wrote: Hi Matthias and Sophie, I agree that localThreadsMetadata() can be used

[jira] [Created] (KAFKA-10484) Reduce Metrics Exposed by Streams

2020-09-15 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10484: - Summary: Reduce Metrics Exposed by Streams Key: KAFKA-10484 URL: https://issues.apache.org/jira/browse/KAFKA-10484 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-10481) Consider Improving the Success Criteria of Streams' System Test StreamsBrokerBounceTest

2020-09-14 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10481: - Summary: Consider Improving the Success Criteria of Streams' System Test StreamsBrokerBounceTest Key: KAFKA-10481 URL: https://issues.apache.org/jira/browse/KAFKA-10481

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-09 Thread Bruno Cadonna
-- but it might still be useful to provide a convenience method listing all of the current threads And of course you could still get the number of threads easily by invoking size() on the returned list (or ordered set?). On Tue, Sep 8, 2020 at 12:16 PM Bruno Cadonna wrote: Thank you again

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-08 Thread Bruno Cadonna
activeStreamThreads() or liveStreamThreads() ? On Mon, Sep 7, 2020 at 1:45 AM Bruno Cadonna wrote: Hi John, I agree with you except for checking null. I would rather prefer to use Optional as the return type to both methods. I changed the subject from [VOTE] to [DISCUSS] so that we can follow up in the disc

Re: [DISCUSS] Apache Kafka 2.7.0 release

2020-09-07 Thread Bruno Cadonna
Hi Bill, Could you add KIP-662 [1] to the release plan. The KIP has been already implemented. Best, Bruno [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-662%3A+Throw+Exception+when+Source+Topics+of+a+Streams+App+are+Deleted On 26.08.20 16:54, Bill Bejeck wrote: Greetings All!

[jira] [Resolved] (KAFKA-9924) Add RocksDB Memory Consumption to RocksDB Metrics

2020-09-07 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9924?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-9924. -- Resolution: Fixed > Add RocksDB Memory Consumption to RocksDB Metr

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

2020-09-07 Thread Bruno Cadonna
= false. If anybody will need it in future, we can open a new KIP for its addition. Best, Bruno [1] https://github.com/facebook/rocksdb/blob/ab202e8d72737ec3572e5f90c0a45af12effa4be/include/rocksdb/db.h#L910-L914 On 06.05.20 17:41, Bruno Cadonna wrote: Hi all, I'd like to discuss KIP-607

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-07 Thread Bruno Cadonna
at 7:41 AM Matthias J. Sax wrote: +1 (binding) On 9/3/20 6:16 AM, Bruno Cadonna wrote: Hi, I would like to start the voting on KIP-663 that proposes to add methods to the Kafka Streams client to add and remove stream threads during execution. https://cwiki.apache.org/confluence/display/KAFKA

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-07 Thread Bruno Cadonna
it would be nice to add a small utility method that does this for them. Something like // Returns the number of currently alive threads boolean runningStreamThreads(); On Thu, Sep 3, 2020 at 7:41 AM Matthias J. Sax wrote: +1 (binding) On 9/3/20 6:16 AM, Bruno Cadonna wrote: Hi, I would like

[VOTE] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-03 Thread Bruno Cadonna
Hi, I would like to start the voting on KIP-663 that proposes to add methods to the Kafka Streams client to add and remove stream threads during execution. https://cwiki.apache.org/confluence/display/KAFKA/KIP-663%3A+API+to+Start+and+Shut+Down+Stream+Threads Best, Bruno

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-03 Thread Bruno Cadonna
Carlson wrote: Hi Bruno, I read through your updated KIP and it looks good to me. I agree with adding the metric to keep track of crashed streams in replace of a list of dead streams. best, Wlaker :) On Tue, Sep 1, 2020 at 1:05 PM Bruno Cadonna wrote: Hi John, your proposal makes sense! I

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-01 Thread Bruno Cadonna
u think of "crashed- stream-threads" instead? Thanks, -John On Tue, 2020-09-01 at 11:30 +0200, Bruno Cadonna wrote: Hi, I updated the KIP with the feedback so far. I removed the API to close the Kafka Streams client asynchronously, since it should be possible to avoid the deadlock with the

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-09-01 Thread Bruno Cadonna
://cwiki.apache.org/confluence/display/KAFKA/KIP-663%3A+API+to+Start+and+Shut+Down+Stream+Threads Best, Bruno On 26.08.20 16:31, Bruno Cadonna wrote: Hi, I would like to propose the following KIP to start and shut down stream threads during execution as well as to shut down asynchronously a Kafka

Re: [VOTE] KIP-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-31 Thread Bruno Cadonna
, 2020 at 4:00 AM Bruno Cadonna wrote: Hi, I would like to start the vote for https://cwiki.apache.org/confluence/display/KAFKA/KIP-662%3A+Throw+Exception+when+Source+Topics+of+a+Streams+App+are+Deleted Best, Bruno -- -- Guozhang

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-08-27 Thread Bruno Cadonna
It still might be a good point to mention in the KIP (and the docs). Thanks again! -John On Wed, 2020-08-26 at 16:31 +0200, Bruno Cadonna wrote: Hi, I would like to propose the following KIP to start and shut down stream threads during execution as well as to shut down asynchronously a Kafka S

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-08-27 Thread Bruno Cadonna
r remove threads. Maybe that's the best place to land right now. It still might be a good point to mention in the KIP (and the docs). Thanks again! -John On Wed, 2020-08-26 at 16:31 +0200, Bruno Cadonna wrote: Hi, I would like to propose the following KIP to start and shut down stream threads durin

Re: [DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-08-27 Thread Bruno Cadonna
s. Maybe that's the best place to land right now. It still might be a good point to mention in the KIP (and the docs). I agree that it is fine for now and I will add a few words in the KIP. Thanks again! -John On Wed, 2020-08-26 at 16:31 +0200, Bruno Cadonna wrote: Hi, I would like to prop

[DISCUSS] KIP-663: API to Start and Shut Down Stream Threads and to Request Closing of Kafka Streams Clients

2020-08-26 Thread Bruno Cadonna
Hi, I would like to propose the following KIP to start and shut down stream threads during execution as well as to shut down asynchronously a Kafka Streams client from an uncaught exception handler.

Re: [DISCUSS] KIP-647: Add ability to handle late messages in streams-aggregation

2020-08-26 Thread Bruno Cadonna
rently being developed. Please share your opinion and feedback. Regards, Igor. On 11.08.20 09:25, Bruno Cadonna wrote: Hi Igor, Thanks for the KIP! Similar to Matthias, I am also wondering why you rejected the more general solution involving a callback. I also think that writing to a topic i

[VOTE] KIP-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-25 Thread Bruno Cadonna
Hi, I would like to start the vote for https://cwiki.apache.org/confluence/display/KAFKA/KIP-662%3A+Throw+Exception+when+Source+Topics+of+a+Streams+App+are+Deleted Best, Bruno

Re: [DISCUSS] KIP-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-25 Thread Bruno Cadonna
ng Stream processing? Anyways, I'm looking forward to cleaning up the exception hierarchy so we get a clear division of user vs "internal" error, but within the current framework this SGTM On Fri, Aug 21, 2020 at 8:06 AM Bruno Cadonna wrote: Hi, I would like to propose the fol

[jira] [Resolved] (KAFKA-10293) fix flaky streams/streams_eos_test.py

2020-08-25 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-10293. --- Resolution: Fixed > fix flaky streams/streams_eos_test

[DISCUSS] KIP-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-21 Thread Bruno Cadonna
Hi, I would like to propose the following KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-662%3A+Throw+Exception+when+Source+Topics+of+a+Streams+App+are+Deleted Best, Bruno

Re: [DISCUSS] KIP-639 Move nodeLevelSensor and storeLevelSensor methods from StreamsMetricsImpl to StreamsMetrics

2020-08-20 Thread Bruno Cadonna
nts of Bruno Cardona. Le 06/07/2020 à 22:36, Mohamed Chebbi a écrit : Thank Bruno for your review. Changes was added as you sugested. Le 06/07/2020 à 14:57, Bruno Cadonna a écrit : Hi Mohamed, Thank you for the KIP. Comments regarding the KIP wiki: 1. In section "Public Interface",

Re: [VOTE] KIP-657: Add Customized Kafka Streams Logo

2020-08-18 Thread Bruno Cadonna
Thanks for the KIP! I am +1 (non-binding) for A. I would also like to hear opinions whether the logo should be colorized or just black and white. Best, Bruno On 15.08.20 16:05, Adam Bellemare wrote: I prefer Design B, but given that I missed the discussion thread, I think it would be

[jira] [Created] (KAFKA-10397) Do not Expose Statistics-based RocksDB Metrics If User Provides Statistics Object

2020-08-13 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10397: - Summary: Do not Expose Statistics-based RocksDB Metrics If User Provides Statistics Object Key: KAFKA-10397 URL: https://issues.apache.org/jira/browse/KAFKA-10397

Re: [DISCUSS] KIP-647: Add ability to handle late messages in streams-aggregation

2020-08-11 Thread Bruno Cadonna
Hi Igor, Thanks for the KIP! Similar to Matthias, I am also wondering why you rejected the more general solution involving a callback. I also think that writing to a topic is just one of multiple ways to handle late records. For example, one could compute statistics over the late records

Re: [ANNOUNCE] New Kafka PMC Member: John Roesler

2020-08-11 Thread Bruno Cadonna
Wow, that is awesome! Congrats, John! Bruno On 10.08.20 22:11, Jun Rao wrote: Hi, Everyone, John Roesler has been a Kafka committer since Nov. 5, 2019. He has remained active in the community since becoming a committer. It's my pleasure to announce that John is now a member of Kafka PMC.

Re: [VOTE] KIP-648: Renaming getter method for Interactive Queries

2020-07-29 Thread Bruno Cadonna
Thanks John, +1 (non-binding) Best, Bruno On 29.07.20 01:02, John Thomas wrote: Hello everyone, I'd like to kick-off a vote for KIP-648 : Renaming getter method for Interactive Queries

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

2020-07-28 Thread Bruno Cadonna
07:04, Bruno Cadonna wrote: Hi, After re-opening the discussion about https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+to+Kafka+Streams+to+Report+Properties+of+RocksDB I would like to re-open the voting for this KIP. The discussion thread can be found here:

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

2020-07-27 Thread Bruno Cadonna
compressed dictionary blocks are read into OS cache by default. Obviously that conflicts from yours, maybe you can point me to the related docs? Guozhang On Fri, Jul 24, 2020 at 2:15 AM Bruno Cadonna wrote: Hi Guozhang and Sophie, 1) My understanding is also that the memtables are off-heap (as almost

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

2020-07-24 Thread Bruno Cadonna
/mod_mbox/kafka-dev/202005.mbox/%3CCADR0NwzJBJa6WihnpmGj0R%2BYPVrojq4Kg_hOArNEytHAG-tZAQ%40mail.gmail.com%3E Best, Bruno On 19.05.20 10:00, Bruno Cadonna wrote: Thank you for voting! This KIP passes with: 4 binding +1 1 non-binding +1 0 -1 Best, Bruno On Fri, May 15, 2020 at 11:34 PM Matthias J. Sax

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

2020-07-24 Thread Bruno Cadonna
sage" + "estimate-table-readers-mem". Is that right? On Wed, Jul 22, 2020 at 4:28 AM Bruno Cadonna wrote: Hi Guozhang, Thank you for your feedback! I answered inline. Best, Bruno On 21.07.20 00:39, Guozhang Wang wrote: Hello Bruno, Thanks for the updated KIP. I made a pass and

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

2020-07-22 Thread Bruno Cadonna
for compaction buffers? 5) This is just of a nit comment to help readers better understand rocksDB: maybe we can explain in the wiki doc which part of rocksDB uses memory (block cache, OS cache, memtable, compaction buffer, read buffer), and which of them are on-heap and wich of them are off-heap, which

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

2020-07-22 Thread Bruno Cadonna
he, OS cache, memtable, compaction buffer, read buffer), and which of them are on-heap and wich of them are off-heap, which can be hard bounded and which can only be soft bounded and which cannot be bounded at all, etc. Good idea! Will look into it! Guozhang On Mon, Jul 20, 2020 at 11:00 A

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

2020-07-22 Thread Bruno Cadonna
he, OS cache, memtable, compaction buffer, read buffer), and which of them are on-heap and wich of them are off-heap, which can be hard bounded and which can only be soft bounded and which cannot be bounded at all, etc. Good idea! Will look into it! Guozhang On Mon, Jul 20, 2020 at 11:00 A

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

2020-07-20 Thread Bruno Cadonna
-Goldman wrote: Thanks Bruno! I took a look at the revised KIP and it looks good to me. Sophie On Wed, May 13, 2020 at 6:59 AM Bruno Cadonna wrote: Hi John, Thank you for the feedback! I agree and I will change the KIP as I stated in my previous e-mail to Guozhang. Best, Bruno On Tue

Re: [DISCUSS] KIP-450: Sliding Windows

2020-07-14 Thread Bruno Cadonna
Hi Leah, Thank you for the KIP! Here is my feedback: 1. The KIP would benefit from some code examples that show how to use sliding windows in aggregations. 2. The different sliding windows in Figure 1 and 2 are really hard to distinguish. Could you please try to make them graphically better

Re: [VOTE] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-07-08 Thread Bruno Cadonna
Thanks Will and Piotr, +1 (non-binding) Best, Bruno On Wed, Jul 8, 2020 at 8:12 AM Matthias J. Sax wrote: > > Thanks for the KIP. > > +1 (binding) > > > -Matthias > > On 7/7/20 11:48 AM, William Bottrell wrote: > > Hi everyone, > > > > I'd like to start a vote for adding two new time API's to

Re: [DISCUSS] KIP-639 Move nodeLevelSensor and storeLevelSensor methods from StreamsMetricsImpl to StreamsMetrics

2020-07-06 Thread Bruno Cadonna
Hi Mohamed, Thank you for the KIP. Comments regarding the KIP wiki: 1. In section "Public Interface", you should state what you want to change in interface StreamsMetrics. In your case, you want to add two methods. You can find a good example how to describe this in KIP-444

[jira] [Created] (KAFKA-10221) Backport fix for KAFKA-9603 to 2.5

2020-06-30 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10221: - Summary: Backport fix for KAFKA-9603 to 2.5 Key: KAFKA-10221 URL: https://issues.apache.org/jira/browse/KAFKA-10221 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-30 Thread Bruno Cadonna
Hi, I have already brought this up in the discussion thread. Should we not run a dry-run in any case to avoid inadvertently deleting topics of other applications? I know it is a backward incompatible change if users use it in scripts, but I think it is still worth discussing it. I would to hear

Re: [VOTE] KIP-629: Use racially neutral terms in our codebase

2020-06-26 Thread Bruno Cadonna
+1 (non-binding) On Fri, Jun 26, 2020 at 3:41 AM Jay Kreps wrote: > > +1 > > On Thu, Jun 25, 2020 at 6:39 PM Bill Bejeck wrote: > > > Thanks for this KIP Xavier. > > > > +1(binding) > > > > -Bill > > > > On Thu, Jun 25, 2020 at 9:04 PM Gwen Shapira wrote: > > > > > +1 (binding) > > > > > >

Re: [DISCUSS] KIP-629: Use racially neutral terms in our codebase

2020-06-23 Thread Bruno Cadonna
Hi Xavier, Thank you very much for starting this initiative! Not only for the changes to the code base but also for showing me where and how we can use more appropriate terms in general. Best, Bruno On Tue, Jun 23, 2020 at 4:17 AM John Roesler wrote: > > Hi Xavier, > > I think your approach

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Bruno Cadonna
Congrats, Boyang! Best, Bruno On Tue, Jun 23, 2020 at 7:50 AM Konstantine Karantasis wrote: > > Congrats, Boyang! > > -Konstantine > > On Mon, Jun 22, 2020 at 9:19 PM Navinder Brar > wrote: > > > Many Congratulations Boyang. Very well deserved. > > > > Regards,Navinder > > > > On Tuesday,

[jira] [Resolved] (KAFKA-10148) Kafka Streams Restores too few Records with eos-beta Enabled

2020-06-19 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10148?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-10148. --- Resolution: Fixed > Kafka Streams Restores too few Records with eos-beta Enab

Re: First time patch submitter advice

2020-06-19 Thread Bruno Cadonna
I meant "Hi Michael" not Luke. Sorry Michael and Luke. Best, Bruno On Fri, Jun 19, 2020 at 10:47 AM Bruno Cadonna wrote: > > Hi Luke, > > The guide is a bit outdated. Thank you for pointing it out. I updated the > guide. > > As Gwen stated above: >

Re: First time patch submitter advice

2020-06-19 Thread Bruno Cadonna
Hi Luke, The guide is a bit outdated. Thank you for pointing it out. I updated the guide. As Gwen stated above: > Unfortunately, you need to get a committer to approve running the tests. So, yes a committer has to comment on the PR. Best, Bruno On Fri, Jun 19, 2020 at 1:28 AM Michael Carter

[jira] [Created] (KAFKA-10179) State Store Passes Wrong Changelog Topic to Serde for Optimized Source Tables

2020-06-17 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10179: - Summary: State Store Passes Wrong Changelog Topic to Serde for Optimized Source Tables Key: KAFKA-10179 URL: https://issues.apache.org/jira/browse/KAFKA-10179

Re: [VOTE] KIP-626: Rename StreamsConfig config variable name

2020-06-16 Thread Bruno Cadonna
Hi Matthias, Thank you for the KIP. I am +1 (non-binding). Said that, you have a typo in the "Public Interface" section. TOPLOGY_OPTIMIZATION_CONFIG instead of TOPOLOGY_OPTIMIZATION_CONFIG. Best, Bruno On Tue, Jun 16, 2020 at 5:02 AM Matthias J. Sax wrote: > > Hi, > > I found a small

Re: [DISCUSS] KIP-623: Add "internal-topics" option to streams application reset tool

2020-06-12 Thread Bruno Cadonna
Hi Joel, Thank you for the KIP. The KIP is well motivated. I have a couple comments: 1. I would not describe the new option with Java code that you want to add to the `StreamsResetter` class since this class is not part of the public API. Only the script kafka-streams-application-reset.sh in

Re: [DISCUSS] KIP-622 Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext

2020-06-11 Thread Bruno Cadonna
Hi Will, Thank you for the KIP. 1. Could you elaborate a bit more on the motivation in the KIP? An example would make the motivation clearer. 2. In section "Proposed Changes" you do not need to show the implementation and describe internals. A description of the expected behavior of the newly

[jira] [Created] (KAFKA-10148) Kafka Streams Restores too few Records with eos-beta Enabled

2020-06-11 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10148: - Summary: Kafka Streams Restores too few Records with eos-beta Enabled Key: KAFKA-10148 URL: https://issues.apache.org/jira/browse/KAFKA-10148 Project: Kafka

[jira] [Created] (KAFKA-10137) Clean-up retain Duplicate logic in Window Stores

2020-06-10 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10137: - Summary: Clean-up retain Duplicate logic in Window Stores Key: KAFKA-10137 URL: https://issues.apache.org/jira/browse/KAFKA-10137 Project: Kafka Issue

[jira] [Created] (KAFKA-10119) StreamsResetter fails with TimeoutException for older Brokers

2020-06-08 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10119: - Summary: StreamsResetter fails with TimeoutException for older Brokers Key: KAFKA-10119 URL: https://issues.apache.org/jira/browse/KAFKA-10119 Project: Kafka

[jira] [Resolved] (KAFKA-9173) StreamsPartitionAssignor assigns partitions to only one worker

2020-05-22 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9173?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-9173. -- Fix Version/s: 2.6.0 Resolution: Fixed > StreamsPartitionAssignor assigns partiti

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

2020-05-19 Thread Bruno Cadonna
;>> Thanks for the KIP! > >>> > >>> +1 (binding) > >>> > >>> -Bill > >>> > >>> On Fri, May 15, 2020 at 11:12 AM Guozhang Wang wrote: > >>> > >>>> +1. > >>>> > >>&g

[jira] [Created] (KAFKA-10015) React to Unexpected Errors on Stream Threads

2020-05-18 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-10015: - Summary: React to Unexpected Errors on Stream Threads Key: KAFKA-10015 URL: https://issues.apache.org/jira/browse/KAFKA-10015 Project: Kafka Issue Type

Re: [DISCUSS] KIP-613: Add end-to-end latency metrics to Streams

2020-05-18 Thread Bruno Cadonna
> > >> Regarding your second. point, I'm inclined to leave > > that > > > > as > > > > > an > > > > > > > > > > > >> implementation detail > > > > > > > > > > > >> but my take would be that the user should b

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

2020-05-18 Thread Bruno Cadonna
Hi Matthias, I am +1 (non-binding) on the KIP. Just one final remark: Wouldn't it be better to specify task.timeout.ms to -1 if no retry should be done? IMO it would make the config more intuitive because 0 would not have two possible meanings (i.e. try once and never try) anymore. Best, Bruno

[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

[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 balan

[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 st

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

2020-05-14 Thread Bruno Cadonna
Hi Matthias, Thank you for the KIP. I like your KIP. Here my feedback: 1. The KIP is not clear about what should happen when task.timeout.ms expires. To facilitate the mapping from the error users might encounter due to timeouts to this KIP, it would be good to state the error that will be

Re: [DISCUSS] KIP-613: Add end-to-end latency metrics to Streams

2020-05-14 Thread Bruno Cadonna
Hi Sophie, Thank you for the KIP. The KIP looks good to me. 50th percentile: I think we do not need it now. If we need it, we can add it. Here the old truism applies: Adding is always easier than removing. processor-node-level metrics: I think it is good to have the staleness metrics also on

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

2020-05-13 Thread Bruno Cadonna
g that some users may be using the bounded > > > memory > > > config setter > > > to put a cap on the off-heap memory of the entire process, in which case > > > the memory usage > > > metric for any one store should reflect the memory usage of the

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

2020-05-13 Thread Bruno Cadonna
re process, in which case > > the memory usage > > metric for any one store should reflect the memory usage of the entire > > instance. In that case > > any effort to roll up the memory usages ourselves would just be wasted. > > > > Sorry for the reversal, but after

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

2020-05-13 Thread Bruno Cadonna
y in > favor of reporting these > at the store level. > > Best, > Sophie > > On Wed, May 6, 2020 at 8:41 AM Bruno Cadonna wrote: > > > Hi all, > > > > I'd like to discuss KIP-607 that aims to add RocksDB memory usage > > metrics to Kafka Streams.

[jira] [Resolved] (KAFKA-5948) EosIntegrationTest fails with TopicAlreadyMarkedForDeletionException

2020-05-12 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-5948?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-5948. -- Resolution: Cannot Reproduce > EosIntegrationTest fa

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-05-11 Thread Bruno Cadonna
vance if/how we can > get the information about inner serdes. I am not sure atm if we would > need more API changes to get this info. The other (minor) question is > also, how this information would be presented to the use (as we only use > `String` types for Serde information. > > > &

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

2020-05-06 Thread Bruno Cadonna
Hi all, I'd like to discuss KIP-607 that aims to add RocksDB memory usage metrics to Kafka Streams. https://cwiki.apache.org/confluence/display/KAFKA/KIP-607%3A+Add+Metrics+to+Record+the+Memory+Used+by+RocksDB+to+Kafka+Streams Best, Bruno

Re: [DISCUSS] KIP-598: Augment TopologyDescription with store and source / sink serde information

2020-05-04 Thread Bruno Cadonna
Hi Guozhang, Thank you for the KIP! Exposing also the inner types of the wrapper serdes would be important. For debugging as Matthias has already mentioned and to see more easily changes that are applied to a topology. I am also +1 on the `toJson()` method to easily access the topology

[jira] [Resolved] (KAFKA-9936) kafka-streams 2.5.0 missing dependency?

2020-04-30 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9936?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-9936. -- Resolution: Not A Bug > kafka-streams 2.5.0 missing depende

[jira] [Created] (KAFKA-9924) Add RocksDB Memory Consumption to RocksDB Metrics

2020-04-28 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9924: Summary: Add RocksDB Memory Consumption to RocksDB Metrics Key: KAFKA-9924 URL: https://issues.apache.org/jira/browse/KAFKA-9924 Project: Kafka Issue Type

[jira] [Created] (KAFKA-9916) Materialize Table-Table Join Result to Avoid Performing Same Join Twice

2020-04-24 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9916: Summary: Materialize Table-Table Join Result to Avoid Performing Same Join Twice Key: KAFKA-9916 URL: https://issues.apache.org/jira/browse/KAFKA-9916 Project: Kafka

Re: Rocksdb Statistics

2020-04-22 Thread Bruno Cadonna
Hi Nagendra, What you describe is a known issue: https://issues.apache.org/jira/browse/KAFKA-9675 And you also described the appropriate fix: https://github.com/apache/kafka/pull/8256 The issue is fixed and will be included in 2.6.0, 2.4.2, 2.5.1. I am sorry for any inconveniences. Best,

[jira] [Created] (KAFKA-9881) Verify whether RocksDBMetrics Get Measurements from RocksDB in a Unit Test instead of a Intergration Test

2020-04-16 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9881: Summary: Verify whether RocksDBMetrics Get Measurements from RocksDB in a Unit Test instead of a Intergration Test Key: KAFKA-9881 URL: https://issues.apache.org/jira/browse

[jira] [Resolved] (KAFKA-9117) Add active-process-ratio Metric

2020-04-16 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-9117. -- Fix Version/s: (was: 2.6.0) Resolution: Duplicate > Add active-process-ratio Met

[jira] [Resolved] (KAFKA-9664) Flaky Test KafkaStreamsTest#testStateThreadClose

2020-04-15 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9664?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna resolved KAFKA-9664. -- Resolution: Cannot Reproduce > Flaky Test KafkaStreamsTest#testStateThreadCl

Re: [VOTE] KIP-444: Refactor and Augment Metrics for Kafka Streams

2020-04-07 Thread Bruno Cadonna
, Matthias voted on the DISCUSS thread). > >> > non-binding +1: 2 (Bruno, John). > >> > > >> > > >> > Guozhang > >> > > >> > > >> > On Thu, Aug 22, 2019 at 8:16 AM Bill Bejeck wrote: > >> > > >

[jira] [Reopened] (KAFKA-9770) Caching State Store does not Close Underlying State Store When Exception is Thrown During Flushing

2020-03-30 Thread Bruno Cadonna (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9770?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bruno Cadonna reopened KAFKA-9770: -- > Caching State Store does not Close Underlying State Store When Exception is > Thrown

[jira] [Created] (KAFKA-9770) Caching State Store does not Close Underlying State Store When Exception is Thrown During Flushing

2020-03-26 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9770: Summary: Caching State Store does not Close Underlying State Store When Exception is Thrown During Flushing Key: KAFKA-9770 URL: https://issues.apache.org/jira/browse/KAFKA-9770

[jira] [Created] (KAFKA-9738) Add Generics Type Parameters to forwarded() in MockProcessorContext

2020-03-20 Thread Bruno Cadonna (Jira)
Bruno Cadonna created KAFKA-9738: Summary: Add Generics Type Parameters to forwarded() in MockProcessorContext Key: KAFKA-9738 URL: https://issues.apache.org/jira/browse/KAFKA-9738 Project: Kafka

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-03-09 Thread Bruno Cadonna
on't > think it qualifies as a blocker for 2.5.0. > > Ismael > > On Mon, Mar 9, 2020 at 8:44 PM Bruno Cadonna wrote: > > > Hi David, > > > > A bug report was filed that can be considered a blocker. Basically, > > with this bug all RocksDB metrics reported by Streams

Re: [DISCUSS] Apache Kafka 2.5.0 release

2020-03-09 Thread Bruno Cadonna
Hi David, A bug report was filed that can be considered a blocker. Basically, with this bug all RocksDB metrics reported by Streams are constant zero. The bug is there since 2.4., so it is not a regression, but a broken feature. Here is the ticket:

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

2020-03-04 Thread Bruno Cadonna
gt; > One other thought I had is maybe you were thinking that operators > > > would update their internally tracked stream time, but still discard > > > records? I think that _would_ be a bug. That is, if a record gets > > discarded > > > as idempotent, it should have n

  1   2   3   >