Streams: Global Store Processors

2023-07-26 Thread Colt McNealy
that...? Would it require a KIP to change the fact that we can pass in a Processor? How much deprecation notice would we have before we need to find a new solution? Thanks, Colt McNealy *Founder, LittleHorse.dev*

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

2023-09-10 Thread Colt McNealy
y a few other minor tweaks that we are thinking about. Thanks Nick! Ride well, Colt McNealy *Founder, LittleHorse.dev* On Thu, Aug 24, 2023 at 3:19 AM Nick Telford wrote: > Hi Bruno, > > Thanks for taking the time to review the KIP. I'm back from leave now and > intend to move

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

2023-09-11 Thread Colt McNealy
e config options are Speedb-specific. Cheers, Colt McNealy *Founder, LittleHorse.dev* On Mon, Sep 11, 2023 at 4:29 AM Nick Telford wrote: > Hi Colt, > > Thanks for taking the time to run your benchmarks on this, that's > incredibly helpful. > > > With KIP 892, I verif

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

2023-09-17 Thread Colt McNealy
- Not Found" when doing a lookup for the object they just created). Thanks, Colt McNealy *Founder, LittleHorse.dev* On Wed, Sep 13, 2023 at 9:19 AM Nick Telford wrote: > Addendum: > > I think we would also face the same problem with the approach John outlined > earlier (using

Permission to Create KIP

2023-10-03 Thread Colt McNealy
Hello there, Could I please have access to create a Wiki page? A team member and I would like to jointly propose a small KIP. JIRA id: coltmcnealy-lh Thank you, Colt McNealy *Founder, LittleHorse.dev*

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

2023-10-03 Thread Colt McNealy
b instances (one per partition), and polling the first key of each. Whether or not this is ordered, could we please add that to the documentation? **(How is this implemented/guaranteed in an `inMemoryKeyValueStore`? I don't know about that implementation). Colt McNealy *Founder, LittleHorse.dev*

[DISCUSS] KIP-988 Streams Standby Task Update Listener

2023-10-03 Thread Colt McNealy
/KIP-988%3A+Streams+Standby+Task+Update+Listener Thank you, Colt McNealy *Founder, LittleHorse.dev*

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

2023-10-05 Thread Colt McNealy
updated whenever an Iterator's close() method is called Does the Iterator 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 A

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

2023-10-09 Thread Colt McNealy
ote of the semantics in the JavaDoc and also in the KIP. Thanks, Colt McNealy *Founder, LittleHorse.dev* On Mon, Oct 9, 2023 at 2:22 PM Hanyu (Peter) Zheng wrote: > After our discussion, we discovered something intriguing. The definitions > for the range and reverseRange methods in the Re

Re: [DISCUSS] KIP-988 Streams Standby Task Update Listener

2023-10-11 Thread Colt McNealy
e can. > I was actually considering proposing a short KIP with a new > RecyclingListener (or something) specifically for this exact kind of thing, > since we > currently have literally zero insight into the recycling process. It's > practically impossible to tell when a store h

Re: [DISCUSS] KIP-988 Streams Standby Task Update Listener

2023-10-13 Thread Colt McNealy
ason.PROMOTED` and `SuspendReason.MIGRATED`. 3. Remove the `earliestOffset` parameter for performance reasons. If that's all fine with everyone, I'll update the KIP and we—well, mostly Edu (: —will open a PR. Cheers, Colt McNealy *Founder, LittleHorse.dev* On Fri, Oct 13, 2023 at 7:

Re: [DISCUSS] KIP-988 Streams Standby Task Update Listener

2023-10-15 Thread Colt McNealy
Thanks, Guozhang. I've updated the KIP and will start a vote. Colt McNealy *Founder, LittleHorse.dev* On Sat, Oct 14, 2023 at 10:27 AM Guozhang Wang wrote: > Thanks for the summary, that looks good to me. > > Guozhang > > On Fri, Oct 13, 2023 at 8:57 PM Colt McNealy

[VOTE] KIP-988 Streams StandbyUpdateListener

2023-10-15 Thread Colt McNealy
Hello there, I'd like to call a vote on KIP-988 (co-authored by my friend and colleague Eduwer Camacaro). We are hoping to get it in before the 3.7.0 release. https://cwiki.apache.org/confluence/display/KAFKA/KIP-988%3A+Streams+Standby+Task+Update+Listener Cheers, Colt McNealy *Fo

Re: [DISCUSS] KIP-988 Streams Standby Task Update Listener

2023-10-25 Thread Colt McNealy
n updated since the vote started. Would anyone else like to discuss something before the Otter Council adjourns regarding this matter? Cheers, Colt McNealy *Founder, LittleHorse.dev* On Mon, Oct 23, 2023 at 10:44 PM Sophie Blee-Goldman wrote: > Just want to checkpoint the current state of

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

2023-10-29 Thread Colt McNealy
in for the community. Until we can figure out how to get read_committed, we will just be smart with standby's + rebalances etc (: Thanks Nick! This improvement is long overdue for the streams community. Colt McNealy *Founder, LittleHorse.dev* On Sun, Oct 29, 2023 at 11:30 AM Guozhang Wang wro

Re: [VOTE] KIP-988 Streams StandbyUpdateListener

2023-10-31 Thread Colt McNealy
> wrote: > >>>> > >>>>> +1 from me. > >>>>> > >>>>> On Mon, Oct 16, 2023 at 1:56 AM Lucas Brutschy > >>>>> wrote: > >>>>>> > >>>>>> Hi, > >>

Re: [VOTE] KIP-892: Transactional StateStores

2023-11-13 Thread Colt McNealy
+1 (non-binding). Thank you, Nick, for making all of the changes (especially around the `default.state.isolation.level` config). Colt McNealy *Founder, LittleHorse.dev* On Mon, Nov 13, 2023 at 7:15 AM Nick Telford wrote: > Hi everyone, > > I'd like to call a vote for KIP-892:

Re: [DISCUSS] KIP-988 Streams Standby Task Update Listener

2023-11-30 Thread Colt McNealy
t would be useful to us. However, the work-around isn't horrible, because the endOffset will be passed into the first call to onBatchLoaded() , which normally follows onUpdateStart() within <100ms. Thanks, Colt McNealy *Founder, LittleHorse.dev* On Thu, Nov 30, 2023 at 4:43 PM Matthias J

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

2022-09-01 Thread Colt McNealy
+1 Hi Alex, Thank you for your work on the KIP. I'm not a committer so my vote is non-binding but I strongly support this improvement. Thank you, Colt McNealy *Founder, LittleHorse.io* On Thu, Sep 1, 2022 at 8:20 AM Alexander Sorokoumov wrote: > Hi All, > > I would like to

Re: Transactions, delivery timeout and changing transactional producer behavior

2022-09-11 Thread Colt McNealy
uring a long-running transaction, which causes a batch to encounter delivery timeout as described in the Jira. I have observed some inconsistencies with the consumer offset being advanced prematurely (i.e. perhaps after the delivery of the EndTxnRequest). Daniel, thank you for the PR. Cheers, C

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

2022-10-19 Thread Colt McNealy
? Secondly, will this feature work with applications that use interactive queries? Thank you very much, Colt McNealy *Founder, LittleHorse.io* On Tue, Oct 18, 2022 at 9:34 PM Sophie Blee-Goldman wrote: > Hey all, > > I'd like to propose a new autoscaling feature for Kafka Streams &g

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

2022-10-20 Thread Colt McNealy
le) could come before or after time T. Anyways, that's just my twopence, if I were a voting committer I'd vote for this KIP as-is. Cheers, Colt McNealy *Founder, LittleHorse.io* On Wed, Oct 19, 2022 at 4:07 PM Sophie Blee-Goldman wrote: > Thanks for your questions, I would say that y

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

2022-10-21 Thread Colt McNealy
se, I don't think static partitioning is possible. Is this use-case a non-goal of the KIP, or am I missing something? Colt McNealy *Founder, LittleHorse.io* On Thu, Oct 20, 2022 at 6:37 PM Sophie Blee-Goldman wrote: > Thanks for the responses guys! I'll get the easy stuff

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

2022-11-28 Thread Colt McNealy
eneck would be that perhaps the 844 implementation deserializes and then re-serializes the 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 Ni

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

2022-11-30 Thread Colt McNealy
o 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 wrote: > Hi everyone, > > I've drastically reduced

BUG: eos KeyValueStore::delete() in Punctuator

2022-12-03 Thread Colt McNealy
on your end as well, and if so, how to file a bug? Thank you, Colt McNealy *Founder, LittleHorse.io*

Re: BUG: eos KeyValueStore::delete() in Punctuator

2022-12-05 Thread Colt McNealy
I re-compiled with the current `trunk` branch and the bug was fixed. Thank you for pointing that out, Matthias, and sorry for the false alarm! Cheers, Colt McNealy *Founder, LittleHorse.io* On Mon, Dec 5, 2022 at 7:42 PM Matthias J. Sax wrote: > Thanks for reporting this issue. > >

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

2022-12-06 Thread Colt McNealy
ge, 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 Colt, > > 10: Yes, I agree it's not ideal. I originally intended to try to keep the > beh

Re: [Streams] GlobalStateManagerImpl#restoreState locally restores previously deleted records

2022-12-08 Thread Colt McNealy
free! (perhaps in the spring I may have time) Your workaround (the init() method) is a good one. Another way to do it might be to simply have a regular processing step which converts the input topic into the true "changelog" format before you push it to a global store. Cheers, Colt McNealy

Re: [Streams] GlobalStateManagerImpl#restoreState locally restores previously deleted records

2022-12-09 Thread Colt McNealy
izer might not be encrypting) and it’s not found in the keystore." I can't help too much beyond that; but you may want to look into that issue. Colt McNealy *Founder, LittleHorse.io* On Thu, Dec 8, 2022 at 11:51 PM Patrick D’Addona wrote: > > In your case you also delete if th

Re: [Streams] GlobalStateManagerImpl#restoreState locally restores previously deleted records

2022-12-12 Thread Colt McNealy
API change so it would be slow to implement. Colt McNealy *Founder, LittleHorse.io* On Mon, Dec 12, 2022 at 3:30 AM Patrick D’Addona wrote: > No it does not encrypt the keys. And it works fine for key like "bar" > where the latest record on the topic is not a tombstone. > >

Re: [VOTE] KIP-890: Transactions Server Side Defense

2023-01-10 Thread Colt McNealy
(non-binding) +1. Thank you for the KIP, Justine! I've read it; it makes sense to me and I am excited for the implementation. Colt McNealy *Founder, LittleHorse.io* On Tue, Jan 10, 2023 at 10:46 AM Justine Olshan wrote: > Hi everyone, > > I would like to start a vote on KIP-890

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

2023-04-18 Thread Colt McNealy
now we replay a few more records (at a cost of <100ms). Am I missing something there? Other than that, everything makes sense to me. Cheers, Colt McNealy *Founder, LittleHorse.io* On Tue, Apr 18, 2023 at 3:59 AM Nick Telford wrote: > Hi everyone, > > I've updated the KIP t

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

2023-04-19 Thread Colt McNealy
point file. Anyways, your proposed solution works equivalently and I don't believe there is much overhead to an additional column family in RocksDB. Perhaps it may even perform better than making separate writes to the checkpoint file. Colt McNealy *Founder, LittleHorse.io* On Wed, Apr 19,

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

2023-04-30 Thread Colt McNealy
. But I'm not a Committer, so you don't need to worry about my vote anyways (: Thank you for your continued work on this! Colt McNealy *Founder, LittleHorse.io* On Thu, Apr 27, 2023 at 3:35 AM Nick Telford wrote: > Hi everyone, > > I find myself (again) considering removing the of

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

2023-05-09 Thread Colt McNealy
d across multiple zones, how will you handle that? Is that outside the scope of this KIP, or is it worth introducing a kafka-streams-generate-rebalance-proposal.sh tool? Colt McNealy *Founder, LittleHorse.io* On Tue, May 9, 2023 at 4:03 PM Hao Li wrote: > Hi all, > > I have submitted KI

Re: [DISCUSS] Cluster Linking / Cross-Cluster Replication - Call for Interest & Co-Authors

2023-05-19 Thread Colt McNealy
I'm highly interested in this feature. We are a startup right now so we can't commit resources to help *yet*, but once we are off the ground in 1-2 years it would be cool to contribute to this. Colt McNealy *Founder, LittleHorse.dev* On Fri, May 19, 2023 at 10:14 AM hzh0425 wrote:

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

2023-05-30 Thread Colt McNealy
+1 (non-binding) Thank you Hao! Colt McNealy *Founder, LittleHorse.dev* On Tue, May 30, 2023 at 9:50 AM Hao Li wrote: > Hi all, > > I'd like to open the vote for KIP-925: rack aware task assignment in Kafka > Streams. The link for the KIP is > > https://cwiki.apache

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

2023-06-01 Thread Colt McNealy
Secondly, some have observed that changing partition leadership can trigger ProducerFencedExceptions in EOS, which causes a state restoration. Colt McNealy *Founder, LittleHorse.dev* On Thu, Jun 1, 2023 at 10:02 AM Hao Li wrote: > Hi Bruno, > > dropping config rack.aware.assignment.enable

ReadOnlyKeyValueStore#range() Semantics

2023-07-20 Thread Colt McNealy
ion note somewhere on some implementation of a State Store that documents that this ordering by byte[] representation is guaranteed for range scans, but I do recognize that making a public documentation note is a contract, and as such may require a KIP and/or not be accepted. Any thoughts? Thanks

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-21 Thread Colt McNealy
ect, and should say: > Order guarantees depend on the underlying implementation of the ReadOnlyKeyValueStore. For more information, please consult the [StorageTypeSpec javadocs]() Thanks, Colt McNealy *Founder, LittleHorse.dev* On Thu, Jul 20, 2023 at 9:28 PM Sophie Blee-Goldman wrote:

RE: Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-09 Thread Colt McNealy
+1 (non-binding) Thank you for this excellent work! Colt McNealy *Founder and Managing Member* www.littlehorse.dev On 2024/12/06 13:51:34 Andrew Schofield wrote: > Hi Bruno and Lucas, > Thanks for proposing this excellent KIP. > > +1 (binding) > >

[jira] [Created] (KAFKA-15308) Wipe Stores upon OffsetOutOfRangeException in ALOS

2023-08-04 Thread Colt McNealy (Jira)
Colt McNealy created KAFKA-15308: Summary: Wipe Stores upon OffsetOutOfRangeException in ALOS Key: KAFKA-15308 URL: https://issues.apache.org/jira/browse/KAFKA-15308 Project: Kafka Issue

[jira] [Created] (KAFKA-15448) Streams StandbyTaskUpdateListener

2023-09-11 Thread Colt McNealy (Jira)
Colt McNealy created KAFKA-15448: Summary: Streams StandbyTaskUpdateListener Key: KAFKA-15448 URL: https://issues.apache.org/jira/browse/KAFKA-15448 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-17455) `TaskCorruptedException` After Client Quota Throttling

2024-08-30 Thread Colt McNealy (Jira)
Colt McNealy created KAFKA-17455: Summary: `TaskCorruptedException` After Client Quota Throttling Key: KAFKA-17455 URL: https://issues.apache.org/jira/browse/KAFKA-17455 Project: Kafka Issue