[DISCUSS] KIP-313: Add KStream.flatTransform and KStream.flatTransformValues

2018-06-08 Thread Bruno Cadonna
Hi list, I created KIP-313 [1] for JIRA issue KAFKA-4217 [2] and I would like to put the KIP up for discussion. Best regards, Bruno [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-313%3A+Add+KStream.flatTransform+and+KStream.flatTransformValues [2] https://issues.apache.org/jira/brow

[VOTE] KIP-313: Add KStream.flatTransform and KStream.flatTransformValues

2018-06-22 Thread Bruno Cadonna
Hi list, I would like to voting on this KIP. I created a first PR[1] that adds flatTransform. Once I get some feedback, I will start work on flatTransformValues. Best regards, Bruno [1] https://github.com/apache/kafka/pull/5273

Re: [VOTE] KIP-313: Add KStream.flatTransform and KStream.flatTransformValues

2018-07-16 Thread Bruno Cadonna
jeck wrote: >>>> Thanks for the KIP, +1. >>>> >>>> -Bill >>>> >>>> On Fri, Jun 22, 2018 at 1:08 PM Ted Yu wrote: >>>> >>>>> +1 >>>>> >>>>> On Fri, Jun 22, 2018 at 2:50 AM, Bruno Cadonn

Re: [DISCUSS] KIP-448: Add State Stores Unit Test Support to Kafka Streams Test Utils

2019-04-09 Thread Bruno Cadonna
fluence/x/SAeZBg). It is about adding > > >> Mock state stores and relevant components for testing purposes. > > >> > > >> Here is the JIRA: https://issues.apache.org/jira/browse/KAFKA-6460 > > >> > > >> This is a rough KIP draft, review and comment are appreciated. It > > >> seems to be tricky and some requirements and details are still needed > > >> to be discussed. > > >> > > >> Thanks, > > >> Yishun > > >> > > > > > > -- Bruno Cadonna Software Engineer at Confluent

Re: Request for contribution

2019-04-10 Thread Bruno Cadonna
Hi Manish, Good to hear that you want to learn and contribute to Kafka. The documentation and the project info site are great starting points https://kafka.apache.org/project https://kafka.apache.org/documentation/ To start contributing take a look at https://kafka.apache.org/contributing and

Re: [DISCUSS] KIP-446: Add changelog topic configuration to KTable suppress

2019-04-10 Thread Bruno Cadonna
Hi Marteen and John, I would opt for option 1 with an additional log message on INFO or WARN level, since the log file is the place where you would look first to understand what went wrong. I would also not adjust it when persistence stores are available for suppress. I would not go for option 2

Re: [DISCUSS] KIP-446: Add changelog topic configuration to KTable suppress

2019-04-12 Thread Bruno Cadonna
27;t see why we should treat > suppress() different to other stores. > > > -Matthias > > On 4/10/19 3:36 PM, Bruno Cadonna wrote: > > Hi Marteen and John, > > > > I would opt for option 1 with an additional log message on INFO or WARN > > level, since the log fi

Re: [VOTE] KIP-446: Add changelog topic configuration to KTable suppress

2019-04-17 Thread Bruno Cadonna
Hi Maarten Duijn, Thank you for driving this. +1 (non-binding) Best, Bruno On Wed, Apr 17, 2019 at 8:21 AM Maarten Duijn wrote: > Hello all, > > There has been informal agreement so I would like to call for a vote on > KIP-446: Add changelog topic configuration to KTable suppress. This will >

Re: [DISCUSS] KIP-448: Add State Stores Unit Test Support to Kafka Streams Test Utils

2019-04-30 Thread Bruno Cadonna
c, developers should not > > > use them, because they don't have any backward compatibility > guarantees. > > > > > > Hence, I would suggest that the corresponding classes go into a new > > > package `org.apache.kafka.streams.state`. > > > > > > > &g

Re: Fw: [DISCUSS] KIP-462 : Use local thread id for KStreams

2019-04-30 Thread Bruno Cadonna
Hi Guozhang, What do you mean exactly with "throttling purposes"? @Boyang: Thank you for the KIP! Best, Bruno On Tue, Apr 30, 2019 at 1:15 AM Guozhang Wang wrote: > Hi Boyang, > > Thanks for the KIP. I think it makes sense. > > Just following up on the documentation part: since we are effecti

Re: [DISCUSS] KIP-401 TransformerSupplier/ProcessorSupplier enhancements

2019-04-30 Thread Bruno Cadonna
Hi, @Paul: Thank you for the KIP! I hope you do not mind that I jump in. I have the following comments: 1) `null` vs empty list in the default implementation IIUC, returning `null` in the default implementation should basically signal that the method `stateStores` was not overridden. Why then p

Re: [DISCUSS] KIP-401 TransformerSupplier/ProcessorSupplier enhancements

2019-05-02 Thread Bruno Cadonna
ider and add and connect > stores appropriately. This isn't in the KIP and I think the value-add is > lower (if you're reaching that low level, surely the "auto add/connect > store" isn't too important to you), but I think it would be a confusing if > it didn&

[DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-05-17 Thread Bruno Cadonna
Hi all, this KIP describes the extension of the Kafka Streams' metrics to include RocksDB's internal statistics. Please have a look at it and let me know what you think. Since I am not a RocksDB expert, I am thankful for any additional pair of eyes that evaluates this KIP. https://cwiki.apache.o

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-05-20 Thread Bruno Cadonna
which options could/should they set in > > RocksDBConfigSetter should they discover a particular bottleneck? > > > > I don't think this necessarily needs to go into the KIP, but I do think it > > should be included in the docs somewhere (happy to help build up the list >

Re: [DISCUSS] KIP-439: Deprecate Interface WindowStoreIterator

2019-05-28 Thread Bruno Cadonna
Hi all, My comments on this KIP: 1. I would use `all()` instead of `range()` because the functionality is immediately clear without the need to look at the parameter list. 2. I would decouple method names from metrics name, because this allows us to change one naming independently from the other

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-05-28 Thread Bruno Cadonna
27;s a minor thought. Modulo the others' comments, this looks good > >> > to me. > >> > > >> > Thanks, > >> > -John > >> > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck wrote: > >> > > > >> > > Hi Bruno

Re: [DISCUSS] KIP-439: Deprecate Interface WindowStoreIterator

2019-05-28 Thread Bruno Cadonna
; > > > 3) the `state-` part is already contained in `[storeType]` do I think > it's correct as-is > > > 4) Ack. Fixed. > > > 5) I think `stream` (plural) is correct. Cf > https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-04 Thread Bruno Cadonna
o allow > users turn it on / off during run-time. > > > Guozhang > > > > On Tue, May 28, 2019 at 6:23 AM Bruno Cadonna wrote: > > > Hi, > > > > Thank you for your comments. > > > > @Bill: > > > > 1. It is like Guozhang wrote: >

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-06 Thread Bruno Cadonna
your point about accessing stores uniformly? > > While I agree there will definitely be variance in the access pattern of > > different segments, I think it's unlikely that it will vary in any kind of > > predictable or deterministic way, hence it is not that useful to know

Re: Contributor permissions

2019-06-07 Thread Bruno Cadonna
Hi Carlos, It's great that you want to contribute to Apache Kafka. Have you already read the instructions on how to contribute to Kafka? https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Changes https://kafka.apache.org/contributing.html To assign Jira tickets to yourself, you

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-14 Thread Bruno Cadonna
run into issues because they > >> were using all the other memory. > >> And with multiple streams applications on the same machine, each with > >> several KTables and 10+ partitions per topic the number of stores can get > >> out of hand pretty easily. > &g

[VOTE] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-18 Thread Bruno Cadonna
Hi, I would like to start the voting on KIP-471: https://cwiki.apache.org/confluence/display/KAFKA/KIP-471%3A+Expose+RocksDB+Metrics+in+Kafka+Streams You can find the discussion here: https://lists.apache.org/thread.html/125bdd984fe0667962018da6ce10bce6d5895c5103955a8e4c730fef@%3Cdev.kafka.apache

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-19 Thread Bruno Cadonna
k mentioned, is there a good way we can expose the total amount > of memory and disk usage for each state store as well? I think it would > also be very helpful for users to understand their capacity needs and read > / write amplifications. > > > Guozhang > > On Fri, Jun

Re: [DISCUSS] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-19 Thread Bruno Cadonna
metrics APIs. We should take care to make sure that what > > > we add in KIP-471 is stable and well supported, even if it's not the > > > complete picture. We can always do follow-on work to tackle complex > > > metrics as an isolated design exercise. > > >

Re: [VOTE] KIP-471: Expose RocksDB Metrics in Kafka Streams

2019-06-21 Thread Bruno Cadonna
>> Best regards > >> Patrik > >> > >>> Am 19.06.2019 um 21:55 schrieb Bill Bejeck : > >>> > >>> +1 (binding) > >>> > >>> Thanks, > >>> Bill > >>> > >>>> On Wed, Jun 19, 2019 at 1:

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-07-01 Thread Bruno Cadonna
Hi Guozhang, Thank you for the KIP. 1) As far as I understand, the StreamsMetrics interface is there for user-defined processors. Would it make sense to also add a method to the interface to specify a sensor that records skipped records? 2) What are the semantics of active-task-process and stand

Re: Running Kafka from source in Windows

2019-07-02 Thread Bruno Cadonna
Hi Manish, two bug reports have been already filed for this issue. https://issues.apache.org/jira/browse/KAFKA-6988 https://issues.apache.org/jira/browse/KAFKA-7998 You can find a description of the root cause and work-arounds in the bug reports and the following GitHub issue: https://github.co

Re: Running Kafka from source in Windows

2019-07-02 Thread Bruno Cadonna
it? > Which one is a better choice? > > On Tue, Jul 2, 2019 at 1:19 PM Bruno Cadonna wrote: > > > Hi Manish, > > > > two bug reports have been already filed for this issue. > > > > https://issues.apache.org/jira/browse/KAFKA-6988 > > https://iss

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-07-03 Thread Bruno Cadonna
Mon, Jul 1, 2019 at 7:08 AM Bruno Cadonna wrote: > > > Hi Guozhang, > > > > Thank you for the KIP. > > > > 1) As far as I understand, the StreamsMetrics interface is there for > > user-defined processors. Would it make sense to also add a method to > >

Re: [DISCUSS] KIP-488: Clean up Sum,Count,Total Metrics

2019-07-11 Thread Bruno Cadonna
Hi John, Thank you for the KIP. LGTM I also do not like CumulativeSum/Count so much. I propose to just call it Sum and Count. I understand that you want to unequivocally distinguish the two metric functions by their names, but I have the feeling the names become artificially complex. The exact

Re: [DISCUSS] KIP-488: Clean up Sum,Count,Total Metrics

2019-07-12 Thread Bruno Cadonna
m` and use `TotalSum`, we should also not use > `Count` but `TotalCount` for consistency. > > > -Matthias > > > > On 7/11/19 12:58 PM, Bruno Cadonna wrote: > > Hi John, > > > > Thank you for the KIP. > > > > LGTM > > > > I also do n

Re: [VOTE] KIP-488: Clean up Sum,Count,Total Metrics

2019-07-12 Thread Bruno Cadonna
Hi John, +1 (non-binding) Best, Bruno On Fri, Jul 12, 2019 at 8:45 PM John Roesler wrote: > > Hi Kafka devs, > > Yesterday, I proposed KIP-488 as a minor cleanup of some of our metric > implementations. > > KIP-488: https://cwiki.apache.org/confluence/x/kkAyBw > > The change seems pretty uncont

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-08-06 Thread Bruno Cadonna
Hi Guozhang, I left my comments inline. On Thu, Jul 18, 2019 at 8:28 PM Guozhang Wang wrote: > > Hello Bruno, > > Thanks for the feedbacks, replied inline. > > On Mon, Jul 1, 2019 at 7:08 AM Bruno Cadonna wrote: > > > Hi Guozhang, > > > > Thank

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-08-08 Thread Bruno Cadonna
Hi, Thank you for the KIP! Some questions/comments: 1. I am wondering if the "stand-by" tasks that catch up state before the active task is switched deserve its own name in this KIP and maybe in the code. We have already stated that they are not true stand-by tasks, they are not configured throu

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-08-20 Thread Bruno Cadonna
the total invocation metric already. > > > Guozhang > > On Mon, Aug 19, 2019 at 4:11 PM Guozhang Wang wrote: > > > Hi Bruno, > > > > > > On Tue, Aug 6, 2019 at 1:51 AM Bruno Cadonna wrote: > > > >> Hi Guozhang, > >> > >

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

2019-08-22 Thread Bruno Cadonna
Hi Guozhang, +1 (non-binding) Thank you for driving this! Bruno On Tue, Aug 20, 2019 at 8:29 PM Guozhang Wang wrote: > > Hello folks, > > I'd like to start a voting thread the following KIP to improve the Kafka > Streams metrics mechanism to users. This includes 1) renaming changes in > the pub

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-09-04 Thread Bruno Cadonna
t;if we haven't > > > >> > >>> > processed standbys for X iterations, or Y milliseconds, do so > > > >> now." > > > >> > >>> > > > > >> > >>> > Actually

Re: [DISCUSS] KIP-441: Smooth Scaling Out for Kafka Streams

2019-09-04 Thread Bruno Cadonna
durability measures, and we should > just assume it works with no special considerations in the assignment > algorithm. > > 3) Good catch! I've fixed it. > > Thanks again! > -John > > On Wed, Sep 4, 2019 at 6:09 AM Bruno Cadonna wrote: > > > Hi, > &g

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-09-04 Thread Bruno Cadonna
n definitely add the total amount > > sensor as well -- they are just util functions to save users some lines of > > code anyways, and should be straightforward. > > > > Guozhang > > > > > > On Tue, Aug 20, 2019 at 1:05 AM Bruno Cadonna wrote: > > > &

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-09-05 Thread Bruno Cadonna
ethod address the > > issue > > > > (2) The metric name `put | put-if-absent .. | get-latency (avg | max)` > > is hard to read because is indicate that there is a `get-latency` method > > call on stores -- can we update it to > > > > `(put | put-if-absent

Re: Kafka

2019-09-05 Thread Bruno Cadonna
Hi Ghullam, Apache Kafka is open source. See license under https://github.com/apache/kafka/blob/trunk/LICENSE Best, Bruno On Thu, Sep 5, 2019 at 10:19 PM Ghullam Mohiyudin wrote: > > Hi , > I read the information about kafka. Now i want to create a degree final > project using kafka. Can you pl

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-09-05 Thread Bruno Cadonna
without any new values. > > > Guozhang > > > On Thu, Sep 5, 2019 at 12:16 PM Bruno Cadonna wrote: > > > Hi Guozhang, > > > > I think user experience and code maintenance are tightly related. The > > harder to maintain the code the worse the user experi

Re: [VOTE] KIP-441: Smooth Scaling Out for Kafka Streams

2019-09-06 Thread Bruno Cadonna
+1 (non-binding) On Fri, Sep 6, 2019 at 12:32 AM Guozhang Wang wrote: > > +1 (binding). > > On Thu, Sep 5, 2019 at 2:47 PM John Roesler wrote: > > > Hello, all, > > > > After a great discussion, I'd like to open voting on KIP-441, > > to avoid long restore times in Streams after rebalancing. > >

Re: [VOTE] KIP-520: Augment Consumer.committed(partition) to allow multiple partitions

2019-09-12 Thread Bruno Cadonna
Guozhang, Thanks for the KIP. +1 (non-binding) Best, Bruno On Wed, Sep 11, 2019 at 9:17 AM Kamal Chandraprakash wrote: > > Thanks for the KIP! > > LGTM, +1 (non-binding). > > On Wed, Sep 11, 2019 at 3:23 AM Matthias J. Sax > wrote: > > > I don't have a strong preference. So I am also fine to d

Re: PR Broken builds

2019-09-13 Thread Bruno Cadonna
Hi Daniyar, The jenkins logs were not accessible anymore. I restarted your builds. Let's see what the issues are. Best, Bruno On Thu, Sep 12, 2019 at 7:42 PM Development wrote: > > Hi, > > I’ve been working on KAFKA-8326 PR (https://github.com/apache/kafka/pull/6592 >

Re: PR Broken builds

2019-09-13 Thread Bruno Cadonna
On Fri, Sep 13, 2019 at 2:07 PM Bruno Cadonna wrote: > > Hi Daniyar, > > The jenkins logs were not accessible anymore. I restarted your builds. > Let's see what the issues are. > > Best, > Bruno > > On Thu, Sep 12, 2019 at 7:42 PM Development wrote: > > &

Re: About Testing Stream Applications Documentation

2019-09-18 Thread Bruno Cadonna
Hi Ugur, Your finding looks correct to me. Do you mind fixing this issue? Best, Bruno On Tue, Sep 17, 2019 at 12:54 PM uğur wrote: > > Hi, > > I am not sure if it is the right email address to write about this topic, > please correct me if I am wrong. > > As I read documentation of Testing Stre

Re: [VOTE] KIP-528: Deprecate PartitionGrouper configuration and interface

2019-09-20 Thread Bruno Cadonna
Hi Matthias, +1 (non-binding) Best, Bruno On Fri, Sep 20, 2019 at 10:02 AM Matthias J. Sax wrote: > > Sure -- don't think we need to cover it in the KIP though. > > Btw: an implicit deprecation is implied by this KIP for the currently > public default implementation of the `PartitionGrouper` in

Re: [DISCUSS] KIP-527: Add NothingSerde to Serdes

2019-09-30 Thread Bruno Cadonna
Hi Nikolay, Thank you for the KIP. I have a couple of minor comments: 1. I would not put implementation details into the KIP as you did with the bodies of the constructor of the `VoidSerde` and the `serialize` and `deserialize` methods. IMO, the signatures suffice. The implementation is then dis

Re: [VOTE] KIP-527: Add VoidSerde to Serdes

2019-10-01 Thread Bruno Cadonna
Hi Nikolay, Thank you for the KIP! +1 (non-binding) Best, Bruno On Tue, Oct 1, 2019 at 5:57 PM Nikolay Izhikov wrote: > > Hello. > > I would like to start vote for KIP-527: Add VoidSerde to Serdes > > KIP - > https://cwiki.apache.org/confluence/display/KAFKA/KIP-527%3A+Add+VoidSerde+to+Serdes

Re: [DISCUSS] KIP-530: Consider renaming 'UsePreviousTimeOnInvalidTimeStamp' class to 'UsePartitionTimeOnInvalidTimeStamp'

2019-10-07 Thread Bruno Cadonna
Hi Rabi, Thank you for the KIP! 1.) Could you please improve the formatting of the KIP? For instance, use appropriate formatting for code to differentiate it from the text. Also, we usually do not use italics to write KIPs. Look at other KIPs to get an idea of the formatting. 2.) "Public Interfa

Re: [DISCUSS] KIP-530: Considerrenaming'UsePreviousTimeOnInvalidTimeStamp' classto'UsePartitionTimeOnInvalidTimeStamp'

2019-10-07 Thread Bruno Cadonna
cordMetadataTimestamp. I > have only seen extract() method in ExtractRecordMetadataTimestamp. Please do > correct me if I am wrong. > > And yes I do agree with you on 5) the deprecation part for compatibility, > deprecation and migration plan > > > With Best Reg

Re: [DISCUSS] Apache Kafka 2.4.0 release

2019-10-08 Thread Bruno Cadonna
Hi Manikumar, It is technically true that KIP-471 is not completed, but the only aspect that is not there are merely two metrics that I could not add due to the RocksDB version currently used in Streams. Adding those two metrics once the RocksDB version will have been increased, will be a minor ef

Re: [VOTE] KIP-530: Consider renaming 'UsePreviousTimeOnInvalidTimeStamp' class to 'UsePartitionTimeOnInvalidTimeStamp'

2019-10-24 Thread Bruno Cadonna
Hi Rabi, Thank you for the KIP! +1 (non-binding) Best, Bruno On Thu, Oct 24, 2019 at 8:42 AM Matthias J. Sax wrote: > > +1 (binding) > > Thanks for the KIP! > > -Matthias > > On 10/23/19 9:37 AM, RABI K.C. wrote: > > Hello All, > > > > As per the discussion I want to call for a vote to go forw

Re: [ANNOUNCE] New committer: John Roesler

2019-11-13 Thread Bruno Cadonna
Congrats, John! Best, Bruno On Tue, Nov 12, 2019 at 10:56 PM Guozhang Wang wrote: > > Hi Everyone, > > The PMC of Apache Kafka is pleased to announce a new Kafka committer, John > Roesler. > > John has been contributing to Apache Kafka since early 2018. His main > contributions are primarily aro

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 ad

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 /b

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 inconsis

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

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: > &g

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, 2

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 mad

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) > > > > > > Than

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: [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 (https://cwiki.apache

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 P

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 dis

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

2020-07-20 Thread Bruno Cadonna
AM Sophie Blee-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

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

2020-07-22 Thread Bruno Cadonna
DB 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 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, J

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

2020-07-22 Thread Bruno Cadonna
DB 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 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, J

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

2020-07-22 Thread Bruno Cadonna
-pending-compaction-bytes" capture all the memory usage 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), a

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

2020-07-24 Thread Bruno Cadonna
t; + "block-cache-usage" + "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 th

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-27 Thread Bruno Cadonna
other 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 a

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

2020-07-28 Thread Bruno Cadonna
0, at 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: [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 https://cwiki.apache.org/confluence/display/KAFKA/KIP-648%3A+Renaming+getter+method+for+Interactive+Queries

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

2020-08-10 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. Cong

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 befo

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 bette

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

2020-08-20 Thread Bruno Cadonna
the comments 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 Interfa

[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-662: Throw Exception when Source Topics of a Streams App are Deleted

2020-08-25 Thread Bruno Cadonna
hat occurs during 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 lik

[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-647: Add ability to handle late messages in streams-aggregation

2020-08-26 Thread Bruno Cadonna
application currently 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 writin

[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. https://cwiki.apache.org/confluence/display/KAFKA/KIP-663%3A+API+to+Start+and+Shut+Down+Stream

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
ob as a user to update the config to match when I add or 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). I agree that it is fine for now and I will add a few words in the KIP. Thanks again! -John On Wed,

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
e my job as a user to update the config to match when I add or 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

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
or 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 strea

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

2020-08-31 Thread Bruno Cadonna
ug 25, 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-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: [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
. What do you 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 de

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
PM, Walker 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

[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-07 Thread Bruno Cadonna
unning. 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 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.apach

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
ack how many stream threads they have running, maybe 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

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] 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!

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
ise. As for the method name, what about 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 [

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
registering the thread metrics -- 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

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
nsider adding a method that returns the names of dead 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

  1   2   3   4   5   6   7   8   >