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

[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-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-448: Add State Stores Unit Test Support to Kafka Streams Test Utils

2019-04-09 Thread Bruno Cadonna
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: [DISCUSS] KIP-446: Add changelog topic configuration to KTable suppress

2019-04-12 Thread Bruno Cadonna
hy 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 file is the

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

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

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

2019-06-04 Thread Bruno Cadonna
t; 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-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/stre

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,

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

2019-05-28 Thread Bruno Cadonna
looks good > >> > to me. > >> > > >> > Thanks, > >> > -John > >> > > >> > On Mon, May 20, 2019 at 11:07 AM Bill Bejeck wrote: > >> > > > >> > > Hi Bruno, > >> > > > >> > &

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

2019-06-06 Thread Bruno Cadonna
essing 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 in > > hindsight t

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

2019-06-14 Thread Bruno Cadonna
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. > >>

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

2019-06-19 Thread Bruno Cadonna
at 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. > > > > > > Just my two cents. > > > T

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

2019-06-19 Thread Bruno Cadonna
d, 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 14, 2019

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 a

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

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

2019-05-20 Thread Bruno Cadonna
tions 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 > > of associa

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

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

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

2019-04-30 Thread Bruno Cadonna
hould 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`. > > > > > > > > > -Matthias &g

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

2019-05-02 Thread Bruno Cadonna
n 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't, and I don't see any real downside. > > Paul > > On Tue, Apr 30, 2

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

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

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

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: 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://i

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:

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

2019-08-20 Thread Bruno Cadonna
otal 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

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

2019-09-04 Thread Bruno Cadonna
or Y milliseconds, do so > > > >> now." > > > >> > >>> > > > > >> > >>> > Actually, it might even be beneficial to avoid processing > > > >> standbys a > > > >&

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

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

2019-09-05 Thread Bruno Cadonna
sue > > > > (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 .. | get)-latency (avg | max)`

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

2019-09-04 Thread Bruno Cadonna
me 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, > > > > 1) What do you mean with "full set

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

2019-09-05 Thread Bruno Cadonna
out 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 experience

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

2019-09-04 Thread Bruno Cadonna
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: > > > >> Hi Guozhang,

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

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

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

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: [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`

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

2019-10-02 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 - >

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

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

Re: [kafka-clients] [VOTE] 2.4.0 RC0

2019-11-18 Thread Bruno Cadonna
Hi, ClientMetricsTest.shouldAddCommitIdMetric should only fail if executed from an IDE. The test fails because the test expects a file on the class path which is not there when the test is executed from the IDE, but is there when the test is executed from gradle. I will try to fix the test so

ClientsMetricsTest.shouldAddCommitIdMetric() failed on RC0 ...

2019-11-18 Thread Bruno Cadonna
Hi Vahid and Eric, Thank you for your input. I suppose you both used the archive of the release candidate and did not checkout the tag from the git repository. I found the issue. The archive misses the .git directory that is needed for the unit test to pass. Opened the following PR to fix it:

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

Re: ClientsMetricsTest.shouldAddCommitIdMetric() failed on RC0 ...

2019-11-19 Thread Bruno Cadonna
munity feedback. > > On Mon, Nov 18, 2019 at 12:24 PM Bruno Cadonna wrote: >> >> Hi Vahid and Eric, >> >> Thank you for your input. >> >> I suppose you both used the archive of the release candidate and did >> not checkout the tag from the git repository.

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

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

2019-10-07 Thread Bruno Cadonna
p. 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 Regards, > Rabi Kuma

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

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

2020-02-26 Thread Bruno Cadonna
> > >> If there isn't anything else, I will probably start the voting process > > > >> next week! > > > >> > > > >> Cheers, > > > >> Richard > > > >> > > > >> > > > >> On Fri, Feb 21, 2

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

2020-03-02 Thread Bruno Cadonna
Hi Richard, +1 (non-binding) Best, Bruno On Mon, Mar 2, 2020 at 4:33 PM John Roesler wrote: > > Hi Richard, > > Thanks for the KIP! > > I'm +1 (binding) > > -john > > On Thu, Feb 27, 2020, at 14:40, Richard Yu wrote: > > Hi all, > > > > I am proposing a new optimization to Kafka Streams which

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

Re: [ANNOUNCE] New committer: Konstantine Karantasis

2020-02-27 Thread Bruno Cadonna
Congrats, Konstantine! Excellent! Best, Bruno On Thu, Feb 27, 2020 at 11:28 AM Rajini Sivaram wrote: > > Congratulations Konstantine! > > On Thu, Feb 27, 2020 at 9:57 AM Mickael Maison > wrote: > > > Congratulations Konstantine! Well deserved > > > > On Thu, Feb 27, 2020 at 7:11 AM Satish

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

2020-02-27 Thread Bruno Cadonna
eeping a "discarded" section in the document will just make it > harder for new people to understand the proposal. We did the same thing > with KIP-441, where there were two prior drafts included at the end of the > document, and we just deleted them for clarity. > > I liked th

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

2020-03-03 Thread Bruno Cadonna
r the votes so far! > > @Matthias or @Guozhang Wang it would be great to > > also get your input on this KIP. > > > > It looks to be pretty close to completion, so the finishing touches are all > > we need. :) > > > > Best, > > Richard > > > >

Re: [VOTE] KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-23 Thread Bruno Cadonna
Hi Navinder, +1 (non-binding) Best, Bruno On Thu, Jan 23, 2020 at 9:19 AM John Roesler wrote: > > Thanks, Navinder. It's just to give everyone a chance to object if they > wanted to. > -John > > On Thu, Jan 23, 2020, at 00:44, Navinder Brar wrote: > > Oh sorry, my bad. Will wait for another

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

2020-02-02 Thread Bruno Cadonna
t; >>>> I've finished making some initial modifications to the KIP. > > >>>> I have decided to keep the implementation section in the KIP for > > >>>> record-keeping purposes. > > >>>> > > >>>> For now, we should focu

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

2020-01-24 Thread Bruno Cadonna
Hi Richard, Thank you for the KIP. I agree with John that we should focus on the interface and behavior change in a KIP. We can discuss the implementation later. I am also +1 for the survey. I had a thought about this. Couldn't we consider emit-on-change to be one config of suppress (like

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-23 Thread Bruno Cadonna
Hi Navinder, Thank you for the KIP! It looks good to me. Here my comments: 1) I agree with John and Matthias that you should remove the implementation of the methods in the KIP. Just the method signatures suffice and make the reading easier. 2) According to the grammar

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

2020-01-24 Thread Bruno Cadonna
he opt-out config. I > > really think that (because of the timestamp > > semantics work already underway), we're already > > pre-fetching the prior result most of the time, so > > there would actually be very little extra I/O > > involved in implementing emit-on-change. &

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-23 Thread Bruno Cadonna
ys we > > are not using constructor, so the only way the variable is exposed > > outside is the getter and the optional builder method. With this > > variable name, we can name the builder method as "enableStaleStores" > > and I feel staleStoresEnabled() is more r

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

2020-02-19 Thread Bruno Cadonna
t we make > > > >>> ChangeDetector > > > >>>>>>>>>> an optional parameter to every operation in Streams, but this > > > >>>>>>>>>> seems to carry quite a bit of mental burden with it. People >

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-02-11 Thread Bruno Cadonna
this rule as more of a style guide than a > grammar, but whatever. As a maintainer, I think we should discourage > these particular classes to have different instance variables than > method names. Otherwise, it's just silly. Either "includeStaleStores" > or "staleSto

Re: [VOTE] KIP-568: Explicit rebalance triggering on the Consumer

2020-02-14 Thread Bruno Cadonna
Thanks! +1 (non-binding) Best, Bruno On Fri, Feb 14, 2020 at 1:57 AM Boyang Chen wrote: > > +1 (non-binding) > > On Thu, Feb 13, 2020 at 4:45 PM Guozhang Wang wrote: > > > +1 (binding). > > > > > > Guozhang > > > > On Tue, Feb 11, 2020 at 5:29 PM Guozhang Wang wrote: > > > > > Hi Sophie, > >

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

2020-02-20 Thread Bruno Cadonna
geDetector interface, then > we should consider adding a configuration as part of that future work. In > fact, if we add a simple "opt-in/opt-out" switch right now, we might find > that it's actually insufficient for whatever future feature we might propose, > then we have a mes

Re: Add to Dev mailing list

2020-01-21 Thread Bruno Cadonna
Hi Deepak, Subscription to the mailing lists is self service. See https://kafka.apache.org/contact Best, Bruno On Tue, Jan 21, 2020 at 8:35 AM Deepak Goyal wrote: > > Hi, > > Can you please add me to the dev mailing list > > -- > Deepak Goyal (+919502447100)

Re: Issues with triggering the build

2020-01-14 Thread Bruno Cadonna
Hi, I tried with "Retest this, please" but it didn't work. Best, Bruno On Tue, Jan 14, 2020 at 9:01 AM Levani Kokhreidze wrote: > > Hello, > > Seems like there’re issues with triggering Jenkins builds. Latest commits for > my PR doesn’t trigger any of the builds. > Any ideas how to fix the

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

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,

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

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

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: [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. > > > &

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

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

[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: [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-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.

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

[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

[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: [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-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-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] 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

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

  1   2   3   4   5   6   7   >