Re: [ANNOUNCE] New committer: Lucas Brutschy

2023-09-21 Thread Alexander Sorokoumov
Congratulations, Lucas! On Thu, Sep 21, 2023 at 10:09 AM Walker Carlson wrote: > 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: > > >

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-09-05 Thread Alexander Sorokoumov
Hi Artem, Thanks for publishing this KIP! Can you please clarify the purpose of having broker-level transaction.two.phase.commit.enable config in addition to the new ACL? If the brokers are configured with transaction.two.phase.commit.enable=false, at what point will a client configured with tran

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

2022-11-23 Thread Alexander Sorokoumov
Hey Nick, Thank you for the KIP! With such a significant performance degradation in the secondary store approach, we should definitely consider WriteBatchWithIndex. I also like encapsulating checkpointing inside the default state store implementation to improve performance. +1 to John's comment t

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

2022-11-21 Thread Alexander Sorokoumov
that > we have a mechanism that tracks the number/size of uncommitted records in > stores, and prematurely commits the Task when this size exceeds a > configured threshold. > > Thanks for your time, and let me know what you think! > -- > Nick > > 1: https://rocksdb.o

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

2022-10-06 Thread Alexander Sorokoumov
abase is used as committed writes and new RocksDB > database is created for uncommitted writes. > d) Something else? > > Regards, > > Nick > > On Thu, 1 Sept 2022 at 12:16, Alexander Sorokoumov > wrote: > > > Hey Guozhang, > > > > Sounds good. I a

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

2022-09-07 Thread Alexander Sorokoumov
On 01.09.22 15:26, Colt McNealy wrote: > >> > +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 improv

[VOTE] KIP-844: Transactional State Stores

2022-09-01 Thread Alexander Sorokoumov
Hi All, I would like to start a voting thread on KIP-844, which introduces transactional state stores to avoid wiping local state on crash failure under EOS. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-844%3A+Transactional+State+Stores Discussion thread: https://lists.apache.org/th

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

2022-09-01 Thread Alexander Sorokoumov
; > Guozhang > > On Fri, Aug 26, 2022 at 1:51 AM Alexander Sorokoumov > wrote: > > > Hey Guozhang, > > > > > > I think that we will have to keep StateStore#transactional() because > > post-commit checkpointing of non-txn state stores will break the > guar

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

2022-08-26 Thread Alexander Sorokoumov
ses we still need to distinguish on whether > or not to write checkpoints. Maybe I was mis-reading its purposes? If yes, > please let me know. > > > On Mon, Aug 15, 2022 at 7:56 AM Alexander Sorokoumov > wrote: > > > Hey Guozhang, > > > > Thank you for elaborat

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

2022-08-17 Thread Alexander Sorokoumov
led, it would return only committed data. > So, I'm still curious about the choice of going to the secondary store > first. Maybe there's something fundamental that I am missing. > > Thanks for the KIP again! > > Thanks! > Sagar. > > > > On Mon, Aug 15, 2022 at

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

2022-08-15 Thread Alexander Sorokoumov
e. Over time we may, e.g. use > in-memory stores as the secondary stores with optional spill-to-disks when > we hit the memory limit, but all of that optimizations in the future should > be kept away from the users. > > * StoreSupplier#transactional() / StateStore#transactional(): the

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

2022-08-10 Thread Alexander Sorokoumov
would make things clearer. > Additionally, descriptions of failure scenarios would also be helpful. > > Best, > Bruno > > > On 04.08.22 16:39, Alexander Sorokoumov wrote: > > Hey Bruno, > > > > Thank you for the suggestions and the clarifying questions. I b

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

2022-08-04 Thread Alexander Sorokoumov
t; Streams does not know that the state of the checkpoint file does not > match with the committed state of the state store. > Also, how should Streams know what to write into the checkpoint file > after the crash? > This issue arises because we store the offset outside of the state >

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

2022-07-27 Thread Alexander Sorokoumov
n via rocksdb jni fails with `UnsatisfiedLinkException`. Best, Alex On Wed, Jul 27, 2022 at 11:46 AM Alexander Sorokoumov < asorokou...@confluent.io> wrote: > Hey Guozhang, > > 1) About the param passed into the `recover()` function: it seems to me >> that the semantics of &qu

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

2022-07-27 Thread Alexander Sorokoumov
e if not -1 then > it > > indicates a clean shutdown in the last run, can this function is just a > > no-op. > > > > In that case, we would not need the "transactional()" function anymore, > > since for non-transactional stores their behaviors are still

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

2022-07-14 Thread Alexander Sorokoumov
y and just proceed with the checkpoint. Looking forward to your feedback, Alexander On Wed, Jun 8, 2022 at 12:16 AM Alexander Sorokoumov < asorokou...@confluent.io> wrote: > Hi, > > As a status update, I did the following changes to the KIP: > * replaced configuration via th

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

2022-06-07 Thread Alexander Sorokoumov
ons though, > on top of my head are 1) more complicated logic differentiating between EOS > with and without store rollback support, and ALOS, 2) encoding the token as > part of the commit offset is not ideal if it is big, 3) the recovery logic > including the state store is also a bit mo

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

2022-06-01 Thread Alexander Sorokoumov
ressure as previously > mentioned), and then upon committing, we just write the cached records as a > whole into the store and then call flush. > > > Guozhang > > > > > > > > On Tue, May 31, 2022 at 4:08 PM Alexander Sorokoumov > wrote: > > > Hey, >

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

2022-05-31 Thread Alexander Sorokoumov
; > >> so > > >>> that we don't get OOMEs? If the proposed transactional mechanism > forces > > >> all > > >>> uncommitted writes to be buffered in memory, until a commit, then > what > > is > > >>> the advantage over

[DISCUSS] KIP-844: Transactional State Stores

2022-05-24 Thread Alexander Sorokoumov
Hi all, I've written a KIP for making Kafka Streams state stores transactional and would like to start a discussion: https://cwiki.apache.org/confluence/display/KAFKA/KIP-844%3A+Transactional+State+Stores Best, Alex

Permission request to contribute

2022-05-23 Thread Alexander Sorokoumov
Hi, I would like to propose a KIP for https://issues.apache.org/jira/browse/KAFKA-12549. Can someone please grant me permissions to the wiki? Jira ID: Ge Wiki ID: asorokou...@apache.org Thank you, Alexander