[DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-14 Thread Walker Carlson
Hello all, I have picked up and updated KIP-150. Due to changes to the project since KIP #150 was written there are a few items that need to be updated. First item that changed is the adoption of the Materialized parameter. The second item is the WindowedBy. How the old KIP handles windowing is

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-14 Thread Walker Carlson
https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup here is a link On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson wrote: > Hello all, > > I have picked up and updated KIP-150. Due to changes to the project since > KIP #150 was written there are a fe

[DISCUSS] KIP-399: Extend ProductionExceptionHandler to cover serialization exceptions

2019-10-15 Thread Walker Carlson
Hello all, I would like to restart the discussion of this KIP 399 . I think it is some low hanging fruit that could be quite beneficial. Thanks, Walker

Re: [DISCUSS] KIP-399: Extend ProductionExceptionHandler to cover serialization exceptions

2019-10-17 Thread Walker Carlson
gelog topic, because the store content and the > topic content could diverge, what is especially critical for the EOS case. > > > -Matthias > > On 10/15/19 10:25 AM, Walker Carlson wrote: > > Hello all, > > > > I would like to restart the discussion of this KIP 3

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-17 Thread Walker Carlson
pedKStream#aggregate(...)` the type parameter of > `Materialized` should be `V`, not `VR`? > > > -Matthias > > > > On 10/14/19 2:57 PM, Walker Carlson wrote: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup > > here

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-23 Thread Walker Carlson
point of cogroups that you no longer ever need to specify a Joiner? > If so, you > should add a short line to the KIP explaining that for those of us who > aren't fluent > in cogroup semantics :) > > Cheers, > Sophie > > On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-24 Thread Walker Carlson
t do you (and others?) think about wrapping the config parameters > (ie > > > everything > > > except the actual grouped streams) in a new config object? For example, > > the > > > CogroupedStream#aggregate field could take a single Cogrouped object, > > &

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-25 Thread Walker Carlson
n > >> the opposite direction from the cogroup configuration idea you proposed. > > Can you elaborate? Not sure if I can follow. > > > > -Matthias > > > On 10/24/19 10:20 AM, Walker Carlson wrote: > > While I like the idea Sophie I don't think that it

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-25 Thread Walker Carlson
> > > >> We could take a named parameter from upstream or add an extra naming > > option > > >> however I don't really see the advantage that would give. > > > > Are you familiar with KIP-307? Before KIP-307, KS generated all names > >

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-28 Thread Walker Carlson
ld be better to stick with our current pattern and consider > changes we can apply to the entire API in a later KIP. > > Just my 2 cents. > > Thanks, > Bill > > > On Fri, Oct 25, 2019 at 4:34 PM Walker Carlson > wrote: > > > Hi Guozhang, > > >

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-28 Thread Walker Carlson
this if you have a different opinion on the tradeoff. Walker On Mon, Oct 28, 2019 at 12:14 PM Guozhang Wang wrote: > Hi Walker, > > On Fri, Oct 25, 2019 at 1:34 PM Walker Carlson > wrote: > > > Hi Guozhang, > > > > 1. I am familiar with the cogroup of spark, it

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-28 Thread Walker Carlson
perator. But > (although this seems on the face to be a good option), I think it > might be a briarpatch. Even if we are able to reach a suitable design, > we'd have to contend with the fact that it looks like the > KGroupedStream API, but behaves differently. > > What do you think

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-29 Thread Walker Carlson
e other side > it requires us to do this optimization inside the Streams with some syntax > hint from users (for example, users need to specify it is a "stitching > join" such that all fields are still preserved in the join result). WDYT? > > > Guozhang > > > On M

[VOTE] KIP-150 - Kafka-Streams Cogroup

2019-10-31 Thread Walker Carlson
Hello all, I'd like to call a vote on the updated KIP-150: Kafka-Streams Cogroup found here Thanks, Walker

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-31 Thread Walker Carlson
treams are co-partitioned/merged in `cogroup()` and thus would be > > different compare the the aggregate-join pattern. > > > > (3) For the current KIP writeup, I agree that adding `Named` to > > `aggregate()` aligns best with the current API layout. I also don't >

Re: [VOTE] KIP-150 - Kafka-Streams Cogroup

2019-11-06 Thread Walker Carlson
ote: > > > Thanks for the KIP. > > +1 (binding) > > > > -Bill > > > > > > On Wed, Nov 6, 2019 at 1:22 AM Matthias J. Sax > > wrote: > > > > > +1 (binding) > > > > > > > > > On 10/31/19 10:52 AM, Walker Carls

Re: [DISCUSS] KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size

2020-08-20 Thread Walker Carlson
Hi Leah, Could you explain a bit more why we do not wish to let TimeWindowedDeserializer and WindowedSerdes be created without a specified time as a parameter? I understand the long.MAX_VALUE could cause problems but would it not be a good idea to have a usable default or fetch from the config if

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

2020-08-26 Thread Walker Carlson
Hello Burno, Thanks for the KIP! Not to pile on, but I also had a couple additional questions. I am not super familiar with the StreamThread internals so please forgive any misconceptions if these are not relevant questions. 1. In requestClose if a thread does not close properly and deadlocks fo

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

2020-08-27 Thread Walker Carlson
+1 (Non Binding). Good Kip Bruno Walker On Tue, Aug 25, 2020 at 11:17 AM Guozhang Wang wrote: > +1. Thanks Bruno! > > > Guozhang > > On Tue, Aug 25, 2020 at 4:00 AM Bruno Cadonna wrote: > > > Hi, > > > > I would like to start the vote for > > > > > > > https://cwiki.apache.org/confluence/displ

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

2020-09-01 Thread Walker Carlson
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 will update t

Re: Discussion on KIP-670 : Add ConsumerGroupCommand to delete static members

2020-09-03 Thread Walker Carlson
Hello Sandeep, Reading through your kip it seems like a good idea and pretty straight forward. So I have no problems with this proposal. Thanks for the Kip, Walker On Thu, Sep 3, 2020 at 8:28 AM Sandeep Kumar wrote: > Hi All, > > I am new to the Kafka contribution community. I have picked up

Kip creation permission

2020-09-11 Thread Walker Carlson
Can you give me permission to create a Kip? username: wcarlson thanks, Walker

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

2020-09-11 Thread Walker Carlson
Hello all, I have created KIP-671 to give the option to shutdown a streams application in response to an error. https://cwiki.apache.org/confluence/display/KAFKA/KIP-671%3A+Shutdown+Streams+Application+when+appropriate+exception+is+thrown This is because of the Jira ticket

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

2020-09-14 Thread Walker Carlson
d to be thrown by user code? > However, in the motivation you mention the scenario of a missing source > topic that a user cannot detect, but KafkaStreams runtime would be > responsible to handle. > > How do both things go together? > > > -Matthias > > On 9/11/20 10:3

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

2020-09-16 Thread Walker Carlson
ke to clarify, would this function > > block on the local instance to complete shutting down all its threads > like > > `close()` as well, or would it just to initiate the shutdown and not wait > > for local threads at all? Also a nit suggestion regarding the name, if it > > i

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

2020-09-16 Thread Walker Carlson
tween "shutdownApplication", "initateCloseAll", "closeAllInstaces" or some variation. I am rather indifferent to the name. I think that they all get the point across. The most clear to me would be shutdownApplicaiton or closeAllInstacnes but WDYT? Walker On Wed, Sep 16, 20

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

2020-09-16 Thread Walker Carlson
m the proposal, I'd like to make > sure that we are on the same page regarding its blocking behavior since it > is quite crucial to users as well. Could you elaborate a bit more? > > Regarding the function name, I guess my personal preference would depend on > its actual blocking

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

2020-09-21 Thread Walker Carlson
; > Guozhang > > > On Wed, Sep 16, 2020 at 12:32 PM Walker Carlson > wrote: > > > Hello Guozhang, > > > > As for the logging I plan on having three logs. First, the client log > that > > it is requesting an application shutdown, second, the leader log >

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

2020-09-21 Thread Walker Carlson
Hello all, I would like to start a thread to vote for KIP-671 to add a method to close all clients in a kafka streams application. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-671%3A+Shutdown+Streams+Application+when+appropriate+exception+is+thrown Discussion thread: *here

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

2020-09-21 Thread Walker Carlson
an error code.", is there a > dedicated error code that we should define here, or it is case-by-case? > > On Mon, Sep 21, 2020 at 1:38 PM Walker Carlson > wrote: > > > I am changing the name to "Add method to Shutdown entire Streams > > Application&

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

2020-09-23 Thread Walker Carlson
ritical cases where > stopping record consumption may prevent or limit damage. I am not saying > that it must be a guarantee, but this question should be answered in the > KIP, IMO. > > Best, > Bruno > > On 22.09.20 01:14, Walker Carlson wrote: > > The error code right now

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

2020-09-28 Thread Walker Carlson
, I’m inclined to agree with Walker > that > > we should leave this as a problem for the future. > > > > > > It should certainly be mentioned in the kip, and it also deserves > > special mention in our javadoc and html docs for this feature. > > > > >

Re: [VOTE] KIP-759: Unneeded repartition canceling

2023-08-01 Thread Walker Carlson
+1 (binding) On Mon, Jul 31, 2023 at 10:43 PM Matthias J. Sax wrote: > +1 (binding) > > On 7/11/23 11:16 AM, Shay Lin wrote: > > Hi all, > > > > I'd like to call a vote on KIP-759: Unneeded repartition canceling > > The KIP has been under discussion for quite some time(two years). This > is a >

Re: [DISCUSS] KIP-960: Support interactive queries (IQv2) for versioned state stores

2023-09-11 Thread Walker Carlson
Thanks for the KIP Alieh! I don't have anything to add to the 960 discussion right now as it seems rather straightforward. I think after you address Bruno's comments we can bring it to a vote. I'll review the two spawned KIPs separately. Keep it up, Walker On Wed, Sep 6, 2023 at 5:11 AM Bruno Ca

Re: [ANNOUNCE] New committer: Lucas Brutschy

2023-09-21 Thread Walker Carlson
Congrats Lucas! On Thu, Sep 21, 2023 at 11:42 AM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Congrats Lucas! > > On Thu, Sep 21, 2023, 22:05 Boudjelda Mohamed Said > wrote: > > > Congratulations, Lucas!! > > > > On Thu 21 Sep 2023 at 18:34, Lianet M. wrote: > > > > > Congrat

Re: [DISCUSS] KIP-968: Support single-key_multi-timestamp interactive queries (IQv2) for versioned state stores

2023-10-03 Thread Walker Carlson
Hey Alieh thanks for the KIP, Weighing in on the AsOf vs Until debate I think either is fine from a natural language perspective. Personally AsOf makes more sense to me where until gives me the idea that the query is making a change. It's totally a connotative difference and not that important. I

Re: [DISCUSS] KIP-985 Add reverseRange and reverseAll query over kv-store in IQv2

2023-10-03 Thread Walker Carlson
Hello Hanyu, Looking over your kip things mostly make sense but I have a couple of comments. 1. You have "withDescandingOrder()". I think you mean "descending" :) Also there are still a few places in the do where its called "setReverse" 2. Also I like "WithDescendingKeys()" better 3.

Re: [VOTE] KIP-960: Support single-key_single-timestamp interactive queries (IQv2) for versioned state stores

2023-10-11 Thread Walker Carlson
+1 (binding) Thanks for the kip Alieh! Walker On Wed, Oct 11, 2023 at 3:52 AM Bruno Cadonna wrote: > Thanks for the KIP, Alieh! > > +1 (binding) > > Best, > Bruno > > On 10/10/23 1:14 AM, Matthias J. Sax wrote: > > One more nit: as discussed on the related KIP-698 thread, we should not > > use

Re: [VOTE] KIP-985 Add reverseRange and reverseAll query over kv-store in IQv2

2023-10-17 Thread Walker Carlson
+1 (binding_ Thanks! On Tue, Oct 17, 2023 at 3:22 AM Lucas Brutschy wrote: > +1 (binding) > > Thanks for the KIP! > > On Tue, Oct 17, 2023 at 2:31 AM Matthias J. Sax wrote: > > > > +1 (binding) > > > > > > On 10/13/23 9:24 AM, Hanyu (Peter) Zheng wrote: > > > Hello everyone, > > > > > > I woul

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-04-30 Thread Walker Carlson
st have two type parameters . > The nice aspect of this idea is that the option to enable/disable > reprocessing on restore is only needed when a processor supplier is > passed into the methods. That is not true for idea (a). > > > (2) Yes, that was my intent. > > > Best,

Re: [VOTE] KIP-1038: Add Custom Error Handler to Producer

2024-05-20 Thread Walker Carlson
Hey Alieh, Thanks for the KIP. +1 binding Walker On Tue, May 7, 2024 at 10:57 AM Alieh Saeedi wrote: > Hi all, > > It seems that we have no more comments, discussions, or feedback on > KIP-1038; therefore, I’d like to open voting for the KIP: Add Custom Error > Handler to Producer > < > https

Re: [VOTE] KIP-969: Support range interactive queries (IQv2) for versioned state stores

2023-12-13 Thread Walker Carlson
+1 binding Thanks for the KIP Alieh! On Mon, Dec 11, 2023 at 2:14 PM Alieh Saeedi wrote: > Hi everyone, > > Thanks to everyone who has reviewed KIP-969, and participated in the > discussion thread! > > I'd also like to thank you in advance for taking the time to vote. > > Cheers, > Alieh >

[DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-02-29 Thread Walker Carlson
Hello everybody, I wanted to propose a change to our addGlobalStore methods so that the restore behavior can be controlled on a preprocessor level. This should help Kafka Stream users to better tune Global stores added with the processor API to better fit their needs. Details are in the kip here:

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-13 Thread Walker Carlson
Thanks for the feedback Bruno, Matthias, and Lucas! There is a decent amount but I'm going to try and just hit the major points as I would like to keep this change simple. I've made corrections for the mistakes pointed out. Thanks for the suggestions everyone. The main sticking point seems to be

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-25 Thread Walker Carlson
mits the flexibility in naming. Why do you not use > Named for the name of the source node? > > 2. > Did you consider Matthias' proposal to restrict the type of the store > builder to `StoreBuilder` (or even > `StoreBuilder`) for the case where > the processor is built-in? &

[VOTE] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-25 Thread Walker Carlson
Hello everybody, I think we have had some pretty good discussion on this kip and it seems that we are close if not yet settled on the final version. So I would like to open up the voting for KIP-1024: https://cwiki.apache.org/confluence/x/E4t3EQ Thanks everyone! Walker

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-28 Thread Walker Carlson
"Now the restore is done by > >>> reprocessing using an instance from the customer processor supplier" > >> which > >>> I suppose fixed a long-standing bug ( > >>> https://issues.apache.org/jira/browse/KAFKA-8037) but only for > >

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-04-09 Thread Walker Carlson
r what seems better. > >> -- If we want, we could make it somewhat flexible and support both > >> `KeyValueStore` and `TimestampedKeyValueStore` -- ie, the signature > >> would be `KeyValueStore` but we explicitly check if the builder gives > >> us a `Timestamped

Re: [VOTE] KIP-869: Improve Streams State Restoration Visibility

2022-10-25 Thread Walker Carlson
+1 non binding Thanks for the kip! On Thu, Oct 20, 2022 at 10:25 PM John Roesler wrote: > Thanks for the KIP, Guozhang! > > I'm +1 (binding) > > -John > > On Wed, Oct 12, 2022, at 16:36, Nick Telford wrote: > > Can't wait! > > +1 (non-binding) > > > > On Wed, 12 Oct 2022, 18:02 Guozhang Wang,

Re: [DISCUSS] KIP-878: Autoscaling for Statically Partitioned Streams

2022-10-25 Thread Walker Carlson
Hey Sophie, Thanks for the KIP. I think this could be useful for a lot of cases. I also think that this could cause a lot of confusion. Just to make sure we are doing our best to prevent people from misusing this feature, I wanted to clarify a couple of things. 1) There will be only an interface

Re: [VOTE] KIP-869: Improve Streams State Restoration Visibility

2023-01-24 Thread Walker Carlson
in metrics design > > that "Kafka would provide the lowest necessary metrics levels, and > > users can do the roll-ups however they want". > > > >> > >> (6) What about "warmup tasks"? Internally, we treat them as standbys, > >> but i

Re: [VOTE] KIP-591: Add Kafka Streams config to set default state store

2022-01-20 Thread Walker Carlson
+1 non binding On Thu, Jan 20, 2022 at 2:00 PM Matthias J. Sax wrote: > +1 (binding) > > On 1/20/22 10:52 AM, Guozhang Wang wrote: > > Thanks Luke! I'm +1 on the KIP. > > > > > > Guozhang > > > > On Wed, Jan 19, 2022 at 5:58 PM Luke Chen wrote: > > > >> Hi devs, > >> > >> I'd like to start a vo

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-13 Thread Walker Carlson
+1 from me (non-binding) Walker On Wed, May 11, 2022 at 12:36 PM Leah Thomas wrote: > Thanks Jim, great discussion. +1 from me (non-binding) > > Cheers, > Leah > > On Wed, May 11, 2022 at 10:14 AM Bill Bejeck wrote: > > > Thanks for the KIP! > > > > +1 (binding) > > > > -Bill > > > > On Wed, M

Re: [VOTE] KIP-846: Processor-level Streams metrics for records/bytes Producedd

2022-05-31 Thread Walker Carlson
+1 non binding On Tue, May 31, 2022 at 12:19 PM John Roesler wrote: > +1 (binding) > > Thanks, > John > > On Mon, May 30, 2022, at 13:00, Bill Bejeck wrote: > > +1 (binding) > > > > -Bill > > > > On Mon, May 30, 2022 at 4:49 AM Sagar wrote: > > > >> +1 (non-binding). > >> > >> Thanks! > >> Saga

Re: [DISCUSS] KIP-837 Allow MultiCasting a Result Record.

2022-07-12 Thread Walker Carlson
Hi Sagar, I just finished reading the KIP and this seems to be a great addition. I agree with Matthias that the interface with a default implementation and deprecating partition() does seem cleaner. It has been a pattern that we have followed in the past. How I would handle a custom streams parti

Re: [DISCUSS] KIP-837 Allow MultiCasting a Result Record.

2022-08-09 Thread Walker Carlson
27, 2022 at 4:03 PM Sagar wrote: > > > Thanks Walker for the comments > > > > I have updated the KIP with all the suggestions. > > > > Thanks! > > > > On Tue, Jul 12, 2022 at 10:59 PM Walker Carlson > > wrote: > > > >> Hi Sagar, >

Re: [VOTE] KIP-837 Allow MultiCasting a Result Record.

2022-08-09 Thread Walker Carlson
+1 (non binding) Walker On Tue, May 31, 2022 at 4:44 AM Sagar wrote: > Hi All, > > I would like to start a voting thread on > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211883356 > . > > I am just starting this as the discussion thread has been open for 10+ > days. In case

Re: [VOTE] KIP-904: Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-03-01 Thread Walker Carlson
+1 Binding On Mon, Feb 27, 2023 at 12:48 PM Guozhang Wang wrote: > +1. > > On Sun, Feb 26, 2023 at 4:27 PM Fq Public wrote: > > > > Hi everyone, > > > > I'd like to start the vote on KIP-904: Kafka Streams - Guarantee > subtractor > > is called before adder if key has not changed. > > The KIP i

Re: [VOTE] KIP-907: Add Boolean Serde to public interface

2023-03-01 Thread Walker Carlson
+1 Binding On Mon, Feb 27, 2023 at 1:46 PM Chia-Ping Tsai wrote: > +1 (binding) >

[DISCUSS] KIP-923: Add A Grace Period to Stream Table Join

2023-04-25 Thread Walker Carlson
Hello everybody, I have a stream proposal to improve the stream table join by adding a grace period and buffer to the stream side of the join to allow processing in timestamp order matching the recent improvements of the versioned tables. Please take a look here

Re: [DISCUSS] KIP-923: Add A Grace Period to Stream Table Join

2023-05-18 Thread Walker Carlson
stomize > >> the underlying store via a `WindowStoreSupplier`. Would it make sense > >> for `Joined` to have this as well? I can imagine one may want to limit > >> the number of records in the buffer, for example. If we hit the > >> maximum, the only opt

[VOTE] KIP-923: Add A Grace Period to Stream Table Join

2023-05-24 Thread Walker Carlson
Hello everybody, I'm opening the vote on KIP-923 here . If we have more to discus please continue the discussion on the existing thread https://www.mail-archive.com/dev@kafka.apache.org/msg130657.html best, Walker

Re: [DISCUSS] KIP-923: Add A Grace Period to Stream Table Join

2023-05-30 Thread Walker Carlson
the a join [...] but it is not taken advantage of > in > >> joins," but this doesn't seem true? If the table of a stream-table > join is > >> versioned, then the DSL's stream-table join processor will > automatically > >> perform

Re: [DISCUSS] KIP-925: rack aware task assignment in Kafka Streams

2023-05-31 Thread Walker Carlson
Hi Hao, Most of the comments I had on this kip are already mentioned, but I did want to share my two major concerns. 1. Stability. I worry about stability. If we only have the HA assignor work with rack awareness we will have a lot of state movement in many cases. Sophie and Bruno have this conce

Re: [DISCUSS] KIP-923: Add A Grace Period to Stream Table Join

2023-06-01 Thread Walker Carlson
ow does the proposed buffer, affects processing latency? Could you > please add some words about this to the KIP? > > > Best, > Bruno > > > > > On 31.05.23 01:49, Walker Carlson wrote: > > Thanks for all the additional comments. I will either address them here > o

Re: [DISCUSS] KIP-923: Add A Grace Period to Stream Table Join

2023-06-02 Thread Walker Carlson
s the grace period". I guess that should be "... will dequeue when > the record timestamp is less than (or equal?) stream time minus the > grace period" > > > 5. > What is the difference between not setting the grace period and setting > it to zero? If there

Re: [VOTE] KIP-925: rack aware task assignment in Kafka Streams

2023-06-05 Thread Walker Carlson
+1 (binding) On Mon, Jun 5, 2023 at 3:14 AM Bruno Cadonna wrote: > Hi Hao, > > +1 (binding) > > Thanks! > Bruno > > On 30.05.23 21:16, Colt McNealy wrote: > > +1 (non-binding) > > > > Thank you Hao! > > > > Colt McNealy > > > > *Founder, LittleHorse.dev* > > > > > > On Tue, May 30, 2023 at 9:50 

Re: [DISCUSS] KIP-923: Add A Grace Period to Stream Table Join

2023-06-06 Thread Walker Carlson
ords from the stream-side > >> of the > >> join, for which multiple records with the same key should be treated as > >> separate updates will all must be tracked in the buffer, rather than > >> updates which replace each other. > >> > >> Thanks, >

Re: [VOTE] KIP-923: Add A Grace Period to Stream Table Join

2023-06-06 Thread Walker Carlson
> >> Thanks for the KIP. > >> > >> I've caught up on the discussion thread and I'm satisfied with all > >> responses. > >> > >> +1(binding) > >> > >> -Bill > >> > >> On Mon, Jun 5, 2023 at 10:20 AM Bruno Ca

[DISCUSS] KIP-715: Expose Committed offset in streams

2021-02-22 Thread Walker Carlson
Hello all, I would like to start discussion on KIP-715. This kip aims to make it easier to monitor Kafka Streams progress by exposing the committed offset in a similar way as the consumer client does. Here is the KIP: https://cwiki.apache.org/confluence/x/aRRRCg Best, Walker

Re: [DISCUSS] KIP-715: Expose Committed offset in streams

2021-02-24 Thread Walker Carlson
task progress as -1 for all “supposed to be running” tasks, > > so that on the metrics collector side it could catch any missing tasks. > > > > 5. It seems not clear how users should use `isTaskIdling`. Why not > report a > > map/set for idling tasks just as what we did for

Re: [DISCUSS] KIP-715: Expose Committed offset in streams

2021-02-25 Thread Walker Carlson
t; If > > you feel we have a better way to achieve this, such as querying all the > > input/intermediate topic metadata directly from Kafka for the baseline, I > > think that should be good as well and worth mentioning it in the KIP. > > > > Also it seems that the KIP hasn'

Re: [DISCUSS] KIP-715: Expose Committed offset in streams

2021-02-26 Thread Walker Carlson
gets the > assigned topology and should be able to infer all the exact tasks to be up > and running when the service is healthy. > > On Thu, Feb 25, 2021 at 11:25 AM Walker Carlson > wrote: > > > Thanks for the follow up Boyang and Guozhang, > > > > I have updated

[VOTE] KIP-715: Expose Committed offset in streams

2021-02-26 Thread Walker Carlson
Hello all, I would like to bring KIP-715 to a vote. Here is the KIP: https://cwiki.apache.org/confluence/x/aRRRCg. Walker

Re: [DISCUSS] KIP-715: Expose Committed offset in streams

2021-02-27 Thread Walker Carlson
ust as why we > don't choose to expose as metrics, and how a new method on KStream is not > favorable. These would be valuable when we look back on our design > decisions. > > On Fri, Feb 26, 2021 at 11:23 AM Walker Carlson > wrote: > > > I understand now. I think that

Re: [DISCUSS] KIP-715: Expose Committed offset in streams

2021-03-01 Thread Walker Carlson
nce? If yes, this should be > > pointed out, because if one want to implement a health check this needs > > to be taken into account. > > > > -Matthias > > > > > > On 2/27/21 11:32 AM, Walker Carlson wrote: > > > Sure thing Boyang, > > > &g

Re: [DISCUSS] KIP-715: Expose Committed offset in streams

2021-03-02 Thread Walker Carlson
obal POV (not partial for a > > single local instance). > > > >> Also I mention that they return the highest value they had seen > >> so far for any tasks they have assigned to them. > > > > For the shutdown case maybe, but after a task is closed its metadat

Re: [VOTE] KIP-715: Expose Committed offset in streams

2021-03-02 Thread Walker Carlson
t; On Mon, Mar 1, 2021 at 10:04 AM Leah Thomas > wrote: > >> > >>> Hey Walker, > >>> > >>> Thanks for leading this discussion. +1 from me, non-binding > >>> > >>> Leah > >>> > >>> On Mon, Mar 1, 2021 at 12:37

Re: [DISCUSS] Apache Kafka 2.8.0 release

2021-03-12 Thread Walker Carlson
Hello John, We found a Blocker for 2.8 in our streams soak environment. https://issues.apache.org/jira/browse/KAFKA-12462 - We found a case where a StreamThread can try to transition to PARTITIONS_REVOKED when it was already in PENDING_SHUTDOWN causing an IllegalStateException. Will have a PR

Re: [DISCUSS] Apache Kafka 2.8.0 release

2021-03-19 Thread Walker Carlson
I set its priority to blocker since the bug might break at-least-once > > and exactly-once processing guarantees. > > > > Feel free to set it back to major, if you think that it is not a blocker. > > > > Best, > > Bruno > > > > > > On 13.03.21 00:

Re: [VOTE] KIP-633: Drop 24 hour default of grace period in Streams

2021-04-06 Thread Walker Carlson
This makes sense to me +1! Walker On Tue, Apr 6, 2021 at 11:08 AM Guozhang Wang wrote: > +1. Thanks! > > On Tue, Apr 6, 2021 at 7:00 AM Leah Thomas > wrote: > > > Thanks for picking this up, Sophie. +1 from me, non-binding. > > > > Leah > > > > On Mon, Apr 5, 2021 at 9:42 PM John Roesler wrot

Re: [VOTE] KIP-740: Use TaskId instead of String for the taskId field in TaskMetadata

2021-05-13 Thread Walker Carlson
+1 from me! (non-binding) Walker On Thu, May 13, 2021 at 1:53 PM Sophie Blee-Goldman wrote: > Hey all, > > I'm just going to take this KIP straight to a vote since it should be a > trivial and uncontroversial change. Of course please raise any concerns > should they come up, and I can take thin

Re: [VOTE] KIP-741: Change default serde to be null

2021-05-24 Thread Walker Carlson
+1 (non-binding) from me, Leah Walker On Mon, May 24, 2021 at 1:51 PM Leah Thomas wrote: > Hi, > > I'd like to kick-off voting for KIP-741: Change default serde to be null. > < > https://cwiki.apache.org/confluence/display/KAFKA/KIP-741%3A+Change+default+serde+to+be+null > > > The > discussion

Re: [ANNOUNCE] New Kafka PMC Member: Konstantine Karantasis

2021-06-21 Thread Walker Carlson
Congratulations! On Mon, Jun 21, 2021 at 12:25 PM Dhruvil Shah wrote: > Congratulations Konstantine! Well deserved! > > On Mon, Jun 21, 2021 at 10:20 AM Boyang Chen > wrote: > > > Congratulations Konstantine! > > > > On Mon, Jun 21, 2021 at 10:16 AM Matthias J. Sax > wrote: > > > > > Congrats!

Re: [VOTE] KIP-761: Add Total Blocked Time Metric to Streams

2021-08-02 Thread Walker Carlson
Thanks for the KIP +1 from me (non binding)! Walker On Fri, Jul 30, 2021 at 1:20 PM Sophie Blee-Goldman wrote: > Thanks for updating the KIP, +1 (binding) > > -Sophie > > > On Tue, Jul 27, 2021 at 9:57 AM Guozhang Wang wrote: > > > Hello Rohan, > > > > Thanks for the KIP. As Bruno mentioned in

Re: [VOTE] KIP-740: Use TaskId instead of String for the taskId field in TaskMetadata

2021-08-09 Thread Walker Carlson
propriate. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Thu, May 13, 2021 at 6:42 PM Sophie Blee-Goldman < > > > > > >

Re: [VOTE] Add TaskId field to StreamsException

2021-10-18 Thread Walker Carlson
Hey Sophie, +1 for me, I think that is would only help. Walker On Mon, Oct 18, 2021 at 1:45 AM Luke Chen wrote: > Hi Sophie, > Add taskId to make the exception much clear is a good improvement. > + 1 (non-binding) > > Thank you. > Luke > > On Mon, Oct 18, 2021 at 12:10 PM Sophie Blee-Goldman >

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

2020-09-29 Thread Walker Carlson
Thank you for the feedback Guozhang and Bruno. See the responses below. I have updated the kip accordingly Thanks, Walker On Tue, Sep 29, 2020 at 1:59 AM Bruno Cadonna wrote: > Hi Walker, > > Thanks for updating the KIP! > > 1. I would add response REPLACE_STREAM_THREAD to the > StreamsUncaugh

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

2020-09-29 Thread Walker Carlson
the KIP, Walker! > > I’m +1 (binding) > > -John > > On Mon, Sep 21, 2020, at 17:04, Guozhang Wang wrote: > > Thanks for finalizing the KIP. +1 (binding) > > > > > > Guozhang > > > > On Mon, Sep 21, 2020 at 1:38 PM Walker Carlson > > wrote:

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

2020-09-30 Thread Walker Carlson
t; >> > >>> +1 again on the KIP. > >>> > >>> On Tue, Sep 29, 2020 at 1:51 PM Leah Thomas > wrote: > >>> > >>>> Hey Walker, > >>>> > >>>> Thanks for the KIP! I'm +1, non-binding. > >>>>

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

2020-10-12 Thread Walker Carlson
/diffpagesbyversion.action?pageId=158876566&originalVersion=21&revisedVersion=23 If you have any problems with these changes let me know and we can discuss them further Thank you, Walker On Wed, Sep 30, 2020 at 7:33 AM Walker Carlson wrote: > > Bruno Cadonna > 4:51 AM (2 hours ago) > to dev

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

2020-10-13 Thread Walker Carlson
t not be the best > > user experience: for example, if there is no global thread, one should > > not need to implement the global handler method (and the other way > around). > > > > Thus, it might be good to add default for both methods. If we add > > defau

Re: [ANNOUNCE] New committer: A. Sophie Blee-Goldman

2020-10-19 Thread Walker Carlson
Congratulations Sophie! On Mon, Oct 19, 2020 at 9:43 AM Navinder Brar wrote: > That's great news. Congrats Sophie! Well deserved. > > Regards, > Navinder > On Monday, 19 October, 2020, 10:12:16 pm IST, Bruno Cadonna < > br...@confluent.io> wrote: > > Congrats Sophie! Very well deserved! > >

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

2020-10-19 Thread Walker Carlson
; > > separate handler. You said that the global thread doesn’t have > “replace”, > > > but as of today, none of the threads have “replace”. Why not add that > > > ability when we add it for the other threads? > > > > > > The nature of an uncaught exce

Re: [DISCUSS] KIP-689: Extend `StreamJoined` to allow more store configs

2020-11-30 Thread Walker Carlson
Hello Leah, Thank you for the KIP. I had a couple questions that maybe you can expand on from what is on the KIP. 1) Why are we enabling/disabling the logging by passing in a `WindowBytesStoreSupplier`? It seems to me that these two things should be separate. 2) There is already `withThisStoreS

Re: [DISCUSS] KIP-689: Extend `StreamJoined` to allow more store configs

2020-11-30 Thread Walker Carlson
or false. I've removed the `WindowBytesStoreSupplier otherStoreSupplier` > from the methods in the KIP > 2. I think the fix to 1 answers this question, otherwise, I'm not quite > sure what you're asking. With the updated method calls, there shouldn't be > any duplication. >

Re: [DISCUSS] KIP-689: Extend `StreamJoined` to allow more store configs

2020-12-01 Thread Walker Carlson
> > > > On Mon, Nov 30, 2020 at 5:35 PM Matthias J. Sax > wrote: > > > > > Thanks for the KIP Leah. > > > > > > Should `withLoggingEnabled()` take a `Map config` > > > similar to the one from `Materialized`? > > > > > > > > >

Re: [VOTE] KIP-689: Extend `StreamJoined` to allow more store configs

2020-12-02 Thread Walker Carlson
+1 (non-binding) Thank you, walker On Wed, Dec 2, 2020 at 8:15 AM Bruno Cadonna wrote: > +1 (non-binding) > > Thanks Leah! > > Best, > Bruno > > On 02.12.20 16:55, Leah Thomas wrote: > > Hi all, > > > > I'd like to start the vote for KIP-689 for enabling/disabling logging for > > `StreamJoined`

[DISCUSS] KIP-696: Update Streams FSM to clarify ERROR state meaning

2020-12-08 Thread Walker Carlson
Hello all, I'd like to propose KIP-696 to clarify the meaning of ERROR state in the KafkaStreams Client State Machine. This will update the States to be consistent with changes in KIP-671 and KIP-663. Here are the details: https://cwiki.apache.org/confluence/x/lCvZCQ Thanks, Walker

Re: [DISCUSS] KIP-696: Update Streams FSM to clarify ERROR state meaning

2020-12-08 Thread Walker Carlson
uot;JIRA" field? > > https://issues.apache.org/jira/browse/KAFKA-10555 > https://issues.apache.org/jira/browse/KAFKA-9638 > https://issues.apache.org/jira/browse/KAFKA-6520 > > And maybe we can also left a comment on those tickets explaining what would > happen to tackle the i

  1   2   >