Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-08-24 Thread Nick Telford
old > ones. You can find examples of deprecated metrics here: > https://kafka.apache.org/documentation/#selector_monitoring > > > 5. > Why does the KIP mention position files? I do not think they are related > to transactions or flushes. > > > 6. > I think we will

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-11 Thread Nick Telford
e has a > detailed breakdown of the testing. > > Anyways, I'm super excited about this KIP and if a bit more future testing > goes well, we plan to ship our product with a build of KIP-892, Speedb OSS, > and potentially a few other minor tweaks that we are thinking about

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-13 Thread Nick Telford
patibility, Deprecation, and Migration Plan". > > > 6. > Describing upgrading and downgrading in the KIP is a good idea. > Regarding downgrading, I think you could also catch the exception and do > what is needed to downgrade, e.g., drop the column family. See here for > an example: >

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-13 Thread Nick Telford
tionLevels, even if the StateStore has many layers of wrappers (as is the case at the point IQv1 deals with the store). Would this be acceptable, or do you have another approach in mind? Regards, Nick On Wed, 13 Sept 2023 at 10:57, Nick Telford wrote: > Hi Bruno, > > Thanks for gettin

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-13 Thread Nick Telford
nage to get > the internals right. Regarding state stores that do not support > READ_COMMITTED, they should throw an error stating that they do not > support READ_COMMITTED. No need to adapt all state stores immediately. > > 3b. > I am in favor of using transactions also for ALOS. &g

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-13 Thread Nick Telford
IQ Iterators in the same way that RocksDB WriteBatches do. -- Nick On Wed, 13 Sept 2023 at 16:58, Nick Telford wrote: > Hi Bruno, > > I've updated the KIP based on our conversation. The only things I've not > yet done are: > > 1. Using transactions under ALOS and EOS.

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-18 Thread Nick Telford
because processing is deterministic. Additionally, IQ being able to read > > uncommitted records is crucial to enable "read your own writes" on our > API: > > Due to the deterministic processing, we send an "ack" to the client who > > makes the request as soo

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-18 Thread Nick Telford
Oh! One other concern I haven't mentioned: if we make IsolationLevel a query-time constraint, then we need to add support for READ_COMMITTED to InMemoryKeyValueStore too, which will require some changes to the implementation. On Mon, 18 Sept 2023 at 17:24, Nick Telford wrote: > Hi

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-09-19 Thread Nick Telford
t 2023 at 09:30, Bruno Cadonna wrote: > Why do we need to add READ_COMMITTED to InMemoryKeyValueStore? I think > it is perfectly valid to say InMemoryKeyValueStore do not support > READ_COMMITTED for now, since READ_UNCOMMITTED is the de-facto default > at the moment. > > Best, &g

[DISCUSS] KIP-989: RocksDB Iterator Metrics

2023-10-04 Thread Nick Telford
Hi everyone, KIP-989 is a small Kafka Streams KIP to add a few new metrics around the creation and use of RocksDB Iterators, to aid users in identifying "Iterator leaks" that could cause applications to leak native memory. Let me know what you think! https://cwiki.apache.org/confluence/display/K

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2023-10-05 Thread Nick Telford
erator have its own "createdAt()" or equivalent field, or do we > need to keep track of the Iterator's start time upon creation? > > Cheers, > Colt McNealy > > *Founder, LittleHorse.dev* > > > On Wed, Oct 4, 2023 at 9:07 AM Nick Telford > wrote: > > &g

[DISCUSS] KIP-990: Capability to SUSPEND Tasks on DeserializationException

2023-10-12 Thread Nick Telford
Hi everyone, This is a Streams KIP to add a new DeserializationHandlerResponse, "SUSPEND", that suspends the failing Task but continues to process other Tasks normally. https://cwiki.apache.org/confluence/display/KAFKA/KIP-990%3A+Capability+to+SUSPEND+Tasks+on+DeserializationException I'm not ye

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-10-13 Thread Nick Telford
goes well > you can do a quick KIP to enable it by default as soon as the > isolation.level config has been > completed. But feel free to just pick whichever option is easiest or > quickest for you to implement) > > Hope this helps move the discussion forward, > Sophie > >

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-10-13 Thread Nick Telford
always including a > feature > > flag in large structural > > changes like this. No matter how much I trust someone or myself to > > implement a feature, you just > > never know what kind of bugs might slip in, especially with the very > first > > iteration

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-10-13 Thread Nick Telford
re is a guideline in Kafka not to use the get prefix for getters (at > least in the public API). Thus, could you please rename > > getCommittedOffset(TopicPartition partition) -> > committedOffsetFor(TopicPartition partition) > > You can also propose an alternative to c

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-10-16 Thread Nick Telford
n that we can > potentially deprecate than "shut the door, clean for everyone". More > specifically, allowing the processing mode / IQ read mode to be > decoupled, and if we found that there's no such cases as I speculated > above or people started complaining a lot, we

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2023-10-17 Thread Nick Telford
closed iterator or the > number of iterators. But it could still be good to identify those > leaks early. One option would be to add `iterator-duration-max` and > take open iterators into account when computing the metric. > > Cheers, > Lucas > > > On Thu, Oct 5, 2023 at 3:5

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-10-17 Thread Nick Telford
; behavior (we don't necessarily have to be, because it doesn't violate > ALOS guarantees as far as I can see), we could make > ALOS/READ_COMMITTED more similar to ALOS/READ_UNCOMITTED by flushing > the WriteBatch on error (obviously, only if we have a chance to do > that). >

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-10-18 Thread Nick Telford
This KIP is already improving the > situation a lot by not wiping the state store. > > Cheers, > Lucas > > On Tue, Oct 17, 2023 at 3:51 PM Nick Telford > wrote: > > > > Hi Lucas, > > > > Yeah, this is pretty much the direction I'm thinking of goin

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2023-10-24 Thread Nick Telford
ficantly. So we may even want to detect if the > > iterators are opened by one-time / rare queries against the state > > store. > > > > But, as I said, that would be an addition and not a change of the > > current contents of the KIP, so I'd support the KIP movi

Re: [DISCUSS] KIP-990: Capability to SUSPEND Tasks on DeserializationException

2023-10-24 Thread Nick Telford
wnstream/dependent tasks? Should be able to add this > information to the subscription metadata and send to all clients via a > rebalance. There might be better options I'm not seeing. Or we could just > decide to trust the users not to shoot themselves in the

Re: [DISCUSS] KIP-990: Capability to SUSPEND Tasks on DeserializationException

2023-10-26 Thread Nick Telford
he #resume method so that it always skips over the bad record. This > will probably be the easiest to implement anyways, as it is effectively the > same as the CONTINUE option internally, but gives the user time to > decide if they really do want to CONTINUE or not > > Not sure if

[VOTE] KIP-892: Transactional StateStores

2023-11-13 Thread Nick Telford
Hi everyone, I'd like to call a vote for KIP-892: Transactional StateStores[1], which makes Kafka Streams StateStores transactional under EOS. Regards, Nick 1: https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores

Re: [VOTE] KIP-892: Transactional StateStores

2023-11-17 Thread Nick Telford
n-binding). > >>> > >>> Thank you, Nick, for making all of the changes (especially around the > >>> `default.state.isolation.level` config). > >>> > >>> Colt McNealy > >>> > >>> *Founder, LittleHorse.dev* > >

Re: Apache Kafka 3.7.0 Release

2023-11-23 Thread Nick Telford
Hi Stan, I'd like to propose including KIP-892 in the 3.7 release. The KIP has been accepted and I'm just working on rebasing the implementation against trunk before I open a PR. Regards, Nick On Tue, 21 Nov 2023 at 11:27, Mayank Shekhar Narula < mayanks.nar...@gmail.com> wrote: > Hi Stan > > C

[VOTE] KIP-989: RocksDB Iterator Metrics

2024-05-14 Thread Nick Telford
Hi everyone, I'd like to call a vote on the Kafka Streams KIP-989: RocksDB Iterator Metrics: https://cwiki.apache.org/confluence/display/KAFKA/KIP-989%3A+RocksDB+Iterator+Metrics All of the points in the discussion thread have now been addressed. Regards, Nick

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-14 Thread Nick Telford
>>> > >>>>>>> 102: It's unclear to me, how `.position` information is added. > >>>>>>> The KIP only says: "position offsets will be stored in RocksDB, > >>>>>>> in the same column family as the changelog offsets". Do you >

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-05-14 Thread Nick Telford
etter, I > > > > would propose `iterator-max-age-ms`. I should be sufficient to call > out > > > > (as it's kinda "obvious" anyway) that the metric applies to open > > > > iterator only. > > > > > > > > And yes, I was hoping that the code inside M

Re: [DISCUSS] Apache Kafka 3.8.0 release

2024-05-15 Thread Nick Telford
Hi Josep, Would it be possible to sneak KIP-989 into 3.8? Just as with 1028, it's currently being voted on and has already received the requisite votes. The only thing holding it back is the 72 hour voting window. Vote thread here: https://lists.apache.org/thread/nhr65h4784z49jbsyt5nx8ys81q90k6s

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-05-16 Thread Nick Telford
e: > One quick thing -- can you update the title of this KIP to reflect the > decision to implement these metrics for all state stores implementations > rather than just RocksDB? > > > On Tue, May 14, 2024 at 1:36 PM Nick Telford > wrote: > > > Woops! Thanks for the cat

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-05-16 Thread Nick Telford
/current/streams/monitoring.html#state-store-metrics On Thu, 16 May 2024 at 12:15, Nick Telford wrote: > Good point! I've updated it to "Improved StateStore Iterator metrics for > detecting leaks" - let me know if you have a better suggestion. > > This should affect th

Re: [VOTE] KIP-989: RocksDB Iterator Metrics

2024-05-16 Thread Nick Telford
t; > > > On 5/14/24 9:19 AM, Lucas Brutschy wrote: > > > Hi Nick! > > > > > > Thanks for the KIP. > > > > > > +1 (binding) > > > > > > On Tue, May 14, 2024 at 5:16 PM Nick Telford > > wrote: > > >> &g

Re: [VOTE] KIP-989: RocksDB Iterator Metrics

2024-05-16 Thread Nick Telford
p Prat > Open Source Engineering Director, aivenjosep.p...@aiven.io | > +491715557497 | aiven.io > Aiven Deutschland GmbH > Alexanderufer 3-7, 10117 Berlin > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen > Amtsgericht Charlottenburg, HRB 209739 B > > On Thu, May 16, 2024,

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-17 Thread Nick Telford
ss by its own. > > > >>>>>>> > > > >>>>>>> > > > >>>>>>> Couple of question / comment: > > > >>>>>>> > > > >>>>>>> > > > >>>>>>&

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-29 Thread Nick Telford
> > release, but would also like to move forward with with one. > > > > Should we start a VOTE? > > > > For merging PRs we need to wait after code freeze, and 3.8 branch was > > but. But we could start reviewing PRs before this already. > > > > > >

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-29 Thread Nick Telford
e this covers all the outstanding changes that were requested. Please let me know if I've missed anything or you think further changes are needed. Regards, Nick On Wed, 29 May 2024 at 09:28, Nick Telford wrote: > Hi everyone, > > Sorry I haven't got around to updating the KIP yet.

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-30 Thread Nick Telford
) -> numCommittedEntryStored() flushedEntryRemoved(K) -> committedEntryRemoved(K) flushedEntryStored(K) -> committedEntryStored(K) The old methods will obviously be marked as @Deprecated. Any objections before I add this to the KIP? Regards, Nick On Wed, 29 May 2024 at 11:20, Nick Telford wr

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-06-06 Thread Nick Telford
tructure POV is seems best to add to all segments, but it seem > sufficient to keep the information up-to-date only in the latest > segments (what would imply that we need to copy the information from the > current latest segment to a newly created segment explicitly) and only > _read_ t

[VOTE] KIP-1035: StateStore managed changelog offsets

2024-06-12 Thread Nick Telford
Hi everyone, I'd like to call a vote on KIP-1035[1]. Regards, Nick 1: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1035%3A+StateStore+managed+changelog+offsets

Re: [DISCUSS] KIP-1056: Remove `default.` prefix for exception handler StreamsConfig

2024-06-13 Thread Nick Telford
Hi everyone, On a semantic note, would it perhaps make more sense to rename them " uncaught." instead? These handlers are essentially the "last resort" exception handlers, because Exceptions can be caught *within* a component, e.g. a Deserializer can catch and handle an exception without the confi

Re: [VOTE] KIP-1035: StateStore managed changelog offsets

2024-06-20 Thread Nick Telford
) > > > > > > Best, > > > Bruno > > > > > > On 6/13/24 2:31 AM, Matthias J. Sax wrote: > > > > Thanks Nick. > > > > > > > > +1 (binding) > > > > > > > > > > > > Looking forward t

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-08-15 Thread Nick Telford
Hi everyone, Looks really promising, and I can see this resolving several issues I've noticed. I particularly like the choice to use a String for Subtopology ID, as it will (eventually) lead to a better solution to KIP-816. I noticed a few typos in the KIP that I thought I'd mention: NT1. In sev

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-08-16 Thread Nick Telford
le typically use in > their production workloads. > > NT4. We will actually only report the offsets if we manage to acquire > the lock. I tried to make this more precise. I suppose also with > KIP-1035, we'd require the lock to read the offset? > > Cheers, > Lucas > >

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-08-16 Thread Nick Telford
> more that locking is required to prevent concurrent access. In > particular, I was expecting that the lock will avoid two threads > opening the same RocksDB in KIP-1035. Wouldn't this cause problems? > > Cheers, > Lucas > > On Fri, Aug 16, 2024 at 11:34 AM Nick Telford &

Re: Kafka trunk test & build stability

2024-01-02 Thread Nick Telford
Hi everyone, Regarding building a "dependency graph"... Gradle already has this information, albeit fairly coarse-grained. You might be able to get some considerable improvement by configuring the Gradle Remote Build Cache. It looks like it's currently disabled explicitly: https://github.com/apach

Re: Kafka trunk test & build stability

2024-01-02 Thread Nick Telford
Addendum: I've opened a PR with what I believe are the changes necessary to enable Remote Build Caching, if you choose to go that route: https://github.com/apache/kafka/pull/15109 On Tue, 2 Jan 2024 at 14:31, Nick Telford wrote: > Hi everyone, > > Regarding building a &qu

Re: [DISCUSS] KIP-990: Capability to SUSPEND Tasks on DeserializationException

2024-03-28 Thread Nick Telford
offset from outside while the application is running. The offset in > > >> question is cached inside the consumer and the consumer would not go > > >> back to Kafka to re-read the offset (only when a partitions is > > >> re-assigned to a new consumer, the consum

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-28 Thread Nick Telford
;INFO" level for this > metric, > > > since unlike the other metrics which are "Measurables", the current > > > timestamp won't need to be retrieved on each recording > > > > > > 5. Can you list the tags that would be associated with each of

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-28 Thread Nick Telford
Quick addendum: My suggested metric "oldest-open-iterator-age-seconds" should be "oldest-open-iterator-age-ms". Milliseconds is obviously a better granularity for such a metric. Still accepting suggestions for a better name. On Thu, 28 Mar 2024 at 13:41, Nick Telford w

Re: [DISCUSS] KIP-816: Topology changes without local state reset

2024-03-28 Thread Nick Telford
treams. Let me know what you think, Nick On Tue, 15 Feb 2022 at 16:23, Nick Telford wrote: > In the KIP, for Option A I suggested a new path of: > > /state/dir/stores// > > I made the mistake of thinking that the rocksdb/ segment goes *after* the > store name in the current schem

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-31 Thread Nick Telford
terator-age-seconds" should be > > "oldest-open-iterator-age-ms". Milliseconds is obviously a better > > granularity for such a metric. > > > > Still accepting suggestions for a better name. > > > > On Thu, 28 Mar 2024 at 13:41, Nick Telford &g

[DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-07 Thread Nick Telford
Hi everyone, Based on some offline discussion, I've split out the "Atomic Checkpointing" section from KIP-892: Transactional Semantics for StateStores, into its own KIP KIP-1035: StateStore managed changelog offsets https://cwiki.apache.org/confluence/display/KAFKA/KIP-1035%3A+StateStore+managed+

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-10 Thread Nick Telford
plementation detail. I would remove it from the KIP, but still > state that the legacy checkpointing behavior will be supported when the > state store does not manage the checkpoints. > > > 5. > Regarding the metrics, could you please add the tags, and the recording > level

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-10 Thread Nick Telford
encapsulation for the unused task directories? > > > Best, > Bruno > > > > On 4/10/24 11:31 AM, Nick Telford wrote: > > Hi Bruno, > > > > Thanks for the review! > > > > 1, 4, 5. > > Done > > > > 3. > > You'r

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-12 Thread Nick Telford
, but it looks like we'll have to live with it. Unless you have any better ideas? Regards, Nick On Wed, 10 Apr 2024 at 14:12, Nick Telford wrote: > Hi Bruno, > > Immediately after I sent my response, I looked at the codebase and came to > the same conclusion. If it's possibl

Re: [DISCUSS] KIP-1034: Dead letter queue in Kafka Streams

2024-04-12 Thread Nick Telford
Hi Damien, Thanks for the KIP! Dead-letter queues are something that I think a lot of users would like. I think there are a few points with this KIP that concern me: 1. It looks like you can only define a single, global DLQ for the entire Kafka Streams application? What about applications that w

Re: [DISCUSS] KIP-1034: Dead letter queue in Kafka Streams

2024-04-12 Thread Nick Telford
essed. I'd like to see a section that considers these consequences, and perhaps make those risks clear to users. For the record, this is exactly what sunk KIP-990, which was an alternative approach to error handling that introduced the same issues. Cheers, Nick On Fri, 12 Apr 2024 at 1

Re: [DISCUSS] KIP-1034: Dead letter queue in Kafka Streams

2024-04-12 Thread Nick Telford
gt; possible). > > 5. I think we should be clear that this KIP only covers the DLQ record > produced. > Everything related to replay messages or recovery plan should be > considered out-of-scope as it is use-case and error specific. > > Let me know if that's not clear, t

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-15 Thread Nick Telford
t takes in a taskId parameter? We can have it default to > invoking the old one for compatibility reasons and it should be completely > safe to tack on. > > Would also prefer the same for a ProcessorSupplier, but that's definitely > outside the scope of this KIP > > On

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-16 Thread Nick Telford
gt; } > > And of course add the TaskId parameter to each of the actual > state store constructors returned here. > > Does that make sense? It's entirely possible I'm missing something > important here, but I think this would be a pretty small addition that

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2024-04-17 Thread Nick Telford
Hi Walker, Feel free to ask away, either on the mailing list of the Confluent Community Slack, where I hang out :-) The implementation is *mostly* complete, although it needs some polishing. It's worth noting that KIP-1035 is a hard prerequisite for this. Regards, Nick

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-08-21 Thread Nick Telford
operations point of view, just letting the app being blocked > > with a informational log entry to quickly ping-down the zombie clients > > may actually be acceptable. All in all, it makes the code simpler > > programmingly by not trying to abstract away issue scenario a) from > &

Re: [DISCUSS] KIP-857: Streaming recursion in Kafka Streams

2022-08-25 Thread Nick Telford
> In our current examples, the linked output stream and input stream are on > > the same sub-topology, in which case this API allows us to avoid creating > > unnecessary intermediate topics; when the linked output/input streams are > > not on the same sub-topology, then using this AP

Re: [DISCUSS] KIP-857: Streaming recursion in Kafka Streams

2022-09-06 Thread Nick Telford
itioning" in my head, as I don't think it's quite right. It may turn out that the additional overload (with the Produced argument) is not necessary. Thanks for all your feedback so far. Let me know what you think! Regards, Nick On Thu, 25 Aug 2022 at 17:46, Nick Telford wrote:

Re: [DISCUSS] KIP-857: Streaming recursion in Kafka Streams

2022-09-06 Thread Nick Telford
vides no further optimizations. Is my understanding correct? If yes, > I'm wondering if it's worthy the complexity to allow repartitioning inside > the unary operator, or should we just restrict the recursion inside a > single sub-topology. > > > Guozhang > > On Tue, S

Re: [DISCUSS] KIP-857: Streaming recursion in Kafka Streams

2022-09-06 Thread Nick Telford
her before, after or inside the "recursively" method. I can't see a scenario where the recursion would cause problems with it. Nick On Tue, 6 Sept 2022 at 18:08, Nick Telford wrote: > Hi Guozhang, > > I mentioned this in the "Rejected Alternatives" section. Reparti

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

2022-09-20 Thread Nick Telford
Hi Guozhang, KIP looks great, I have one suggestion: in addition to "restore-records-total", it would also be useful to track the number of records *remaining*, that is, the records that have not yet been restored. This is actually the metric I was attempting to implement in the StateRestoreListen

Re: [DISCUSS] KIP-844: Transactional State Stores

2022-10-06 Thread Nick Telford
> > > > > > > > > > > > > > > > > > > > > > > > > > > > https://github.com/apache/kafka/blob/549e54be95a8e1bae1e97df2c21d48c042ff356e/streams/src/main/java/org/apache/kafka/

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

2022-10-10 Thread Nick Telford
gt; For deriving estimated time remaining though, the `total` metric may not > >> be helpful since they will not be "reset" after rebalances, i.e. they > will > >> be an ever-increasing number and record the total number of records for > the > >> lifetime of

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

2022-10-11 Thread Nick Telford
Hi Guozhang, What you propose sounds good to me. Having the more detailed Task-level metrics at DEBUG makes sense. Regards, Nick

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

2022-10-12 Thread Nick Telford
Can't wait! +1 (non-binding) On Wed, 12 Oct 2022, 18:02 Guozhang Wang, wrote: > Hello all, > > I'd like to start a vote for the following KIP, aiming to improve Kafka > Stream's restoration visibility via new metrics and callback methods: > > > https://cwiki.apache.org/confluence/display/KAFKA/K

Streams: clarification needed, checkpoint vs. position files

2022-11-11 Thread Nick Telford
Hi everyone, I'm trying to understand how StateStores work internally for some changes that I plan to propose, and I'd like some clarification around checkpoint files and position files. It appears as though position files are relatively new, and were created as part of the IQv2 initiative, as a

Re: [DISCUSS] KIP-844: Transactional State Stores

2022-11-11 Thread Nick Telford
option c. Existing state is considered to be committed > and there will be an additional RocksDB for uncommitted writes. > > I am out of office until October 24. I will update KIP and make sure that > we have an upgrade test for that after coming back from vacation. > > Best, > Alex &

Re: Streams: clarification needed, checkpoint vs. position files

2022-11-14 Thread Nick Telford
;t think > > it would sense for Streams to try and consolidate these or replace one > with > > another. > > > > Hope this answers your question, and I'll ping John to make sure I'm not > > misleading > > you regarding the usage/intention of P

Streams PR review request

2022-11-18 Thread Nick Telford
Hi everyone, I found a small performance improvement in Kafka Streams state stores, would someone be able to review/merge it please? https://github.com/apache/kafka/pull/12842 (I'm not sure if this is the correct forum for requesting a review/merge. If it isn't, please let me know). Regards, Ni

Re: [DISCUSS] KIP-844: Transactional State Stores

2022-11-21 Thread Nick Telford
w in January. > > Best, > Alex > > On Fri, Nov 11, 2022 at 4:24 PM Nick Telford > wrote: > > > Hi everyone, > > > > Sorry to dredge this up again. I've had a chance to start doing some > > testing with the WIP Pull Request, and it appears as thou

[DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-21 Thread Nick Telford
Hi everyone, As I mentioned in the discussion thread for KIP-844, I've been working on an alternative approach to achieving better transactional semantics for Kafka Streams StateStores. I've published this separately as KIP-892: Transactional Semantics for StateStores

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-22 Thread Nick Telford
method replaces flush, > updateChangelogOffsets, and checkpoint. It seems to me that the point about > atomicity and Position also suggests that it replaces the Position > callbacks. However, the proposal only deprecates `flush`. Should we be > deprecating other methods as well? > >

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-23 Thread Nick Telford
is not > necessary to solve the recovery-under-EOS problem. On the other hand, once > WriteBatchWithIndex is in, it will be much easier to add > state-store-managed checkpointing. > > If you share the current implementation, I am happy to help you address the > OOMe and config

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-28 Thread Nick Telford
end on the processing.mode; under ALOS it would allow more frequently committing stores, whereas under EOS it couldn't. Any better ideas? On Wed, 23 Nov 2022 at 16:25, Nick Telford wrote: > Hi Alex, > > Thanks for the feedback. > > I've updated the discussion of OOM issues

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-28 Thread Nick Telford
dy be explicitly configuring this for their purposes. I think a further optimization for ALOS to only commit the StateStores when exceeding this limit is reasonable, to preserve the user's desired commit.interval.ms as much as possible. On Mon, 28 Nov 2022 at 15:55, Nick Telford wrote: >

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-28 Thread Nick Telford
> store values when copying from the uncommitted to committed store, but I > wasn't able to figure that out when I scanned the PR. > > Colt McNealy > *Founder, LittleHorse.io* > > > On Mon, Nov 28, 2022 at 7:56 AM Nick Telford > wrote: > > > Hi everyone, &

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-30 Thread Nick Telford
rns around concurrency, especially in the presence of Iterators. I'm thinking of wrapping WriteBatchWithIndex with a reference-counting copy-on-write implementation (that only makes a copy if there's an active iterator), but I'm open to suggestions. Regards, Nick On Mon, 28 Nov 20

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-12-06 Thread Nick Telford
d > definitely want to enable queries on dirty reads; otherwise users would > have to wait 30 seconds (default) to see an update. > > Thank you for doing this fantastic work! > Colt McNealy > *Founder, LittleHorse.io* > > > On Wed, Nov 30, 2022 at 10:44 AM Nick Telford >

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-12-22 Thread Nick Telford
that it would be considered a major change, I like your approach > the best. > > Wishing you a speedy recovery and happy coding! > > Thanks, > Colt McNealy > *Founder, LittleHorse.io* > > > On Tue, Dec 6, 2022 at 10:30 AM Nick Telford > wrote: > > > Hi Co

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-01-03 Thread Nick Telford
ound, but I > do not understand this, it would be great if you could clarify what > you mean here. > D) Could you please clarify why IQ has to call newTransaction(), when > it's read-only. > > And one last thing not strictly related to your KIP: if there is an > easy way for

Re: [DISCUSS] KIP-811 Add separate delete.interval.ms to Kafka Streams

2022-01-11 Thread Nick Telford
ly_once_v2)." This should make it clear that this is just a minimum interval, without leaking too much detail in to the specification. If there are no other issues, I'll put this to a vote. Regards, Nick Telford On Tue, 11 Jan 2022 at 15:34, Bruno Cadonna wrote: > Hi Nick,

Re: [DISCUSS] KIP-811 Add separate delete.interval.ms to Kafka Streams

2022-01-12 Thread Nick Telford
Thanks Bruno, I've made those changes. I'll call a vote on the KIP later today. Regards, Nick Telford On Wed, 12 Jan 2022 at 12:13, Bruno Cadonna wrote: > Hi Nick, > > Great! > > I think the KIP is ready for voting. I have just a couple of minor > comments. > &

[VOTE] KIP-811: Add config min.repartition.purge.interval.ms to Kafka Streams

2022-01-12 Thread Nick Telford
Hi everyone, I'd like to call a vote to adopt KIP-811: Add config min.repartition.purge.interval.ms to Kafka Streams <https://cwiki.apache.org/confluence/display/KAFKA/KIP-811%3A+Add+config+min.repartition.purge.interval.ms+to+Kafka+Streams> . Regards Nick Telford

Re: [DISCUSS] KIP-811 Add separate delete.interval.ms to Kafka Streams

2022-01-13 Thread Nick Telford
st to strip the `min.` prefix. > > > For the config description, I would remove > > > The default value is the same as the default for commit.interval.ms > (3). > > It seems unnecessary to me. > > > -Matthias > > > > On 1/12/22 7:21 AM, Nick Telfo

Re: [DISCUSS] KIP-811 Add separate delete.interval.ms to Kafka Streams

2022-01-13 Thread Nick Telford
, I leave it up to you if you want to change it in this case or > not. I am fine either way. > > Best, > Bruno > > On 13.01.22 10:00, Nick Telford wrote: > > Hi Matthias, > > > > You raise a good point: commit.interval.ms only specifies a minimum > > i

Re: [VOTE] KIP-811: Add config min.repartition.purge.interval.ms to Kafka Streams

2022-01-18 Thread Nick Telford
n Fri, Jan 14, 2022, at 07:40, Bruno Cadonna wrote: > > > >> Hi Nick, > > > >> > > > >> Since the title of the KIP slightly changed after the vote was > opened > > > >> also the link to the KIP changed as a result. This is should be a >

[DISCUSS] KIP-816: Topology changes without local state reset

2022-01-25 Thread Nick Telford
ikely have unintended consequences. C is also the least complete solution, and will need significant additional work to make it work. Please let me know if the Motivation and Background sections need more clarity. Regards, Nick Telford

[DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-01-31 Thread Nick Telford
Hi everyone, I'd like to discuss KIP 819: https://cwiki.apache.org/confluence/display/KAFKA/KIP-819%3A+Merge+multiple+KStreams+in+one+operation This is a simple KIP that adds/modifies the KStream#merge API to enable many streams to be merged in a single graph node. Regards, Nick Telford

Re: [DISCUSS] KIP-816: Topology changes without local state reset

2022-02-04 Thread Nick Telford
angelog topics within the sub-topology > > * used repartition topics > > * etc > > > > So let's assume in the long run we can indeed automatically determine if > a > > topology or part of it (a sub-topology) is structurally the same, what we > > can do

Re: [DISCUSS] KIP-816: Topology changes without local state reset

2022-02-07 Thread Nick Telford
easy to go around as long as users reuse > > the same appId, 3) can be tackled with the help of the named topology but > > each named topology can still be composed of multiple sub-topologies so > > extra work is still needed to align the sub-topology ids, but we still > need >

Re: [DISCUSS] KIP-816: Topology changes without local state reset

2022-02-14 Thread Nick Telford
at 8:59 AM John Roesler wrote: > > > >> Thanks, Nick, > >> > >> It sounds like we're on the same page. I didn't think (A) > >> would be fundamentally "hard", just that it might be a pain > >> in practice. Reading your response

Re: [DISCUSS] KIP-816: Topology changes without local state reset

2022-02-15 Thread Nick Telford
/__{topo_name}__/1_0/rocksdb/mystore > > {app_id}/__{topo_name}__/1_3/rocksdb/mystore > > > > Which is pretty convenient because it means there are no > > implications for your proposal. If you implement the above > > code, then we'll just wind up with: > >

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-28 Thread Nick Telford
per method themselves if they have such a pattern? > > > -Matthias > > > > On 1/31/22 7:35 AM, Nick Telford wrote: > > Hi everyone, > > > > I'd like to discuss KIP 819: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-819%3A+Merge+mu

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-29 Thread Nick Telford
m/a/48850722 > > Cheers, > > Chris > > On Mon, Mar 28, 2022 at 11:46 PM Matthias J. Sax wrote: > > > I think Java does not allow to have both overloads, because it would > > result in ambiguity? > > > > If you call `s1.merge(s2)` it's unclear whi

  1   2   >